Skip to content

Commit

Permalink
Issue apache#3653: Kerberos authentication for web resource support (a…
Browse files Browse the repository at this point in the history
…pache#4097)

Fixes apache#3653

Master Issue: apache#3491

** Motivation
Add kerberos support for web resource support.
This mainly include 2 parts:

- the HttpClient that works for HttpLookup.
- the BaseResource that works for admin rest end point.

*** Modifications
For kerberos authentication, there need several back/forth requests to do the negotiation between client and server.
This change add a method authenticationStage in AuthenticationSasl, and a method authenticateHttpRequest in AuthenticationProviderSasl to do the mutual negotiation.
And a saslRoleToken is cached in AuthenticationSasl once the authentication get success.
When do the sasl authentication, it will first use saslRoleToken cache, and if sever check this token failed, do real sasl authentication.
Changed unit test SaslAuthenticateTest, which enable sasl authentication in admin and also use http lookup to verify the change.
  • Loading branch information
jiazhai authored Apr 25, 2019
1 parent c642e5d commit 2777b0e
Show file tree
Hide file tree
Showing 31 changed files with 1,242 additions and 178 deletions.
5 changes: 0 additions & 5 deletions conf/broker.conf
Original file line number Diff line number Diff line change
Expand Up @@ -351,11 +351,6 @@ tokenAuthClaim=

### --- SASL Authentication Provider --- ###

# Whether Use SASL Authentication or not.
# TODO: used to bypass web resource check. will remove it after implementation the support.
# github issue #3653 {@link: https://github.com/apache/pulsar/issues/3653}
isSaslAuthentication=

# This is a regexp, which limits the range of possible ids which can connect to the Broker using SASL.
# Default value: `SaslConstants.JAAS_CLIENT_ALLOWED_IDS_DEFAULT`, which is ".*pulsar.*",
# so only clients whose id contains 'pulsar' are allowed to connect.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,21 +18,39 @@
*/
package org.apache.pulsar.broker.authentication;

import static org.apache.pulsar.common.sasl.SaslConstants.JAAS_SERVER_SECTION_NAME;
import static com.google.common.base.Preconditions.checkState;
import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedDataAttributeName;
import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedRoleAttributeName;
import static org.apache.pulsar.common.sasl.SaslConstants.JAAS_CLIENT_ALLOWED_IDS;
import static org.apache.pulsar.common.sasl.SaslConstants.JAAS_SERVER_SECTION_NAME;
import static org.apache.pulsar.common.sasl.SaslConstants.KINIT_COMMAND;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_AUTH_ROLE_TOKEN;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_AUTH_ROLE_TOKEN_EXPIRED;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_AUTH_TOKEN;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_HEADER_STATE;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_STATE_CLIENT_INIT;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_STATE_COMPLETE;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_STATE_NEGOTIATE;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_STATE_SERVER;
import static org.apache.pulsar.common.sasl.SaslConstants.SASL_STATE_SERVER_CHECK_TOKEN;

import java.io.IOException;
import java.net.SocketAddress;
import java.util.Base64;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;

import javax.naming.AuthenticationException;
import javax.net.ssl.SSLSession;
import javax.security.auth.login.LoginException;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;

import com.google.common.collect.Maps;
import lombok.SneakyThrows;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.common.api.AuthData;
Expand Down Expand Up @@ -76,6 +94,8 @@ public void initialize(ServiceConfiguration config) throws IOException {
throw new IOException(e);
}
}

this.signer = new SaslRoleTokenSigner(Long.toString(new Random().nextLong()).getBytes());
}

@Override
Expand Down Expand Up @@ -109,4 +129,167 @@ public AuthenticationState newAuthState(AuthData authData,
throw new AuthenticationException(t.getMessage());
}
}

// for http auth.
private static final long SASL_ROLE_TOKEN_LIVE_SECONDS = 3600;
// A signer for http role token, with random secret.
private SaslRoleTokenSigner signer;

