Skip to content

Commit 139209a

Browse files
[fix][proxy] Refresh auth data if ProxyLookupRequests (apache#20067)
Fixes: apache#10816 PIP: apache#19771 Supersedes: apache#19026 Depends on: apache#20062 The Pulsar Proxy does not properly handle authentication data refresh when in state `ProxyLookupRequests`. The consequence is described in apache#10816. Essentially, the problem is that the proxy caches stale authentication data and sends it to the broker leading to connection failures. apache#17831 attempted to fix the underlying problem, but it missed an important edge cases. Specifically, it missed the case that the `ConnectionPool` will have multiple connections when a lookup gets redirected. As such, the following problem exists (and is fixed by this PR): 1. Client opens connection to perform lookups. 2. Proxy connects to broker 1 to get the topic ownership info. 3. Time passes. 4. Client does an additional lookup, and this topic is on a newly created broker 2. In this case, the proxy opens a new connection with the stale client auth data. 5. Broker 2 rejects the connection because it fails with expired authentication. * Remove some of the implementation from apache#17831. This new implementation still allows a broker to challenge the client through the proxy, but notably, it limits the number of challenges sent to the client. Further, the proxy does not challenge the client when the auth data is not expired. * Introduce authentication refresh in the proxy so that the proxy challenges the client any time the auth data is expired. * Update the `ProxyClientCnx` to get the `clientAuthData` from the `ProxyConnection` to ensure that it gets new authentication data. * Add clock skew to the `AuthenticationProviderToken`. This is necessary to make some of the testing not flaky and it will also be necessary for users to configure in their clusters. The `ProxyRefreshAuthTest` covers the existing behavior and I expanded it to cover the edge case described above. Additionally, testing this part of the code will be much easier to test once we implement apache#19624. - [x] `doc-not-needed` PR in forked repository: the relevant tests pass locally, so I am going to skip the forked tests. (cherry picked from commit 075b625)
1 parent 6332aa4 commit 139209a

File tree

5 files changed

+220
-95
lines changed

5 files changed

+220
-95
lines changed

pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java

+21-1
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,9 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
7373

7474
// The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
7575
static final String CONF_TOKEN_AUDIENCE = "tokenAudience";
76+
// The amount of time in seconds that a token is allowed to be out of sync with the server's time when performing
77+
// token validation.
78+
static final String CONF_TOKEN_ALLOWED_CLOCK_SKEW_SECONDS = "tokenAllowedClockSkewSeconds";
7679

7780
static final String TOKEN = "token";
7881

@@ -101,6 +104,7 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
101104
private String confTokenPublicAlgSettingName;
102105
private String confTokenAudienceClaimSettingName;
103106
private String confTokenAudienceSettingName;
107+
private String confTokenAllowedClockSkewSecondsSettingName;
104108

105109
@Override
106110
public void close() throws IOException {
@@ -125,6 +129,7 @@ public void initialize(ServiceConfiguration config) throws IOException, IllegalA
125129
this.confTokenPublicAlgSettingName = prefix + CONF_TOKEN_PUBLIC_ALG;
126130
this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
127131
this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
132+
this.confTokenAllowedClockSkewSecondsSettingName = prefix + CONF_TOKEN_ALLOWED_CLOCK_SKEW_SECONDS;
128133

129134
// we need to fetch the algorithm before we fetch the key
130135
this.publicKeyAlg = getPublicKeyAlgType(config);
@@ -133,7 +138,12 @@ public void initialize(ServiceConfiguration config) throws IOException, IllegalA
133138
this.audienceClaim = getTokenAudienceClaim(config);
134139
this.audience = getTokenAudience(config);
135140

136-
this.parser = Jwts.parserBuilder().setSigningKey(this.validationKey).build();
141+
long allowedSkew = getConfTokenAllowedClockSkewSeconds(config);
142+
143+
this.parser = Jwts.parserBuilder()
144+
.setAllowedClockSkewSeconds(allowedSkew)
145+
.setSigningKey(this.validationKey)
146+
.build();
137147

138148
if (audienceClaim != null && audience == null) {
139149
throw new IllegalArgumentException("Token Audience Claim [" + audienceClaim
@@ -329,6 +339,16 @@ private String getTokenAudience(ServiceConfiguration conf) throws IllegalArgumen
329339
}
330340
}
331341

342+
// get Token's allowed clock skew in seconds. If not configured, defaults to 0.
343+
private long getConfTokenAllowedClockSkewSeconds(ServiceConfiguration conf) throws IllegalArgumentException {
344+
String allowedSkewStr = (String) conf.getProperty(confTokenAllowedClockSkewSecondsSettingName);
345+
if (StringUtils.isNotBlank(allowedSkewStr)) {
346+
return Long.parseLong(allowedSkewStr);
347+
} else {
348+
return 0;
349+
}
350+
}
351+
332352
private static final class TokenAuthenticationState implements AuthenticationState {
333353
private final AuthenticationProviderToken provider;
334354
private final SocketAddress remoteAddress;

pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java

+29-41
Original file line numberDiff line numberDiff line change
@@ -23,26 +23,31 @@
2323
import io.netty.channel.EventLoopGroup;
2424
import java.util.Arrays;
2525
import lombok.extern.slf4j.Slf4j;
26+
import org.apache.pulsar.PulsarVersion;
2627
import org.apache.pulsar.client.impl.ClientCnx;
2728
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
2829
import org.apache.pulsar.common.api.AuthData;
2930
import org.apache.pulsar.common.api.proto.CommandAuthChallenge;
3031
import org.apache.pulsar.common.protocol.Commands;
3132

33+
/**
34+
* Channel handler for Pulsar proxy's Pulsar broker client connections for lookup requests.
35+
* <p>
36+
* Please see {@link org.apache.pulsar.common.protocol.PulsarDecoder} javadoc for important details about handle*
37+
* method parameter instance lifecycle.
38+
*/
3239
@Slf4j
3340
public class ProxyClientCnx extends ClientCnx {
3441
private final boolean forwardClientAuthData;
3542
private final String clientAuthMethod;
3643
private final String clientAuthRole;
37-
private final AuthData clientAuthData;
3844
private final ProxyConnection proxyConnection;
3945

4046
public ProxyClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, String clientAuthRole,
41-
AuthData clientAuthData, String clientAuthMethod, int protocolVersion,
47+
String clientAuthMethod, int protocolVersion,
4248
boolean forwardClientAuthData, ProxyConnection proxyConnection) {
4349
super(conf, eventLoopGroup, protocolVersion);
4450
this.clientAuthRole = clientAuthRole;
45-
this.clientAuthData = clientAuthData;
4651
this.clientAuthMethod = clientAuthMethod;
4752
this.forwardClientAuthData = forwardClientAuthData;
4853
this.proxyConnection = proxyConnection;
@@ -53,9 +58,15 @@ protected ByteBuf newConnectCommand() throws Exception {
5358
if (log.isDebugEnabled()) {
5459
log.debug("New Connection opened via ProxyClientCnx with params clientAuthRole = {},"
5560
+ " clientAuthData = {}, clientAuthMethod = {}",
56-
clientAuthRole, clientAuthData, clientAuthMethod);
61+
clientAuthRole, proxyConnection.getClientAuthData(), clientAuthMethod);
62+
}
63+
AuthData clientAuthData = null;
64+
if (forwardClientAuthData) {
65+
// There is a chance this auth data is expired because the ProxyConnection does not do early token refresh.
66+
// Based on the current design, the best option is to configure the broker to accept slightly stale
67+
// authentication data.
68+
clientAuthData = proxyConnection.getClientAuthData();
5769
}
58-
5970
authenticationDataProvider = authentication.getAuthData(remoteHostName);
6071
AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
6172
return Commands.newConnect(authentication.getAuthMethodName(), authData, protocolVersion,
@@ -69,43 +80,20 @@ protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
6980
checkArgument(authChallenge.getChallenge().hasAuthData());
7081

7182
boolean isRefresh = Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData());
72-
if (!forwardClientAuthData || !isRefresh) {
73-
super.handleAuthChallenge(authChallenge);
74-
return;
75-
}
76-
77-
try {
78-
if (log.isDebugEnabled()) {
79-
log.debug("Proxy {} request to refresh the original client authentication data for "
80-
+ "the proxy client {}", proxyConnection.ctx().channel(), ctx.channel());
81-
}
82-
83-
proxyConnection.ctx().writeAndFlush(Commands.newAuthChallenge(clientAuthMethod, AuthData.REFRESH_AUTH_DATA,
84-
protocolVersion))
85-
.addListener(writeFuture -> {
86-
if (writeFuture.isSuccess()) {
87-
if (log.isDebugEnabled()) {
88-
log.debug("Proxy {} sent the auth challenge to original client to refresh credentials "
89-
+ "with method {} for the proxy client {}",
90-
proxyConnection.ctx().channel(), clientAuthMethod, ctx.channel());
91-
}
92-
} else {
93-
log.error("Failed to send the auth challenge to original client by the proxy {} "
94-
+ "for the proxy client {}",
95-
proxyConnection.ctx().channel(),
96-
ctx.channel(),
97-
writeFuture.cause());
98-
closeWithException(writeFuture.cause());
99-
}
83+
if (forwardClientAuthData && isRefresh) {
84+
proxyConnection.getValidClientAuthData()
85+
.thenApplyAsync(authData -> {
86+
ctx.writeAndFlush(Commands.newAuthResponse(clientAuthMethod, authData, this.protocolVersion,
87+
String.format("Pulsar-Java-v%s", PulsarVersion.getVersion())));
88+
return null;
89+
}, ctx.executor())
90+
.exceptionally(ex -> {
91+
log.warn("Failed to get valid client auth data. Closing connection.", ex);
92+
ctx.close();
93+
return null;
10094
});
101-
102-
if (state == State.SentConnectFrame) {
103-
state = State.Connecting;
104-
}
105-
} catch (Exception e) {
106-
log.error("Failed to send the auth challenge to origin client by the proxy {} for the proxy client {}",
107-
proxyConnection.ctx().channel(), ctx.channel(), e);
108-
closeWithException(e);
95+
} else {
96+
super.handleAuthChallenge(authChallenge);
10997
}
11098
}
11199
}

pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java

+7
Original file line numberDiff line numberDiff line change
@@ -348,6 +348,13 @@ public class ProxyConfiguration implements PulsarConfiguration {
348348
+ "to take effect"
349349
)
350350
private boolean forwardAuthorizationCredentials = false;
351+
352+
@FieldContext(
353+
category = CATEGORY_AUTHENTICATION,
354+
doc = "Interval of time for checking for expired authentication credentials. Disable by setting to 0."
355+
)
356+
private int authenticationRefreshCheckSeconds = 60;
357+
351358
@FieldContext(
352359
category = CATEGORY_AUTHENTICATION,
353360
doc = "Whether the '/metrics' endpoint requires authentication. Defaults to true."

0 commit comments

Comments
 (0)