You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/08/26 09:19:00 UTC

[GitHub] [pulsar] MathiasHaudgaard opened a new pull request #11794: Upstream/jwks feature

MathiasHaudgaard opened a new pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794


   ### Motivation
   
   This PR adds support for JWKS in the AuthenticationProviderToken class. The reason why I didn't create a new plugin for this, is because I think it's a standard and it's really common to use.
   
   ### Modifications
   
   I've changed the AuthenticationProviderToken class so you're able to enable JWKS through the conf files.
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): yes
     - The public API: no
     - The schema: no
     - The default values of configurations: yes
     - The wire protocol: no
     - The rest endpoints: no
     - The admin cli options: no
     - Anything that affects deployment: don't know
   
   ### Documentation
   
   You can try it out by setting jwkEnabled=true in standalone.conf and then also set the jwkUrl=[the URL for the JWKS]
   
   ### HELP NEEDED
   
   I can get this to work for the standalone version but when I try to use it in a cluster, it doesn't quite work. I hope someone can help me with this PR so we can merge the JWKS functionality. 😄  :octocat: 
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r699072086



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }

Review comment:
       Not quite sure what you mean here. Why would you ever want to store  `AuthenticationProviderTokenJWK` and `AuthenticationProviderToken` on the same broker? :shipit: 
   
   I used the org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 plugin to test the implementation 😄 

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       I'll look into this. Thanks 👍  🔍 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-909041052


   > What about creating a separate Maven Module and use the Maven shade plugin to include all the third party deps in the .jar file for this Provider ?
   > This way we are not bundling the provider out of the box but we are adding it to the build.
   > Users will be able to pick the jar file and put it into "lib" in order to be able to use it
   > So zero impact to existing users and especially to other users who built something like this and need other versions of the same libraries
   
   I can try this out if it's necessary. @cckellogg @michaeljmarshall I would like your feedback on this on as well 😄 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r707259103



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }
+            //extracting the jwks_uri
+            JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+            this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       well, I think HS256 is also a valid alg type even though rs256 is the default 😄 
   
   I'll implement the feature 👍 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r707241488



##########
File path: distribution/server/src/assemble/LICENSE.bin.txt
##########
@@ -642,3 +645,11 @@ interchange format, which can be obtained at:
    * licenses/LICENSE-protobuf.txt (New BSD License)
  * HOMEPAGE:
 * https://github.com/google/protobuf
+
+Contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data

Review comment:
       is this due to Oauth0 ? in this case please add a reference

##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,23 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>com.microsoft.azure</groupId>

Review comment:
       is this still required ?
   in this case we have to deal with LICENSE and NOTICE, and also please declare the dependency as property as you did for auth0




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] github-actions[bot] commented on pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-1058886988


   @MathiasHaudgaard:Thanks for your contribution. For this PR, do we need to update docs?
   (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r706099559



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);

Review comment:
       Thanks. This is not needed for this plugin. I'll delete it 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r707244104



##########
File path: distribution/server/src/assemble/LICENSE.bin.txt
##########
@@ -642,3 +645,11 @@ interchange format, which can be obtained at:
    * licenses/LICENSE-protobuf.txt (New BSD License)
  * HOMEPAGE:
 * https://github.com/google/protobuf
+
+Contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data

Review comment:
       whoops, that was a mistake. Sorry




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-909041052


   > What about creating a separate Maven Module and use the Maven shade plugin to include all the third party deps in the .jar file for this Provider ?
   > This way we are not bundling the provider out of the box but we are adding it to the build.
   > Users will be able to pick the jar file and put it into "lib" in order to be able to use it
   > So zero impact to existing users and especially to other users who built something like this and need other versions of the same libraries
   
   I can try this out if it's necessary. @cckellogg @michaeljmarshall I would like your feedback on this on as well 😄 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r697171555



##########
File path: conf/standalone.conf
##########
@@ -462,7 +468,7 @@ tokenSecretKey=
 # The key can be specified like:
 # tokenPublicKey=data:;base64,xxxxxxxxx
 # tokenPublicKey=file:///my/public.key    ( Note: key file must be DER-encoded )
-tokenPublicKey=
+tokenPublicKey=file:///Users/dkMatHau/pulsar/oauth0.key

Review comment:
       Sorry. Didn't see that one 😅 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r704805019



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }

Review comment:
       It may be that the oauth2 service built by some companies does not open https, this should be an optional




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r706100586



##########
File path: conf/standalone.conf
##########
@@ -424,6 +424,9 @@ authenticationProviders=
 # Enforce authorization
 authorizationEnabled=false
 
+# issuer url that points openid-configurations from a third party provider
+issuerUrl=

