Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.time.Clock;
import java.time.Duration;
import org.apache.pulsar.client.api.Authentication;
import org.apache.pulsar.client.impl.auth.oauth2.protocol.DefaultMetadataResolver;

/**
* Factory class that allows to create {@link Authentication} instances
Expand Down Expand Up @@ -69,6 +70,17 @@ public static ClientCredentialsBuilder clientCredentialsBuilder() {
return new ClientCredentialsBuilder();
}

/**
* A builder to create an authentication with client credentials using standard OAuth 2.0 metadata path
* as defined in RFC 8414 ("/.well-known/oauth-authorization-server").
*
* @return the builder pre-configured to use standard OAuth 2.0 metadata path
*/
public static ClientCredentialsBuilder clientCredentialsWithStandardAuthzServerBuilder() {
return new ClientCredentialsBuilder()
.wellKnownMetadataPath(DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH);
}

public static class ClientCredentialsBuilder {

private URL issuerUrl;
Expand All @@ -78,6 +90,7 @@ public static class ClientCredentialsBuilder {
private Duration connectTimeout;
private Duration readTimeout;
private String trustCertsFilePath;
private String wellKnownMetadataPath;

private ClientCredentialsBuilder() {
}
Expand Down Expand Up @@ -163,6 +176,17 @@ public ClientCredentialsBuilder trustCertsFilePath(String trustCertsFilePath) {
return this;
}

/**
* Optional well-known metadata path.
*
* @param wellKnownMetadataPath the well-known metadata path (must start with "/.well-known/")
* @return the builder
*/
public ClientCredentialsBuilder wellKnownMetadataPath(String wellKnownMetadataPath) {
this.wellKnownMetadataPath = wellKnownMetadataPath;
return this;
}

/**
* Authenticate with client credentials.
*
Expand All @@ -177,6 +201,7 @@ public Authentication build() {
.connectTimeout(connectTimeout)
.readTimeout(readTimeout)
.trustCertsFilePath(trustCertsFilePath)
.wellKnownMetadataPath(wellKnownMetadataPath)
.build();
return new AuthenticationOAuth2(flow, Clock.systemDefaultZone());
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.client.impl.auth.oauth2;

import java.io.IOException;
import java.time.Clock;
import java.util.Map;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.client.impl.AuthenticationUtil;
import org.apache.pulsar.client.impl.auth.oauth2.protocol.DefaultMetadataResolver;

/**
* Pulsar client authentication provider based on OAuth 2.0 using RFC 8414 standard metadata path.
* This class is identical to {@link AuthenticationOAuth2} but it always uses the standard
* "/.well-known/oauth-authorization-server" metadata path as defined in RFC 8414.
*/
public class AuthenticationOAuth2StandardAuthzServer extends AuthenticationOAuth2 {

private static final long serialVersionUID = 1L;

public AuthenticationOAuth2StandardAuthzServer() {
super();
}

AuthenticationOAuth2StandardAuthzServer(Flow flow, Clock clock) {
super(flow, clock);
}

@Override
public void configure(String encodedAuthParamString) {
if (StringUtils.isBlank(encodedAuthParamString)) {
throw new IllegalArgumentException("No authentication parameters were provided");
}
Map<String, String> params;
try {
params = AuthenticationUtil.configureFromJsonString(encodedAuthParamString);
} catch (IOException e) {
throw new IllegalArgumentException("Malformed authentication parameters", e);
}

// Always set the OAuth 2.0 standard metadata path
params.put(FlowBase.CONFIG_PARAM_WELL_KNOWN_METADATA_PATH,
DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH);

String type = params.getOrDefault(CONFIG_PARAM_TYPE, TYPE_CLIENT_CREDENTIALS);
switch(type) {
case TYPE_CLIENT_CREDENTIALS:
this.flow = ClientCredentialsFlow.fromParameters(params);
break;
default:
throw new IllegalArgumentException("Unsupported authentication type: " + type);
}
}
}
Comment on lines +33 to +70
Copy link
Member

Choose a reason for hiding this comment

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

What's the purpose of this class? Could we eliminate this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This class allows users to switch authentication plugins and use the standard metadata path defined in RFC 8414 without additional configuration.
It’s especially useful for CLI tool, where ClientCredentialsBuilder cannot be used.

Original file line number Diff line number Diff line change
Expand Up @@ -62,8 +62,9 @@ class ClientCredentialsFlow extends FlowBase {

@Builder
public ClientCredentialsFlow(URL issuerUrl, String audience, String privateKey, String scope,
Duration connectTimeout, Duration readTimeout, String trustCertsFilePath) {
super(issuerUrl, connectTimeout, readTimeout, trustCertsFilePath);
Duration connectTimeout, Duration readTimeout, String trustCertsFilePath,
String wellKnownMetadataPath) {
super(issuerUrl, connectTimeout, readTimeout, trustCertsFilePath, wellKnownMetadataPath);
this.audience = audience;
this.privateKey = privateKey;
this.scope = scope;
Expand All @@ -84,6 +85,7 @@ public static ClientCredentialsFlow fromParameters(Map<String, String> params) {
Duration connectTimeout = parseParameterDuration(params, CONFIG_PARAM_CONNECT_TIMEOUT);
Duration readTimeout = parseParameterDuration(params, CONFIG_PARAM_READ_TIMEOUT);
String trustCertsFilePath = params.get(CONFIG_PARAM_TRUST_CERTS_FILE_PATH);
String wellKnownMetadataPath = params.get(CONFIG_PARAM_WELL_KNOWN_METADATA_PATH);

return ClientCredentialsFlow.builder()
.issuerUrl(issuerUrl)
Expand All @@ -93,6 +95,7 @@ public static ClientCredentialsFlow fromParameters(Map<String, String> params) {
.connectTimeout(connectTimeout)
.readTimeout(readTimeout)
.trustCertsFilePath(trustCertsFilePath)
.wellKnownMetadataPath(wellKnownMetadataPath)
.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ abstract class FlowBase implements Flow {
public static final String CONFIG_PARAM_CONNECT_TIMEOUT = "connectTimeout";
public static final String CONFIG_PARAM_READ_TIMEOUT = "readTimeout";
public static final String CONFIG_PARAM_TRUST_CERTS_FILE_PATH = "trustCertsFilePath";
public static final String CONFIG_PARAM_WELL_KNOWN_METADATA_PATH = "wellKnownMetadataPath";

protected static final Duration DEFAULT_CONNECT_TIMEOUT = Duration.ofSeconds(10);
protected static final Duration DEFAULT_READ_TIMEOUT = Duration.ofSeconds(30);
Expand All @@ -55,12 +56,15 @@ abstract class FlowBase implements Flow {

protected final URL issuerUrl;
protected final AsyncHttpClient httpClient;
protected final String wellKnownMetadataPath;

protected transient Metadata metadata;

protected FlowBase(URL issuerUrl, Duration connectTimeout, Duration readTimeout, String trustCertsFilePath) {
protected FlowBase(URL issuerUrl, Duration connectTimeout, Duration readTimeout, String trustCertsFilePath,
String wellKnownMetadataPath) {
this.issuerUrl = issuerUrl;
this.httpClient = defaultHttpClient(readTimeout, connectTimeout, trustCertsFilePath);
this.wellKnownMetadataPath = wellKnownMetadataPath;
}

private AsyncHttpClient defaultHttpClient(Duration readTimeout, Duration connectTimeout,
Expand Down Expand Up @@ -110,7 +114,7 @@ public void initialize() throws PulsarClientException {
}

protected MetadataResolver createMetadataResolver() {
return DefaultMetadataResolver.fromIssuerUrl(issuerUrl, httpClient);
return DefaultMetadataResolver.fromIssuerUrl(issuerUrl, httpClient, wellKnownMetadataPath);
}

static String parseParameterString(Map<String, String> params, String name) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,13 @@
*/
public class DefaultMetadataResolver implements MetadataResolver {

private static final String WELL_KNOWN_PREFIX = "/.well-known/";
private static final String DEFAULT_WELL_KNOWN_METADATA_PATH = WELL_KNOWN_PREFIX + "openid-configuration";
/**
* The OAuth 2.0 Authorization Server Metadata path as defined in RFC 8414.
*/
public static final String OAUTH_WELL_KNOWN_METADATA_PATH = WELL_KNOWN_PREFIX + "oauth-authorization-server";

private final URL metadataUrl;
private final ObjectReader objectReader;
private final AsyncHttpClient httpClient;
Expand All @@ -50,23 +57,41 @@ public DefaultMetadataResolver(URL metadataUrl, AsyncHttpClient httpClient) {
* Gets a well-known metadata URL for the given OAuth issuer URL.
*
* @param issuerUrl The authorization server's issuer identifier
* @param httpClient The HTTP client
* @param wellKnownMetadataPath The well-known metadata path (must start with "/.well-known/")
* @return a resolver
*/
public static DefaultMetadataResolver fromIssuerUrl(URL issuerUrl, AsyncHttpClient httpClient) {
return new DefaultMetadataResolver(getWellKnownMetadataUrl(issuerUrl), httpClient);
public static DefaultMetadataResolver fromIssuerUrl(URL issuerUrl, AsyncHttpClient httpClient,
String wellKnownMetadataPath) {
return new DefaultMetadataResolver(getWellKnownMetadataUrl(issuerUrl, wellKnownMetadataPath), httpClient);
}

/**
* Gets a well-known metadata URL for the given OAuth issuer URL.
*
* @param issuerUrl The authorization server's issuer identifier
* @param wellKnownMetadataPath The well-known metadata path (must start with "/.well-known/")
* @return a URL
* @see <a href="https://tools.ietf.org/id/draft-ietf-oauth-discovery-08.html#ASConfig">
* OAuth Discovery: Obtaining Authorization Server Metadata</a>
*/
public static URL getWellKnownMetadataUrl(URL issuerUrl) {
public static URL getWellKnownMetadataUrl(URL issuerUrl, String wellKnownMetadataPath) {
try {
return URI.create(issuerUrl.toExternalForm() + "/.well-known/openid-configuration").normalize().toURL();
if (wellKnownMetadataPath == null || wellKnownMetadataPath.isEmpty()) {
return URI.create(issuerUrl.toExternalForm() + DEFAULT_WELL_KNOWN_METADATA_PATH).normalize().toURL();
}
if (wellKnownMetadataPath.startsWith(WELL_KNOWN_PREFIX)) {
String issuerUrlString = issuerUrl.toExternalForm();
// For OAuth2, insert well-known path before the issuer URL path
URL url = new URL(issuerUrlString);
String path = url.getPath();
String basePath = issuerUrlString.substring(0,
issuerUrlString.length() - (path.isEmpty() ? 0 : path.length()));
return URI.create(basePath + wellKnownMetadataPath + path).normalize().toURL();
} else {
throw new IllegalArgumentException("Metadata path must start with '" + WELL_KNOWN_PREFIX
+ "', but was: " + wellKnownMetadataPath);
}
} catch (MalformedURLException e) {
throw new IllegalArgumentException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,11 +36,26 @@ public void testBuilder() throws IOException {
Duration connectTimeout = Duration.parse("PT11S");
Duration readTimeout = Duration.ofSeconds(31);
String trustCertsFilePath = null;
String wellKnownMetadataPath = "/.well-known/custom-path";
try (Authentication authentication =
AuthenticationFactoryOAuth2.clientCredentialsBuilder().issuerUrl(issuerUrl)
.credentialsUrl(credentialsUrl).audience(audience).scope(scope)
.connectTimeout(connectTimeout).readTimeout(readTimeout)
.trustCertsFilePath(trustCertsFilePath).build()) {
.trustCertsFilePath(trustCertsFilePath)
.wellKnownMetadataPath(wellKnownMetadataPath).build()) {
assertTrue(authentication instanceof AuthenticationOAuth2);
}
}

@Test
public void testStandardAuthzServerBuilder() throws IOException {
URL issuerUrl = new URL("http://localhost");
URL credentialsUrl = new URL("http://localhost");
String audience = "audience";
String scope = "scope";
try (Authentication authentication =
AuthenticationFactoryOAuth2.clientCredentialsWithStandardAuthzServerBuilder().issuerUrl(issuerUrl)
.credentialsUrl(credentialsUrl).audience(audience).scope(scope).build()) {
assertTrue(authentication instanceof AuthenticationOAuth2);
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.client.impl.auth.oauth2;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.lang.reflect.Field;
import java.util.HashMap;
import java.util.Map;
import org.apache.pulsar.client.impl.auth.oauth2.protocol.DefaultMetadataResolver;
import org.testng.annotations.Test;

public class AuthenticationOAuth2StandardAuthzServerTest {

@Test
public void testConfigureWithOAuth2MetadataPath() throws Exception {
Map<String, String> params = new HashMap<>();
params.put("type", "client_credentials");
params.put("privateKey", "data:base64,e30=");
params.put("issuerUrl", "http://localhost");
params.put("audience", "test-audience");
ObjectMapper mapper = new ObjectMapper();
String authParams = mapper.writeValueAsString(params);
AuthenticationOAuth2StandardAuthzServer auth = new AuthenticationOAuth2StandardAuthzServer();
auth.configure(authParams);
assertTrue(auth.flow instanceof ClientCredentialsFlow);
ClientCredentialsFlow flow = (ClientCredentialsFlow) auth.flow;
Field wellKnownMetadataPathField = FlowBase.class.getDeclaredField("wellKnownMetadataPath");
wellKnownMetadataPathField.setAccessible(true);
String wellKnownMetadataPath = (String) wellKnownMetadataPathField.get(flow);
assertEquals(wellKnownMetadataPath, DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@ public void testConfigure() throws Exception {
params.put("issuerUrl", "http://localhost");
params.put("audience", "http://localhost");
params.put("scope", "http://localhost");
params.put("wellKnownMetadataPath", "/.well-known/custom-path");
ObjectMapper mapper = new ObjectMapper();
String authParams = mapper.writeValueAsString(params);
this.auth.configure(authParams);
Expand Down Expand Up @@ -132,8 +133,48 @@ public void testGetAuthData() throws Exception {

@Test
public void testMetadataResolver() throws MalformedURLException {
URL url = DefaultMetadataResolver.getWellKnownMetadataUrl(URI.create("http://localhost/path/oauth").toURL());
URL url = DefaultMetadataResolver.getWellKnownMetadataUrl(
URI.create("http://localhost/path/oauth").toURL(),
null);
assertEquals("http://localhost/path/oauth/.well-known/openid-configuration", url.toString());

// custom wellKnownMetadataPath with full well-known prefix
URL customUrl = DefaultMetadataResolver.getWellKnownMetadataUrl(
URI.create("http://localhost/path/oauth").toURL(),
"/.well-known/custom-path");
assertEquals("http://localhost/.well-known/custom-path/path/oauth", customUrl.toString());

// null wellKnownMetadataPath (should use default)
URL customUrl2 = DefaultMetadataResolver.getWellKnownMetadataUrl(
URI.create("http://localhost/path/oauth").toURL(),
null);
assertEquals("http://localhost/path/oauth/.well-known/openid-configuration", customUrl2.toString());

// empty wellKnownMetadataPath (should use default)
URL customUrl3 = DefaultMetadataResolver.getWellKnownMetadataUrl(
URI.create("http://localhost/path/oauth").toURL(),
"");
assertEquals("http://localhost/path/oauth/.well-known/openid-configuration", customUrl3.toString());

// using RFC8414 OAuth2 metadata path
URL oauthUrl = DefaultMetadataResolver.getWellKnownMetadataUrl(
URI.create("http://localhost/path/oauth").toURL(),
DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH);
assertEquals("http://localhost/.well-known/oauth-authorization-server/path/oauth", oauthUrl.toString());

// test with issuer URL without path
URL oauthUrlNoPath = DefaultMetadataResolver.getWellKnownMetadataUrl(
URI.create("http://localhost").toURL(),
DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH);
assertEquals("http://localhost/.well-known/oauth-authorization-server", oauthUrlNoPath.toString());
}

@Test(expectedExceptions = IllegalArgumentException.class,
expectedExceptionsMessageRegExp = ".*Metadata path must start with.*")
public void testMetadataResolverWithInvalidPath() throws MalformedURLException {
DefaultMetadataResolver.getWellKnownMetadataUrl(
URI.create("http://localhost/path/oauth").toURL(),
"/custom-path");
}

@Test
Expand Down