Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[PIP 97][SASL Authentication] Remove Deprecated SASL AuthenticationDataSource#authenticate Implementation #12955

Merged
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 @@ -119,9 +119,8 @@ public AuthenticationState newAuthState(AuthData authData,
SocketAddress remoteAddress,
SSLSession sslSession) throws AuthenticationException {
try {
return new SaslAuthenticationState(
new SaslAuthenticationDataSource(
new PulsarSaslServer(jaasCredentialsContainer.getSubject(), allowedIdsPattern)));
PulsarSaslServer server = new PulsarSaslServer(jaasCredentialsContainer.getSubject(), allowedIdsPattern);
return new SaslAuthenticationState(server);
} catch (Throwable t) {
log.error("Failed create sasl auth state" , t);
throw new AuthenticationException(t.getMessage());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,12 @@
*/
package org.apache.pulsar.broker.authentication;

import javax.naming.AuthenticationException;

import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.common.api.AuthData;

@Slf4j
public class SaslAuthenticationDataSource implements AuthenticationDataSource {
private static final long serialVersionUID = 1L;

// server side token data, that will passed to sasl client side.
protected AuthData serverSideToken;
private PulsarSaslServer pulsarSaslServer;

public SaslAuthenticationDataSource(PulsarSaslServer saslServer) {
Expand All @@ -40,16 +35,6 @@ public boolean hasDataFromCommand() {
return true;
}

@Override
public AuthData authenticate(AuthData data) throws AuthenticationException {
serverSideToken = pulsarSaslServer.response(data);
return serverSideToken;
}

public boolean isComplete() {
return this.pulsarSaslServer.isComplete();
}

public String getAuthorizationID() {
return pulsarSaslServer.getAuthorizationID();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,16 +39,17 @@ public class SaslAuthenticationState implements AuthenticationState {
private final long stateId;
private static final AtomicLong stateIdGenerator = new AtomicLong(0L);
private final SaslAuthenticationDataSource authenticationDataSource;
private PulsarSaslServer pulsarSaslServer;

public SaslAuthenticationState(AuthenticationDataSource authenticationDataSource) {
public SaslAuthenticationState(PulsarSaslServer server) {
stateId = stateIdGenerator.incrementAndGet();
checkArgument(authenticationDataSource instanceof SaslAuthenticationDataSource);
this.authenticationDataSource = (SaslAuthenticationDataSource)authenticationDataSource;
this.authenticationDataSource = new SaslAuthenticationDataSource(server);
this.pulsarSaslServer = server;
}

@Override
public String getAuthRole() {
return authenticationDataSource.getAuthorizationID();
return pulsarSaslServer.getAuthorizationID();
}

@Override
Expand All @@ -58,7 +59,7 @@ public AuthenticationDataSource getAuthDataSource() {

@Override
public boolean isComplete() {
return authenticationDataSource.isComplete();
return pulsarSaslServer.isComplete();
}

/**
Expand All @@ -67,7 +68,7 @@ public boolean isComplete() {
*/
@Override
public AuthData authenticate(AuthData authData) throws AuthenticationException {
return authenticationDataSource.authenticate(authData);
return pulsarSaslServer.response(authData);
}

@Override
Expand Down