Review comment:
       maybe OIDCIssuerUrl?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698503213



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
##########
@@ -237,6 +280,69 @@ private static String validateToken(final String token) throws AuthenticationExc
         }
     }
 
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateTokenJwk(final String token) throws AuthenticationException {

Review comment:
       @cckellogg I added the feature as a separate plugin instead. This gets rid of most of the if else statements and the jwkEnable flag which makes it a bit cleaner 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-912267604


   Thanks for your response @eolivelli.
   
   > Probably having some high level description of this feature, would help in understanding why this feature is so useful and why we should add it to Pulsar core.
   
   Yes, I think we need more definition for the problem that @MathiasHaudgaard would like to solve with this PR.
   
   >  Maybe it is just me, that I do not know **JWKS** and so I do not see the value in adding it to the core distribution, but I see more third party dependencies added to the core distribution.
   
   The `JWKs` data structure is defined in this RFC: https://datatracker.ietf.org/doc/html/rfc7517. The data structure can hold several kinds of cryptographic information, like public keys or private keys. Here is a helpful quote from the introduction:
   
   > JWKs and JWK Sets are used in the JSON Web Signature [JWS] and JSON Web Encryption [JWE] specifications.
   
   I don't know all of the use cases of the JWK and JWKs data structure. I know it is used in the OpenID Connect protocol, an extension of the OAuth2.0 protocol, to store public keys that can be used to verify the signature on a JWT. An Identity Provider implementing the OIDC protocol will issue JWTs for an authenticated entity. These tokens will be signed by the Identity Provider using a private key. The Identity Provider will serve the paired Public Keys at an endpoint on the server to make it easy to verify token signatures.
   
   The PR currently uses the JWKs endpoint to retrieve a single Identity Provider's public keys to verify the signature of the JWT.
   
   For reference, the current `AuthenticationProviderToken` class verifies a JWT in the same way that this new class verifies a token (although each uses different libraries). The `AuthenticationProviderToken` class relies on a public key that is available in to the broker's file system. This implementation is limited because it allows for only one active public key, which means that to rotate keys requires downtime: first to distribute new tokens signed by a new private key and then to update the paired public key in the broker.
   
   By adding support to retrieve a `JWKs` from an Identity Provider, we will gain the ability to have multiple valid public keys at the same time. That means it'll be possible to rotate keys without downtime. Further, because the public keys are discovered, operators will be able to remove public keys in the event that some part of the chain were compromised.
   
   Ultimately, this PR would make it much easier for end users to integrate with an independent Identity Provider service, which is a great feature. However, from my perspective, I think this PR does not go far enough to implement a specific authentication protocol. I know that the JWKS endpoint is used in OpenID Connect. I'm not sure if it is used in any other authentication protocols. I think I would prefer to see a complete implementation of the OpenID Connect protocol (or some other protocol) instead of a partial implementation.
   
   The main features that I currently see missing in this implementation (assuming we want to implement the OpenID Connect protocol):
   1. Discover the issuer's `jwks_uri` by retrieving it at the issuer's `/.well-known/openid-configuration` endpoint (https://openid.net/specs/openid-connect-discovery-1_0.html#ProviderMetadata)
   2. Implement configurable caching of public keys (https://openid.net/specs/openid-connect-core-1_0.html#RotateEncKeys)
   3. The trusted issuers should be required to use `https`, by default (it could be optional to override this requirement). This is technically required in the protocol: https://openid.net/specs/openid-connect-discovery-1_0.html.
   4. (Optional) Allow for multiple issuers to be considered "allowed" or "trusted". This is convenient if you're running your identity provider in the same kubernetes cluster due to the way that k8s DNS works. (This component could be debatable, but in the event that we implement number 1, it'd be pretty easy to allow for this implementation too.)
   5. (Optional) Allow for more algorithms than just `RS256`. Available algorithms are defined here: https://datatracker.ietf.org/doc/html/rfc7518#section-3.1. (This would be easy to add later, so it doesn't need to be a priority now.)
   
   If we do implement the OpenID Connect protocol (or even some other protocol), I think it would certainly deserve to be in the core distribution.
   
   @eolivelli - I think the question of which dependencies to use is a valid one that needs to be answered. I am not sure of the right answer, but I do think it'd be worth adding dependencies to get full support of the OpenID Connect protocol. The current 3rd party libraries from `auth0` do not support retrieving the the provider metadata from the `/.well-known/openid-configuration` endpoint. Further, the `auth0` library does not use nonblocking asynchronous IO. It'd be great if we could find a library that offers this. I'm not sure if one of these exists for OIDC.
   
   Finally, I think we should resolve the asynchronous authentication provider issue that I raised on the dev mailing list before we can merge this PR. Otherwise, any network calls made by this provider will block netty event loop threads.
   
   @MathiasHaudgaard - what do you think about my comments? Do you agree that this is meant to implement the OpenID Connect protocol or were you seeking to implement something different? I am not an expert on uses of the JWKS, so your insight will be helpful.
   
   @eolivelli - I hope my comment has helped explain some of the context for this PR and its significance.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-918221173


   > I'm not quite sure how to test this plugin since it depends on a third-party provider. Any ideas? 😄
   it is possible to create a mock server with WireMock ?
   otherwise we could (as last resort) use PowerMock to override the behaviour of Auth0 library 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-918168242


   > I see that we are missing tests.
   
   @eolivelli  I'm not quite sure how to test this plugin since it depends on a third-party provider. Any ideas? 😄 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r706041052



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);

Review comment:
       CONF_TOKEN_SETTING_PREFIX is an empty string. is there some bug here ?

##########
File path: conf/standalone.conf
##########
@@ -424,6 +424,9 @@ authenticationProviders=
 # Enforce authorization
 authorizationEnabled=false
 
+# issuer url that points openid-configurations from a third party provider
+issuerUrl=

Review comment:
       What about adding some prefix here or some better name that gives a context ?
   otherwise this name is not much meaningful if you pick it separated from the context.
   This entry is useful only for the new Auth provider

##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       di you think that we could remove "io.jsonwebtoken" and use everywhere this new library ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698949243



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       The two `auth0` libraries were likely chosen because they have support for retrieving and deserializing `JWKS` (Json Web Key Sets) objects. The `io.jsonwebtoken` libraries currently used by the `AuthenticationProviderToken` do not have this support.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r715749203



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";

Review comment:
       I would suggest that audience validation is not always performed in OIDC-based authentication.  OIDC tokens typically have an audience that is equal to the subject (because the token is obtained by, and primarily intended for, the user themself).   Am not objecting to having audience validation, but is an edge case.
   
   Audience validation is typical in OAuth 2.0 authentication, which this provider is assumedly not intended for.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-910281308


   > I don't think I agree with making this its own module. If we did, I'd wonder why it wasn't in its own repository, or some kind of Pulsar Contrib repository. I don't have a fully formed opinion, though. I can provider more feedback tomorrow.
   
   @michaeljmarshall we have other components that are in the repo and that are not released in the "lib" directory, for instance the offloaders.
   
   If this is meant to be a core feature, available out of the box, then it is fine to put it in "lib", otherwise it is better to create a separate module and let users add it to their system if they want.
   
   My point here is about not adding "optional" modules to the core distribution.
   
   Probably having some high level description of this feature, would help in understanding why this feature is so useful and why we should add it to Pulsar core.
   Maybe it is just me, that I do not know **JWKS** and so I do not see the value in adding it to the core distribution, but I see more third party dependencies added to the core distribution.
   
   Please note that I am not against this feature, I am sure it is useful. I am concerned only about the distribution of this code and the dependencies to the end users
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698505866



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       is it really necessary to add all of these third party deps ?
   I wonder if it won't be better to create a separate module and let users drop the jar in pulsar/lib
   
   if we want to go this way we have to verify all the licenses/NOTICE files and list the artifacts in https://github.com/apache/pulsar/tree/master/distribution/server/src/assemble and probably in https://github.com/apache/pulsar/tree/master/distribution/server/licenses 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r704292360



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }

