Skip to content

Commit

Permalink
[fix][proxy] Fix refresh client auth
Browse files Browse the repository at this point in the history
Signed-off-by: Zixuan Liu <[email protected]>
  • Loading branch information
nodece committed Sep 27, 2022
1 parent 08df28a commit c2a83b8
Show file tree
Hide file tree
Showing 6 changed files with 381 additions and 76 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import javax.naming.AuthenticationException;
import lombok.AccessLevel;
import lombok.Getter;
import org.apache.commons.lang3.exception.ExceptionUtils;
Expand Down Expand Up @@ -176,6 +177,9 @@ public class ClientCnx extends PulsarHandler {
@Getter
private final ClientCnxIdleState idleState;

@Getter
private long lastDisconnectedTimestamp;

enum State {
None, SentConnectFrame, Ready, Failed, Connecting
}
Expand Down Expand Up @@ -253,34 +257,53 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception {
} else {
log.info("{} Connected through proxy to target broker at {}", ctx.channel(), proxyToTargetBrokerAddress);
}
// Send CONNECT command
ctx.writeAndFlush(newConnectCommand())
.addListener(future -> {
if (future.isSuccess()) {
if (log.isDebugEnabled()) {
log.debug("Complete: {}", future.isSuccess());
}
state = State.SentConnectFrame;
} else {
log.warn("Error during handshake", future.cause());
ctx.close();
}
});
handleChannelActive();
}

protected String getOriginalAuthRole() {
return null;
}

protected CompletableFuture<AuthData> getOriginalAuthDataSupplier(boolean isRefresh) {
return CompletableFuture.completedFuture(null);
}

protected String getOriginalAuthMethod() {
return null;
}

protected ByteBuf newConnectCommand() throws Exception {
protected void handleChannelActive() throws Exception {
// mutual authentication is to auth between `remoteHostName` and this client for this channel.
// each channel will have a mutual client/server pair, mutual client evaluateChallenge with init data,
// and return authData to server.
authenticationDataProvider = authentication.getAuthData(remoteHostName);
AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
return Commands.newConnect(authentication.getAuthMethodName(), authData, this.protocolVersion,
PulsarVersion.getVersion(), proxyToTargetBrokerAddress, null, null, null);
getOriginalAuthDataSupplier(false).thenAccept(originalAuthData -> {
ByteBuf byteBuf = Commands.newConnect(authentication.getAuthMethodName(), authData, this.protocolVersion,
PulsarVersion.getVersion(), proxyToTargetBrokerAddress, getOriginalAuthRole(), originalAuthData,
getOriginalAuthMethod());
ctx.writeAndFlush(byteBuf).addListener(future -> {
if (future.isSuccess()) {
if (log.isDebugEnabled()) {
log.debug("Complete: {}", future.isSuccess());
}
state = State.SentConnectFrame;
} else {
closeWithException(future.cause());
log.error("Error during handshake", future.cause());
}
});
}).exceptionally(e -> {
closeWithException(e);
log.error("Error during handshake", e);
return null;
});
}

@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
super.channelInactive(ctx);
lastDisconnectedTimestamp = System.currentTimeMillis();
log.info("{} Disconnected", ctx.channel());
if (!connectionFuture.isDone()) {
connectionFuture.completeExceptionally(new PulsarClientException("Connection already closed"));
Expand Down Expand Up @@ -361,6 +384,19 @@ protected void handleConnected(CommandConnected connected) {
state = State.Ready;
}

protected CompletableFuture<AuthData> getMutualAuthData(CommandAuthChallenge authChallenge) {
try {
return CompletableFuture.completedFuture(
authenticationDataProvider.authenticate(AuthData.of(authChallenge.getChallenge().getAuthData())));
} catch (AuthenticationException e) {
return FutureUtil.failedFuture(e);
}
}

protected String getMutualAuthMethod() {
return authentication.getAuthMethodName();
}

@Override
protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
checkArgument(authChallenge.hasChallenge());
Expand All @@ -371,42 +407,39 @@ protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
authenticationDataProvider = authentication.getAuthData(remoteHostName);
} catch (PulsarClientException e) {
log.error("{} Error when refreshing authentication data provider: {}", ctx.channel(), e);
connectionFuture.completeExceptionally(e);
closeWithException(e);
return;
}
}