/**
* Returns null if authentication has not completed.
* Return auth role if authentication has completed, and httpRequest's role token contains the authRole
*/
public String authRoleFromHttpRequest(HttpServletRequest httpRequest) throws AuthenticationException {
String tokenStr = httpRequest.getHeader(SASL_AUTH_ROLE_TOKEN);

if (tokenStr == null) {
return null;
}

String unSigned = signer.verifyAndExtract(tokenStr);
SaslRoleToken token;

try {
token = SaslRoleToken.parse(unSigned);
if (log.isDebugEnabled()) {
log.debug("server side get role token: {}, session in token:{}, session in request:{}",
token, token.getSession(), httpRequest.getRemoteAddr());
}
} catch (Exception e) {
log.error("token parse failed, with exception: ", e);
return SASL_AUTH_ROLE_TOKEN_EXPIRED;
}

if (!token.isExpired()) {
return token.getUserRole();
} else if (token.isExpired()) {
return SASL_AUTH_ROLE_TOKEN_EXPIRED;
} else {
return null;
}
}

private String createAuthRoleToken(String role, String sessionId) {
long expireAtMs = System.currentTimeMillis() + SASL_ROLE_TOKEN_LIVE_SECONDS * 1000; // 1 hour
SaslRoleToken token = new SaslRoleToken(role, sessionId, expireAtMs);

String signed = signer.sign(token.toString());
if (log.isDebugEnabled()) {
log.debug("create role token token: {}, role: {} session :{}, expires:{}\nsigned:{}",
token, token.getUserRole(), token.getSession(), token.getExpires(), signed);
}
return signed;
}

private ConcurrentHashMap<Long, AuthenticationState> authStates = new ConcurrentHashMap<>();

// return authState if it is in cache.
private AuthenticationState getAuthState(HttpServletRequest request) {
String id = request.getHeader(SASL_STATE_SERVER);
if (id == null) {
return null;
}

try {
return authStates.get(Long.parseLong(id));
} catch (NumberFormatException e) {
log.error("[{}] Wrong Id String in Token {}. e:", request.getRequestURI(),
id, e);
return null;
}
}

private void setResponseHeaderState(HttpServletResponse response, String state) {
response.setHeader(SaslConstants.SASL_HEADER_TYPE, SaslConstants.SASL_TYPE_VALUE);
response.setHeader(SASL_HEADER_STATE, state);
}