Review comment:
       @michaeljmarshall not sure where I should check for https. So far I've put it here, but I would like your suggestion. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] hangc0276 commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r703109279



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;

Review comment:
       Please avoid to use `import xxx.*`

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");

Review comment:
       Please put the exception stack into the exception message.

##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>com.microsoft.azure</groupId>
+      <artifactId>msal4j</artifactId>
+      <version>1.11.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.auth0</groupId>
+      <artifactId>jwks-rsa</artifactId>
+      <version>0.11.0</version>
+    </dependency>
+
+    <!-- https://mvnrepository.com/artifact/com.auth0/java-jwt -->
+    <dependency>
+        <groupId>com.auth0</groupId>
+        <artifactId>java-jwt</artifactId>
+        <version>3.10.3</version>

Review comment:
       Please don't hard code version into this pom, you can use properties in root pom.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+

Review comment:
       Please delete the blank lines




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r718759888



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";

Review comment:
       My preference would be to not reuse the configuration parameters that are designed for a different auth provider, unless this provider is designed to emulate or subsume it.  




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-913375508


   > @MathiasHaudgaard - what do you think about my comments? Do you agree that this is meant to implement the OpenID Connect protocol or were you seeking to implement something different? I am not an expert on uses of the JWKS, so your insight will be helpful.
   
   @michaeljmarshall, I think you're absolutely right. My knowledge of OIDC is a bit deficient, so I'll look into how I can implement it. Thanks for the feedback! 😃 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r716506729



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";

Review comment:
       > I would suggest this configuration property be named `tokenSubjectClaim` 
   
   If I change the name, I need to add another configuration parameter in the conf files, which I'm not a big fan of. `tokenAuthClaim` is already being used by the AuthenticationProviderToken class so instead of adding a new parameter, we can just use the same. I think it makes more sense imo 😄 
   
   > and that its default value be `sub`.
   
   it already is: L271
   
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-916893926


   > https://github.com/auth0/auth0-java/blob/master/LICENSE
   
   This is very good.
   You will probably see that CI will fail about the LICENSE check, because we list all third party libs here:
   https://github.com/apache/pulsar/tree/master/distribution/server/src/assemble
   and here
   https://github.com/apache/pulsar/tree/master/distribution/server/licenses


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r715742548



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+        if(!url.getProtocol().equals("https")){
+            throw new MalformedURLException("protocol needs to be https");
+        }
+        //extracting the jwks_uri
+        JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+        this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");

Review comment:
       Please rephrase, e.g. ", please also configure Audience".
   
   Even better, don't perform this check and allow an audience claim to be configured without a corresponding audience.  Simply do audience validation only if an audience is configured.  In that way, the audience claim may have a default configuration of `aud` without implying that audience validation is enabled.
   

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";

Review comment:
       I would suggest that audience validation is not typically performed in OIDC-based authentication.  OIDC tokens typically have an audience that is equal to the subject (because the token is obtained by, and primarily intended for, the user themself).   Am not objecting to having audience validation, but is an edge case.
   
   Audience validation is typical in OAuth 2.0 authentication, which this provider is assumedly not intended for.
   

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+        if(!url.getProtocol().equals("https")){
+            throw new MalformedURLException("protocol needs to be https");
+        }
+        //extracting the jwks_uri
+        JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+        this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;

Review comment:
       nitpit: don't declare variables upfront.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+        if(!url.getProtocol().equals("https")){