// mutual authn. If auth not complete, continue auth; if auth complete, complete connectionFuture.
try {
AuthData authData = authenticationDataProvider
.authenticate(AuthData.of(authChallenge.getChallenge().getAuthData()));

checkState(!authData.isComplete());

ByteBuf request = Commands.newAuthResponse(authentication.getAuthMethodName(),
authData,
this.protocolVersion,
PulsarVersion.getVersion());

if (log.isDebugEnabled()) {
log.debug("{} Mutual auth {}", ctx.channel(), authentication.getAuthMethodName());
}
getMutualAuthData(authChallenge).thenAccept(authData->{
checkState(!authData.isComplete());
String authMethod = getMutualAuthMethod();
if (log.isDebugEnabled()) {
log.debug("{} Mutual auth {}", ctx.channel(), authMethod);
}

ctx.writeAndFlush(request).addListener(writeFuture -> {
if (!writeFuture.isSuccess()) {
log.warn("{} Failed to send request for mutual auth to broker: {}", ctx.channel(),
writeFuture.cause().getMessage());
connectionFuture.completeExceptionally(writeFuture.cause());
ByteBuf request =
Commands.newAuthResponse(authMethod, authData, protocolVersion, PulsarVersion.getVersion());
ctx.writeAndFlush(request).addListener(writeFuture -> {
if (!writeFuture.isSuccess()) {
log.warn("{} Failed to send request for mutual auth to broker: {}", ctx.channel(),
writeFuture.cause().getMessage());
closeWithException(writeFuture.cause());
}
});
if (state == State.SentConnectFrame) {
state = State.Connecting;
}
}).exceptionally(e->{
log.error("{} Error mutual verify: {}", ctx.channel(), e);
closeWithException(e);
return null;
});

if (state == State.SentConnectFrame) {
state = State.Connecting;
}
} catch (Exception e) {
log.error("{} Error mutual verify: {}", ctx.channel(), e);
connectionFuture.completeExceptionally(e);
return;
closeWithException(e);
}
}

Expand Down Expand Up @@ -1243,6 +1276,13 @@ public void close() {
}
}

protected void closeWithException(Throwable e) {
if (ctx != null) {
ctx.close();
connectionFuture.completeExceptionally(e);
}
}

private void checkRequestTimeout() {
while (!requestTimeoutQueue.isEmpty()) {
RequestTime request = requestTimeoutQueue.peek();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -453,5 +453,9 @@ public void doMarkAndReleaseUselessConnections(){
// Do release idle connections.
releaseIdleConnectionTaskList.forEach(Runnable::run);
}
}

@VisibleForTesting
public ConcurrentMap<InetSocketAddress, ConcurrentMap<Integer, CompletableFuture<ClientCnx>>> getPool() {
return pool;
}
}
5 changes: 5 additions & 0 deletions pulsar-proxy/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,11 @@
<artifactId>ipaddress</artifactId>
<version>${seancfoley.ipaddress.version}</version>
</dependency>
<dependency>
<groupId>org.awaitility</groupId>
<artifactId>awaitility</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<plugins>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,46 +18,59 @@
*/
package org.apache.pulsar.proxy.server;

import io.netty.buffer.ByteBuf;
import io.netty.channel.EventLoopGroup;
import org.apache.pulsar.PulsarVersion;
import java.util.Arrays;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import org.apache.pulsar.client.impl.ClientCnx;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.common.api.AuthData;
import org.apache.pulsar.common.protocol.Commands;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.pulsar.common.api.proto.CommandAuthChallenge;

public class ProxyClientCnx extends ClientCnx {
private final boolean forwardClientAuthData;
private final String clientAuthMethod;
private final String clientAuthRole;
private final Function<Boolean, CompletableFuture<AuthData>> clientAuthDataSupplier;

String clientAuthRole;
AuthData clientAuthData;
String clientAuthMethod;
int protocolVersion;

@Deprecated
public ProxyClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, String clientAuthRole,
AuthData clientAuthData, String clientAuthMethod, int protocolVersion) {
super(conf, eventLoopGroup);
this(conf, eventLoopGroup, clientAuthRole, (isRefresh) -> CompletableFuture.completedFuture(clientAuthData),
clientAuthMethod, protocolVersion, clientAuthData != null);
}

public ProxyClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, String clientAuthRole,
Function<Boolean, CompletableFuture<AuthData>> clientAuthDataSupplier,
String clientAuthMethod, int protocolVersion, boolean forwardClientAuthData) {
super(conf, eventLoopGroup, protocolVersion);
this.clientAuthRole = clientAuthRole;
this.clientAuthData = clientAuthData;
this.clientAuthDataSupplier = clientAuthDataSupplier;
this.clientAuthMethod = clientAuthMethod;
this.protocolVersion = protocolVersion;
this.forwardClientAuthData = forwardClientAuthData;
}

@Override
protected ByteBuf newConnectCommand() throws Exception {
if (log.isDebugEnabled()) {
log.debug("New Connection opened via ProxyClientCnx with params clientAuthRole = {},"
+ " clientAuthData = {}, clientAuthMethod = {}",
clientAuthRole, clientAuthData, clientAuthMethod);
}
protected String getOriginalAuthRole() {
return this.clientAuthRole;
}

authenticationDataProvider = authentication.getAuthData(remoteHostName);
AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
return Commands.newConnect(authentication.getAuthMethodName(), authData, this.protocolVersion,
PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
clientAuthMethod);
@Override
protected String getOriginalAuthMethod() {
return this.clientAuthMethod;
}

private static final Logger log = LoggerFactory.getLogger(ProxyClientCnx.class);
@Override
protected CompletableFuture<AuthData> getOriginalAuthDataSupplier(boolean isRefresh) {
return clientAuthDataSupplier.apply(isRefresh);
}

@Override
protected CompletableFuture<AuthData> getMutualAuthData(CommandAuthChallenge authChallenge) {
boolean isRefresh = Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData());
if (!forwardClientAuthData || !isRefresh) {
return super.getMutualAuthData(authChallenge);
}
return getOriginalAuthDataSupplier(true);
}
}
Loading

0 comments on commit c2a83b8

Please sign in to comment.