/**
* Passed in request, set response, according to request.
* and return whether we should do following chain.doFilter or not.
*/
@Override
public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletResponse response) throws Exception {
AuthenticationState state = getAuthState(request);
String saslAuthRoleToken = authRoleFromHttpRequest(request);

// role token exist
if (saslAuthRoleToken != null) {
// role token expired, send role token expired to client.
if (saslAuthRoleToken.equalsIgnoreCase(SASL_AUTH_ROLE_TOKEN_EXPIRED)) {
setResponseHeaderState(response, SASL_AUTH_ROLE_TOKEN_EXPIRED);
response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "Role token expired");
if (log.isDebugEnabled()) {
log.debug("[{}] Server side role token expired: {}", request.getRequestURI(), saslAuthRoleToken);
}
return false;
}

// role token OK to use,
// if request is ask for role token verify, send auth complete to client
// if request is a real request with valid role token, pass this request down.
if (request.getHeader(SASL_HEADER_STATE).equalsIgnoreCase(SASL_STATE_COMPLETE)) {
request.setAttribute(AuthenticatedRoleAttributeName, saslAuthRoleToken);
request.setAttribute(AuthenticatedDataAttributeName,
new AuthenticationDataHttps(request));
if (log.isDebugEnabled()) {
log.debug("[{}] Server side role token OK to go on: {}", request.getRequestURI(), saslAuthRoleToken);
}
return true;
} else {
checkState(request.getHeader(SASL_HEADER_STATE).equalsIgnoreCase(SASL_STATE_SERVER_CHECK_TOKEN));
setResponseHeaderState(response, SASL_STATE_COMPLETE);
response.setHeader(SASL_STATE_SERVER, request.getHeader(SASL_STATE_SERVER));
response.setStatus(HttpServletResponse.SC_OK);
if (log.isDebugEnabled()) {
log.debug("[{}] Server side role token verified success: {}", request.getRequestURI(), saslAuthRoleToken);
}
return false;
}
} else {
// no role token, do sasl auth
// need new authState
if (state == null || request.getHeader(SASL_HEADER_STATE).equalsIgnoreCase(SASL_STATE_CLIENT_INIT)) {
state = newAuthState(null, null, null);
authStates.put(state.getStateId(), state);
}
checkState(request.getHeader(SASL_AUTH_TOKEN) != null,
"Header token should exist if no role token.");

// do the sasl auth
AuthData clientData = AuthData.of(Base64.getDecoder().decode(
request.getHeader(SASL_AUTH_TOKEN)));
AuthData brokerData = state.authenticate(clientData);

// authentication has completed, it has get the auth role.
if (state.isComplete()) {
if (log.isDebugEnabled()) {
log.debug("[{}] SASL server authentication complete, send OK to client.", request.getRequestURI());
}
String authRole = state.getAuthRole();
String authToken = createAuthRoleToken(authRole, String.valueOf(state.getStateId()));
response.setHeader(SASL_AUTH_ROLE_TOKEN, authToken);

// auth request complete, return OK, wait for a new real request to come.
response.setHeader(SASL_STATE_SERVER, String.valueOf(state.getStateId()));
setResponseHeaderState(response, SASL_STATE_COMPLETE);
response.setStatus(HttpServletResponse.SC_OK);

// auth completed, no need to keep authState
authStates.remove(state.getStateId());
return false;
} else {
// auth not complete
if (log.isDebugEnabled()) {
log.debug("[{}] SASL server authentication not complete, send {} back to client.",
request.getRequestURI(), HttpServletResponse.SC_UNAUTHORIZED);
}
setResponseHeaderState(response, SASL_STATE_NEGOTIATE);
response.setHeader(SASL_STATE_SERVER, String.valueOf(state.getStateId()));
response.setHeader(SASL_AUTH_TOKEN, Base64.getEncoder().encodeToString(brokerData.getBytes()));
response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "SASL Authentication not complete.");
return false;
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -186,18 +186,6 @@ private void handleAuthorizeCallback(AuthorizeCallback ac) {
ac.setAuthorized(true);
log.info("Successfully authenticated client: authenticationID: {}; authorizationID: {}.",
authenticationID, authorizationID);

KerberosName kerberosName = new KerberosName(authenticationID);
try {
StringBuilder userNameBuilder = new StringBuilder(kerberosName.getShortName());
userNameBuilder.append("/").append(kerberosName.getHostName());
userNameBuilder.append("@").append(kerberosName.getRealm());

log.info("Setting authorizedID: {} ", userNameBuilder);
ac.setAuthorizedID(userNameBuilder.toString());
} catch (IOException e) {
log.error("Failed to set name based on Kerberos authentication rules.");
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,10 +53,4 @@ public boolean isComplete() {
public String getAuthorizationID() {
return pulsarSaslServer.getAuthorizationID();
}

// TODO: for http support. github issue #3653 {@link: https://github.com/apache/pulsar/issues/3653}
/* default boolean hasDataFromHttp() {
return false;
}*/

}
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,11 @@

import static com.google.common.base.Preconditions.checkArgument;

import java.util.concurrent.atomic.AtomicLong;

import javax.naming.AuthenticationException;

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

/**
Expand All @@ -31,10 +34,14 @@
* It is basically holding the the authentication state.
* It tell broker whether the authentication is completed or not,
*/
@Slf4j
public class SaslAuthenticationState implements AuthenticationState {
private SaslAuthenticationDataSource authenticationDataSource;
private final long stateId;
private static final AtomicLong stateIdGenerator = new AtomicLong(0L);
private final SaslAuthenticationDataSource authenticationDataSource;

public SaslAuthenticationState(AuthenticationDataSource authenticationDataSource) {
stateId = stateIdGenerator.incrementAndGet();
checkArgument(authenticationDataSource instanceof SaslAuthenticationDataSource);
this.authenticationDataSource = (SaslAuthenticationDataSource)authenticationDataSource;
}
Expand All @@ -58,9 +65,15 @@ public boolean isComplete() {
* Returns null if authentication has completed, and no auth data is required to send back to client.
* Do auth and Returns the auth data back to client, if authentication has not completed.
*/
@Override
public AuthData authenticate(AuthData authData) throws AuthenticationException {
return authenticationDataSource.authenticate(authData);
}

@Override
public long getStateId() {
return stateId;
}


}
Loading

0 comments on commit 2777b0e

Please sign in to comment.