+            throw new MalformedURLException("protocol needs to be https");
+        }
+        //extracting the jwks_uri
+        JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+        this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);
+            algorithm.verify(jwt); // if the token signature is invalid, the method will throw SignatureVerificationException
+
+            if (audienceClaim != null) {
+                Object object = jwt.getClaim(audienceClaim);
+
+                if (object == null) {
+                    throw new JwtException("Found null Audience in token, for claimed field: " + audienceClaim);
+                }
+
+                if (object instanceof List) {
+                    List<String> audiences = (List<String>) object;
+                    // audience not contains this broker, throw exception.
+                    if (audiences.stream().noneMatch(audienceInToken -> audienceInToken.equals(audience))) {
+                        throw new AuthenticationException("Audiences in token: [" + String.join(", ", audiences)
+                                + "] not contains this broker: " + audience);
+                    }
+                } else if (object instanceof String) {
+                    if (!object.equals(audience)) {
+                        throw new AuthenticationException("Audiences in token: [" + object
+                                + "] not contains this broker: " + audience);

Review comment:
       Regarding the error message, `not contains this broker`, don't assume that the audience represents the broker because that's an oauth2 thing.  It might simply represent the allowed users.
   
   To be honest, if you want to support audience validation in an OIDC scenario, the broker configuration should accept a list of allowed audiences.
   

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");

Review comment:
       Please don't formulate the URL in this way.  The issuer URL is likely to have a trailing slash.  Notice how the `issuer` property in a typical OpenID Configuration document has a trailing slash ([example](https://auth.streamnative.cloud/.well-known/openid-configuration)).  Some libraries even validate that the `issuer` property is equal to the configured issuer URL.
   
   Please use an approach like:
   ```
   URL issuerUrl = getIssuerUrl(config);
   URL configUrl = new URL(issuerUrl, "/.well-known/openid-configuration");
   ```
   

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";

Review comment:
       Not used - `CONF_TOKEN_PUBLIC_ALG`? 

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }
+            //extracting the jwks_uri
+            JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+            this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       May I suggest that the `io.jsonwebtoken` library be used for token validation.  Pulsar already has the dependency. This will take care of applying the correct algorithm based on the provided key.  
   
   It is easy to use, if you implement `io.jsonwebtoken.SigningKeyResolver` as a wrapper over the `UrlJwkProvider`.  Then you can instantiate a `io.jsonwebtoken.JwtParser` that correctly validates the token.
   
   ```
   this.jwtParser = Jwts.parserBuilder()
                   .setSigningKeyResolver(signingKeyResolver)
                   .requireIssuer(issuerUrl)
                   .build();
   ...
   Jws<Claims> jwt = this.jwtParser.parseClaimsJws(tokenString);
   // perform app-specific validations, e.g audience
   ```
   

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+        if(!url.getProtocol().equals("https")){
+            throw new MalformedURLException("protocol needs to be https");
+        }
+        //extracting the jwks_uri
+        JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+        this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);
+            algorithm.verify(jwt); // if the token signature is invalid, the method will throw SignatureVerificationException
+
+            if (audienceClaim != null) {
+                Object object = jwt.getClaim(audienceClaim);
+
+                if (object == null) {
+                    throw new JwtException("Found null Audience in token, for claimed field: " + audienceClaim);
+                }
+
+                if (object instanceof List) {
+                    List<String> audiences = (List<String>) object;
+                    // audience not contains this broker, throw exception.
+                    if (audiences.stream().noneMatch(audienceInToken -> audienceInToken.equals(audience))) {
+                        throw new AuthenticationException("Audiences in token: [" + String.join(", ", audiences)
+                                + "] not contains this broker: " + audience);
+                    }
+                } else if (object instanceof String) {
+                    if (!object.equals(audience)) {
+                        throw new AuthenticationException("Audiences in token: [" + object
+                                + "] not contains this broker: " + audience);
+                    }
+                } else {
+                    // should not reach here.
+                    throw new AuthenticationException("Audiences in token is not in expected format: " + object);
+                }
+            }
+
+            if(jwt.getExpiresAt() != null){

Review comment:
       If you use the `io.jsonwebtoken` library, you won't need to do expiration check here.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+

Review comment:
       Should we add strongly-typed fields to the ServiceConfiguration for these configuration properties? I assume that would improve the categorization, documentation etc.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";

Review comment:
       I would suggest this configuration property be named `tokenSubjectClaim` and that its default value be `sub`.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+        if(!url.getProtocol().equals("https")){
+            throw new MalformedURLException("protocol needs to be https");
+        }
+        //extracting the jwks_uri
+        JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+        this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));

Review comment:
       Consider storing the result of `authenticateToken` into a local variable before invoking `getPrincipal`.  Anticipate that other claims might be extracted from the token in future.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;

Review comment:
       These are constants, why stored as variables?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-908995352


   What about creating a separate Maven Module and use the Maven shade plugin to include all the third party deps in the .jar file for this Provider ?
   This way we are not bundling the provider out of the box but we are adding it to the build.
   Users will be able to pick the jar file and put it into "lib" in order to be able to use it
   So zero impact to existing users and especially to other users who built something like this and need other versions of the same libraries


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698949243



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       The two `auth0` libraries were likely chosen because they have support for retrieving and deserializing `JWKS` (Json Web Key Sets) objects. The `io.jsonwebtoken` libraries currently used by the `AuthenticationProviderToken` do not have this support.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }

Review comment:
       To use `token` as the auth method name is extremely helpful for integrating with Pulsar's OAuth2.0 Client Authentication plugin. However, because the `AuthenticationService` stores `AuthenticationProviders` in a map from `AuthMethodName` to `AuthenticationProvider`, that means that a broker wouldn't be able to load `AuthenticationProviderTokenJWK` and `AuthenticationProviderToken` on the same broker. This nuance would need to be clearly documented.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       This could throw a `ClassCastException` if the jwk's Public Key uses the `ECPublicKey` algorithm. This code should ideally handle that case.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());

Review comment:
       This method relies on a blocking call to the Authorization Server. I think we should improve the interface to allow this call to be asynchronous. Otherwise, we'll block a netty thread.
   
   @lhotari and I noticed this issue earlier today, and I sent an email to the pulsar dev mailing list proposing a solution: https://lists.apache.org/x/thread.html/r6c2522ca62242109758586696261cb1f4b4ce8e94ae593fda6e97b99@%3Cdev.pulsar.apache.org%3E




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r715785333



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }
+            //extracting the jwks_uri
+            JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+            this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       May I suggest that the `io.jsonwebtoken` library be used for token validation.  Pulsar already has the dependency. This will take care of applying the correct algorithm based on the key.  
   
   It is easy to use, if you implement `io.jsonwebtoken.SigningKeyResolver` as a wrapper over the `UrlJwkProvider`.  Then you can instantiate a `io.jsonwebtoken.JwtParser` that correctly validates the token.
   
   ```
   this.jwtParser = Jwts.parserBuilder()
                   .setSigningKeyResolver(signingKeyResolver)
                   .requireIssuer(issuerUrl)
                   .build();
   ...
   Jws<Claims> jwt = this.jwtParser.parseClaimsJws(tokenString);
   // perform app-specific validations, e.g audience
   ```
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r704799961



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }
+            //extracting the jwks_uri
+            JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+            this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       Does it only support this kind of algorithm? Are you considering getting the algorithm type from the alg field?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-909041052


   > What about creating a separate Maven Module and use the Maven shade plugin to include all the third party deps in the .jar file for this Provider ?
   > This way we are not bundling the provider out of the box but we are adding it to the build.
   > Users will be able to pick the jar file and put it into "lib" in order to be able to use it
   > So zero impact to existing users and especially to other users who built something like this and need other versions of the same libraries
   
   I can try this out if it's necessary. @cckellogg @michaeljmarshall I would like your feedback on this on as well 😄 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r706167276



##########
File path: conf/standalone.conf
##########
@@ -424,6 +424,9 @@ authenticationProviders=
 # Enforce authorization
 authorizationEnabled=false
 
+# issuer url that points openid-configurations from a third party provider
+issuerUrl=

Review comment:
       what about `tokenAuthenticationOIDCIssuerUrl` ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r700158546



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }

Review comment:
       Alright. I'll start documenting the setup I've used. Where should I document it?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r699864775



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }

Review comment:
       I could see a use case in the short term (the official helm chart already includes support for properly mounting super user tokens into pulsar components using the basic `AuthenticationToken` client class), but we can ignore that case, as it'd be possible to add support for mounting in a client id and client secret to enable OAuth2.0 verification from these pulsar components.
   
   Ultimately, my main point is that the nuance here should be documented to prevent users from unexpected behavior and to help them understand which clients and providers are compatible.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r716526280



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";

Review comment:
       missing clean-up again 😅 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-1058886797


   The pr had no activity for 30 days, mark with Stale label.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r718759888



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";

Review comment:
       My preference would be to not reuse the configuration parameters that are designed for a different auth provider, unless this provider is designed to emulate or subsume it.  




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r707250649



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,23 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>com.microsoft.azure</groupId>

Review comment:
       > is this still required? 
   
   nope, I used it for some old stuff. Thanks! 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r715787755



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+        if(!url.getProtocol().equals("https")){
+            throw new MalformedURLException("protocol needs to be https");
+        }
+        //extracting the jwks_uri
+        JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+        this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);
+            algorithm.verify(jwt); // if the token signature is invalid, the method will throw SignatureVerificationException
+
+            if (audienceClaim != null) {
+                Object object = jwt.getClaim(audienceClaim);
+
+                if (object == null) {
+                    throw new JwtException("Found null Audience in token, for claimed field: " + audienceClaim);
+                }
+
+                if (object instanceof List) {
+                    List<String> audiences = (List<String>) object;
+                    // audience not contains this broker, throw exception.
+                    if (audiences.stream().noneMatch(audienceInToken -> audienceInToken.equals(audience))) {
+                        throw new AuthenticationException("Audiences in token: [" + String.join(", ", audiences)
+                                + "] not contains this broker: " + audience);
+                    }
+                } else if (object instanceof String) {
+                    if (!object.equals(audience)) {
+                        throw new AuthenticationException("Audiences in token: [" + object
+                                + "] not contains this broker: " + audience);

Review comment:
       Regarding the error message, `not contains this broker`, don't assume that the audience represents the broker because that's an oauth2 thing.  It might simply represent the allowed user(s).
   
   To be honest, if you want to support audience validation in an OIDC scenario, the broker configuration should accept a list of allowed audiences.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-923860183


   @eolivelli I've started on the tests for the plugin. I'm not done yet, but I would like your feedback so I know that I'm heading in the right direction 😄 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r716485916



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;

Review comment:
       My bad. This is basically some clean-up I forgot to do 😅 . Thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r697155651



##########
File path: conf/standalone.conf
##########
@@ -462,7 +468,7 @@ tokenSecretKey=
 # The key can be specified like:
 # tokenPublicKey=data:;base64,xxxxxxxxx
 # tokenPublicKey=file:///my/public.key    ( Note: key file must be DER-encoded )
-tokenPublicKey=
+tokenPublicKey=file:///Users/dkMatHau/pulsar/oauth0.key

Review comment:
       Please revert this line

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
##########
@@ -125,15 +146,31 @@ public void initialize(ServiceConfiguration config) throws IOException, IllegalA
         this.confTokenPublicAlgSettingName = prefix + CONF_TOKEN_PUBLIC_ALG;
         this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
         this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confIsJwkEnabledSettingName = prefix + CONF_JWK_ENABLED;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
 
         // we need to fetch the algorithm before we fetch the key
         this.publicKeyAlg = getPublicKeyAlgType(config);
         this.validationKey = getValidationKey(config);
         this.roleClaim = getTokenRoleClaim(config);
         this.audienceClaim = getTokenAudienceClaim(config);
         this.audience = getTokenAudience(config);
+        this.confIsJwkEnabled = getIsJwkEnabled(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        if(this.confIsJwkEnabled){
+            try {
+                this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+            } catch (MalformedURLException e){
+                e.printStackTrace();

Review comment:
       Use Logger please




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698503213



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
##########
@@ -237,6 +280,69 @@ private static String validateToken(final String token) throws AuthenticationExc
         }
     }
 
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateTokenJwk(final String token) throws AuthenticationException {

Review comment:
       @cckellogg I added the feature as a separate plugin instead. This gets rid of most of the if else statements and the jwkEnable flag which makes it a bit cleaner 😄 

##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       > The two `auth0` libraries were likely chosen because they have support for retrieving and deserializing `JWKS` (Json Web Key Sets) objects. The `io.jsonwebtoken` libraries currently used by the `AuthenticationProviderToken` do not have this support.
   
   Exactly, io.jsonwebtoken doesn't support the JWKS feature I need 😄 

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }

Review comment:
       Not quite sure what you mean here. Why would you ever want to store  `AuthenticationProviderTokenJWK` and `AuthenticationProviderToken` on the same broker? :shipit: 
   
   I used the org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 plugin to test the implementation 😄 

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       I'll look into this. Thanks 👍  🔍 

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+
+            if(!(jwk instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }

Review comment:
       @michaeljmarshall I'm expecting the key to be an instance of RSAPublicKey so I want to throw an exception if it's not. I'm not sure if you had something else in mind 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-909926294


   I don't think I agree with making this its own module. If we did, I'd wonder why it wasn't in its own repository, or some kind of Pulsar Contrib repository. I don't have a fully formed opinion, though. I can provider more feedback tomorrow.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698953199



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }

Review comment:
       To use `token` as the auth method name is extremely helpful for integrating with Pulsar's OAuth2.0 Client Authentication plugin. However, because the `AuthenticationService` stores `AuthenticationProviders` in a map from `AuthMethodName` to `AuthenticationProvider`, that means that a broker wouldn't be able to load `AuthenticationProviderTokenJWK` and `AuthenticationProviderToken` on the same broker. This nuance would need to be clearly documented.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       This could throw a `ClassCastException` if the jwk's Public Key uses the `ECPublicKey` algorithm. This code should ideally handle that case.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());

Review comment:
       This method relies on a blocking call to the Authorization Server. I think we should improve the interface to allow this call to be asynchronous. Otherwise, we'll block a netty thread.
   
   @lhotari and I noticed this issue earlier today, and I sent an email to the pulsar dev mailing list proposing a solution: https://lists.apache.org/x/thread.html/r6c2522ca62242109758586696261cb1f4b4ce8e94ae593fda6e97b99@%3Cdev.pulsar.apache.org%3E




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r699037733



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       > The two `auth0` libraries were likely chosen because they have support for retrieving and deserializing `JWKS` (Json Web Key Sets) objects. The `io.jsonwebtoken` libraries currently used by the `AuthenticationProviderToken` do not have this support.
   
   Exactly, io.jsonwebtoken doesn't support the JWKS feature I need 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r704445420



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");

Review comment:
       Is there a prefered way to do that? I don't see the stacktrace included in other exceptions.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+

Review comment:
       Is there a prefered way to do that? I don't see the stacktrace included in other exceptions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r699106001



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+
+            if(!(jwk instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }

Review comment:
       @michaeljmarshall I'm expecting the key to be an instance of RSAPublicKey so I want to throw an exception if it's not. I'm not sure if you had something else in mind 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-916876839


   > I left some more comments.
   > 
   > Can you post a reference to Licenses for the new libraries ?
   
   https://github.com/auth0/auth0-java/blob/master/LICENSE
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r715788433



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "tokenAuthenticationOIDCIssuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        this.confTokenAuthClaimSettingName = CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = CONF_ISSUER_URL;
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+        if(!url.getProtocol().equals("https")){
+            throw new MalformedURLException("protocol needs to be https");
+        }
+        //extracting the jwks_uri
+        JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+        this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);
+            algorithm.verify(jwt); // if the token signature is invalid, the method will throw SignatureVerificationException
+
+            if (audienceClaim != null) {
+                Object object = jwt.getClaim(audienceClaim);
+
+                if (object == null) {
+                    throw new JwtException("Found null Audience in token, for claimed field: " + audienceClaim);
+                }
+
+                if (object instanceof List) {
+                    List<String> audiences = (List<String>) object;
+                    // audience not contains this broker, throw exception.
+                    if (audiences.stream().noneMatch(audienceInToken -> audienceInToken.equals(audience))) {
+                        throw new AuthenticationException("Audiences in token: [" + String.join(", ", audiences)
+                                + "] not contains this broker: " + audience);
+                    }
+                } else if (object instanceof String) {
+                    if (!object.equals(audience)) {
+                        throw new AuthenticationException("Audiences in token: [" + object
+                                + "] not contains this broker: " + audience);
+                    }
+                } else {
+                    // should not reach here.
+                    throw new AuthenticationException("Audiences in token is not in expected format: " + object);
+                }
+            }
+
+            if(jwt.getExpiresAt() != null){

Review comment:
       If you use the `io.jsonwebtoken` library, you won't need to do expiration check because it will do it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r706152570



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       Maybe. I need to check if com.oauth0 supports the functionality needed for AuthenticationProviderToken  




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r704293295



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+

Review comment:
       Is there a prefered way to do that? I don't see the stacktrace included in other exceptions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-908995352


   What about creating a separate Maven Module and use the Maven shade plugin to include all the third party deps in the .jar file for this Provider ?
   This way we are not bundling the provider out of the box but we are adding it to the build.
   Users will be able to pick the jar file and put it into "lib" in order to be able to use it
   So zero impact to existing users and especially to other users who built something like this and need other versions of the same libraries


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698505866



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       is it really necessary to add all of these third party deps ?
   I wonder if it won't be better to create a separate module and let users drop the jar in pulsar/lib
   
   if we want to go this way we have to verify all the licenses/NOTICE files and list the artifacts in https://github.com/apache/pulsar/tree/master/distribution/server/src/assemble and probably in https://github.com/apache/pulsar/tree/master/distribution/server/licenses 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r715734178



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }

Review comment:
       This check may complicate dev/test scenarios, seems superfluous to me.  




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-915262890


   > Implement configurable caching of public keys (https://openid.net/specs/openid-connect-core-1_0.html#RotateEncKeys)
   
   @michaeljmarshall is there a prefered way of doing this? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r698229513



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
##########
@@ -237,6 +280,69 @@ private static String validateToken(final String token) throws AuthenticationExc
         }
     }
 
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateTokenJwk(final String token) throws AuthenticationException {

Review comment:
       Sounds like a good idea 👍 
   
   I'll see what I can do 🔍 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r704956370



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }

Review comment:
       That check makes sense to me.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r697194480



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
##########
@@ -125,15 +146,31 @@ public void initialize(ServiceConfiguration config) throws IOException, IllegalA
         this.confTokenPublicAlgSettingName = prefix + CONF_TOKEN_PUBLIC_ALG;
         this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
         this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confIsJwkEnabledSettingName = prefix + CONF_JWK_ENABLED;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
 
         // we need to fetch the algorithm before we fetch the key
         this.publicKeyAlg = getPublicKeyAlgType(config);
         this.validationKey = getValidationKey(config);
         this.roleClaim = getTokenRoleClaim(config);
         this.audienceClaim = getTokenAudienceClaim(config);
         this.audience = getTokenAudience(config);
+        this.confIsJwkEnabled = getIsJwkEnabled(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        if(this.confIsJwkEnabled){
+            try {
+                this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+            } catch (MalformedURLException e){
+                e.printStackTrace();

Review comment:
       I just removed e.printStackTrace() since I can see that you never really use LOG.error. The user will be informed of the malformed url from the exception 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r704956143



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }

Review comment:
       Per [RFC-8414](https://datatracker.ietf.org/doc/html/rfc8414#section-2), the `jwks_uri` must use the "https" scheme. If we want to make this requirement optional, perhaps we can make the plugin require "https" by default and then allow for a configuration that can override the requirement.
   
   >    OPTIONAL.  URL of the authorization server's JWK Set [JWK]
         document.  The referenced document contains the signing key(s) the
         client uses to validate signatures from the authorization server.
         This URL MUST use the "https" scheme.  The JWK Set MAY also
         contain the server's encryption key or keys, which are used by
         clients to encrypt requests to the server.  When both signing and
         encryption keys are made available, a "use" (public key use)
         parameter value is REQUIRED for all keys in the referenced JWK Set
         to indicate each key's intended usage.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-916894885


   I believe that we are on our way. 
   Thank you for your patience, we are giving feedback in multiple steps
   Good job


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r699864775



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }

Review comment:
       I could see a use case in the short term (the official helm chart already includes support for properly mounting super user tokens into pulsar components using the basic `AuthenticationToken` client class), but we can ignore that case, as it'd be possible to add support for mounting in a client id and client secret to enable OAuth2.0 verification from these pulsar components.
   
   Ultimately, my main point is that the nuance here should be documented to prevent users from unexpected behavior and to help them understand which clients and providers are compatible.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r706168048



##########
File path: pulsar-broker-common/pom.xml
##########
@@ -65,6 +65,25 @@
       <artifactId>jjwt-impl</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MathiasHaudgaard commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
MathiasHaudgaard commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r699106001



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenJWK.java
##########
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.*;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenJWK implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_JWK_URL = "jwkUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String jwkUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_JWK_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.jwkUrl = getJwkUrl(config);
+
+        try {
+            this.provider = new UrlJwkProvider(new URL(this.jwkUrl));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+
+            if(!(jwk instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }

Review comment:
       @michaeljmarshall I'm expecting the key to be an instance of RSAPublicKey so I want to throw an exception if it's not. I'm not sure if you had something else in mind 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] cckellogg commented on a change in pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
cckellogg commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r697897377



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
##########
@@ -237,6 +280,69 @@ private static String validateToken(final String token) throws AuthenticationExc
         }
     }
 
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateTokenJwk(final String token) throws AuthenticationException {

Review comment:
       Instead of adding this code to the AuthenticationProvider could we refactor some of this code out into separate classes so there is not a lot of if else checks (i think that makes the code more difficult to follow test and maintain) within the code?
   
   Maybe there can be a TokenDecoder interface and TokenInfo or Token class. Then there can be multiple implementations of the TokenDecoder and the code can be simplified and not care how it works underneath. The TokenInfo or Token class can add more attributes over time but at this point I think we only care about the principal and expiration.
   
   Thoughts?
   
   
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on pull request #11794: jwks feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#issuecomment-909926294


   I don't think I agree with making this its own module. If we did, I'd wonder why it wasn't in its own repository, or some kind of Pulsar Contrib repository. I don't have a fully formed opinion, though. I can provider more feedback tomorrow.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] EronWright commented on a change in pull request #11794: OIDC feature for oAuth flow

Posted by GitBox <gi...@apache.org>.
EronWright commented on a change in pull request #11794:
URL: https://github.com/apache/pulsar/pull/11794#discussion_r715785333



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenOIDC.java
##########
@@ -0,0 +1,372 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.SocketAddress;
+import java.net.URL;
+
+import java.nio.charset.Charset;
+import java.security.interfaces.RSAPublicKey;
+import java.util.Date;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+
+import com.auth0.jwk.Jwk;
+import com.auth0.jwk.JwkException;
+import com.auth0.jwk.JwkProvider;
+import com.auth0.jwk.UrlJwkProvider;
+import com.auth0.jwt.JWT;
+import com.auth0.jwt.algorithms.Algorithm;
+import com.auth0.jwt.exceptions.SignatureVerificationException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics;
+import org.apache.pulsar.common.api.AuthData;
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.JwtException;
+import com.auth0.jwt.interfaces.DecodedJWT;
+
+public class AuthenticationProviderTokenOIDC implements AuthenticationProvider {
+
+    static final String HTTP_HEADER_NAME = "Authorization";
+    static final String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
+
+    // When symmetric key is configured
+    static final String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // The token's claim that corresponds to the "role" string
+    static final String CONF_TOKEN_AUTH_CLAIM = "tokenAuthClaim";
+
+
+    static final String CONF_ISSUER_URL = "issuerUrl";
+
+    // When using public key's, the algorithm of the key
+    static final String CONF_TOKEN_PUBLIC_ALG = "tokenPublicKeytokenPublicKey";
+
+    // The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
+    static final String CONF_TOKEN_AUDIENCE_CLAIM = "tokenAudienceClaim";
+
+    // The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
+    static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
+
+    static final String TOKEN = "token";
+
+    private static final Counter expiredTokenMetrics = Counter.build()
+            .name("pulsar_expired_token_count")
+            .help("Pulsar expired token")
+            .register();
+
+    private static final Histogram expiringTokenMinutesMetrics = Histogram.build()
+            .name("pulsar_expiring_token_minutes")
+            .help("The remaining time of expiring token in minutes")
+            .buckets(5, 10, 60, 240)
+            .register();
+
+    private String roleClaim;
+    private String audienceClaim;
+    private String audience;
+    private JwkProvider provider;
+    private String issuerUrl;
+
+    // config keys
+
+    private String confTokenAuthClaimSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+    private String confJWkUrlSettingName;
+
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @VisibleForTesting
+    public static void resetMetrics() {
+        expiredTokenMetrics.clear();
+        expiringTokenMinutesMetrics.clear();
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+        this.confJWkUrlSettingName = prefix + CONF_ISSUER_URL;
+
+
+        // we need to fetch the algorithm before we fetch the key
+        this.roleClaim = getTokenRoleClaim(config);
+        this.audienceClaim = getTokenAudienceClaim(config);
+        this.audience = getTokenAudience(config);
+        this.issuerUrl = getIssuerUrl(config);
+
+        try {
+            URL url = new URL(this.issuerUrl+"/.well-known/openid-configuration");
+            if(!url.getProtocol().equals("https")){
+                throw new MalformedURLException("protocol needs to be https");
+            }
+            //extracting the jwks_uri
+            JsonObject json = new Gson().fromJson(IOUtils.toString(url, UTF_8), JsonObject.class);
+            this.provider = new UrlJwkProvider(new URL(json.get("jwks_uri").getAsString()));
+        } catch (MalformedURLException e){
+            throw new MalformedURLException("Url is malformed");
+        }
+
+
+
+
+        if (audienceClaim != null && audience == null ) {
+            throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
+                    + "] configured, but Audience stands for this broker not.");
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return TOKEN;
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        try {
+            // Get Token
+            String token;
+            String role;
+            token = getToken(authData);
+
+            // Parse Token by validating
+
+
+            role = getPrincipal(authenticateToken(token));
+
+
+            AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName());
+            return role;
+        } catch (AuthenticationException exception) {
+            AuthenticationMetrics.authenticateFailure(getClass().getSimpleName(), getAuthMethodName(), exception.getMessage());
+            throw exception;
+        }
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+            throws AuthenticationException {
+        return new TokenAuthenticationState(this, authData, remoteAddress, sslSession);
+    }
+
+    public static String getToken(AuthenticationDataSource authData) throws AuthenticationException {
+        if (authData.hasDataFromCommand()) {
+            // Authenticate Pulsar binary connection
+            return validateToken(authData.getCommandData());
+        } else if (authData.hasDataFromHttp()) {
+            // Authentication HTTP request. The format here should be compliant to RFC-6750
+            // (https://tools.ietf.org/html/rfc6750#section-2.1). Eg: Authorization: Bearer xxxxxxxxxxxxx
+            String httpHeaderValue = authData.getHttpHeader(HTTP_HEADER_NAME);
+            if (httpHeaderValue == null || !httpHeaderValue.startsWith(HTTP_HEADER_VALUE_PREFIX)) {
+                throw new AuthenticationException("Invalid HTTP Authorization header");
+            }
+
+            // Remove prefix
+            String token = httpHeaderValue.substring(HTTP_HEADER_VALUE_PREFIX.length());
+            return validateToken(token);
+        } else {
+            throw new AuthenticationException("No token credentials passed");
+        }
+    }
+
+    private static String validateToken(final String token) throws AuthenticationException {
+        if (StringUtils.isNotBlank(token)) {
+            return token;
+        } else {
+            throw new AuthenticationException("Blank token found");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private DecodedJWT authenticateToken(final String token) throws AuthenticationException {
+        DecodedJWT jwt = JWT.decode(token);
+
+        Jwk jwk = null;
+        Algorithm algorithm = null;
+        try {
+            jwk = provider.get(jwt.getKeyId());
+            if(!(jwk.getPublicKey() instanceof RSAPublicKey)){
+                throw new JwtException("key needs to be a RSA Publickey");
+            }
+
+            algorithm = Algorithm.RSA256((RSAPublicKey) jwk.getPublicKey(), null);

Review comment:
       May I suggest that the `io.jsonwebtoken` library be used for token validation.  Pulsar already has the dependency. This will take care of applying the correct algorithm based on the key.  
   
   It is easy to use, if you implement `io.jsonwebtoken.SigningKeyResolver` as a wrapper over the `UrlJwkProvider`.  Then you can instantiate a `io.jsonwebtoken.JwtParser` that correctly validates the token.
   
   ```
   this.jwtParser = Jwts.parserBuilder()
                   .setSigningKeyResolver(signingKeyResolver)
                   .requireIssuer(issuerUrl)
                   .build();
   ...
   // parse and validate the token
   Jws<Claims> jwt = this.jwtParser.parseClaimsJws(tokenString);
   // perform app-specific validations, e.g audience
   ```
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org