You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by pe...@apache.org on 2021/01/06 05:33:23 UTC

[pulsar] branch branch-2.6 updated: [Authentication] Support chained authentication with same auth method name (#9094)

This is an automated email from the ASF dual-hosted git repository.

penghui pushed a commit to branch branch-2.6
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-2.6 by this push:
     new bae9a12  [Authentication] Support chained authentication with same auth method name (#9094)
bae9a12 is described below

commit bae9a1218137682ef908e292f256ee9a6536628a
Author: Sijie Guo <si...@apache.org>
AuthorDate: Tue Jan 5 15:36:31 2021 -0800

    [Authentication] Support chained authentication with same auth method name (#9094)
    
    *Motivation*
    
    Chained authentication is a very useful mechanism for migrating a cluster from
    one authentication provider to the other authentication provider. However,
    Pulsar doesn't support configuring multiple authentication providers with same
    auth method name.
    
    For example, a Pulsar cluster was using standard JWT authentication initially.
    The users want to upgrade the Pulsar cluster to use an OAuth2 authentication
    mechanism. But both JWT and OAuth2 share the same authentication method name.
    
    This change improves the authentication logic to support chained authentication
    with same auth method name.
    
    (cherry picked from commit c2a4e6607ab6693cf93851867f649430c3c6ba54)
---
 .../authentication/SaslAuthenticateTest.java       |   4 +-
 .../authentication/AuthenticationProviderList.java | 218 +++++++++++++++++++++
 .../AuthenticationProviderToken.java               |  58 ++++--
 .../authentication/AuthenticationService.java      |  28 ++-
 .../AuthenticationProviderListTest.java            | 191 ++++++++++++++++++
 5 files changed, 474 insertions(+), 25 deletions(-)

diff --git a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java
index c57d002..e94d2ae 100644
--- a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java
+++ b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java
@@ -251,9 +251,11 @@ public class SaslAuthenticateTest extends ProducerConsumerBase {
 
         // prepare client and server side resource
         AuthenticationDataProvider dataProvider =  authSasl.getAuthData(hostName);
-        AuthenticationProviderSasl saslServer  = (AuthenticationProviderSasl)
+        AuthenticationProviderList providerList = (AuthenticationProviderList)
             (pulsar.getBrokerService().getAuthenticationService()
                 .getAuthenticationProvider(SaslConstants.AUTH_METHOD_NAME));
+        AuthenticationProviderSasl saslServer =
+            (AuthenticationProviderSasl) providerList.getProviders().get(0);
         AuthenticationState authState = saslServer.newAuthState(null, null, null);
 
         // auth between server and client.
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java
new file mode 100644
index 0000000..2d55f51
--- /dev/null
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java
@@ -0,0 +1,218 @@
+/**
+ * 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.broker.authentication;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import javax.naming.AuthenticationException;
+import javax.net.ssl.SSLSession;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.common.api.AuthData;
+
+/**
+ * An authentication provider wraps a list of auth providers.
+ */
+@Slf4j
+public class AuthenticationProviderList implements AuthenticationProvider {
+
+    private interface AuthProcessor<T, P> {
+
+        T apply(P process) throws AuthenticationException;
+
+    }
+
+    static <T, P> T applyAuthProcessor(List<P> processors, AuthProcessor<T, P> authFunc)
+        throws AuthenticationException {
+        AuthenticationException authenticationException = null;
+        for (P ap : processors) {
+            try {
+                return authFunc.apply(ap);
+            } catch (AuthenticationException ae) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Authentication failed for auth provider " + ap.getClass() + ": ", ae);
+                }
+                // Store the exception so we can throw it later instead of a generic one
+                authenticationException = ae;
+            }
+        }
+
+        if (null == authenticationException) {
+            throw new AuthenticationException("Authentication required");
+        } else {
+            throw authenticationException;
+        }
+
+    }
+
+    private static class AuthenticationListState implements AuthenticationState {
+
+        private final List<AuthenticationState> states;
+        private AuthenticationState authState;
+
+        AuthenticationListState(List<AuthenticationState> states) {
+            this.states = states;
+            this.authState = states.get(0);
+        }
+
+        private AuthenticationState getAuthState() throws AuthenticationException {
+            if (authState != null) {
+                return authState;
+            } else {
+                throw new AuthenticationException("Authentication state is not initialized");
+            }
+        }
+
+        @Override
+        public String getAuthRole() throws AuthenticationException {
+            return getAuthState().getAuthRole();
+        }
+
+        @Override
+        public AuthData authenticate(AuthData authData) throws AuthenticationException {
+            return applyAuthProcessor(
+                states,
+                as -> {
+                    AuthData ad = as.authenticate(authData);
+                    AuthenticationListState.this.authState = as;
+                    return ad;
+                }
+            );
+        }
+
+        @Override
+        public AuthenticationDataSource getAuthDataSource() {
+            return authState.getAuthDataSource();
+        }
+
+        @Override
+        public boolean isComplete() {
+            return authState.isComplete();
+        }
+
+        @Override
+        public long getStateId() {
+            if (null != authState) {
+                return authState.getStateId();
+            } else {
+                return states.get(0).getStateId();
+            }
+        }
+
+        @Override
+        public boolean isExpired() {
+            return authState.isExpired();
+        }
+
+        @Override
+        public AuthData refreshAuthentication() throws AuthenticationException {
+            return getAuthState().refreshAuthentication();
+        }
+    }
+
+    private final List<AuthenticationProvider> providers;
+
+    public AuthenticationProviderList(List<AuthenticationProvider> providers) {
+        this.providers = providers;
+    }
+
+    public List<AuthenticationProvider> getProviders() {
+        return providers;
+    }
+
+    @Override
+    public void initialize(ServiceConfiguration config) throws IOException {
+        for (AuthenticationProvider ap : providers) {
+            ap.initialize(config);
+        }
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return providers.get(0).getAuthMethodName();
+    }
+
+    @Override
+    public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+        return applyAuthProcessor(
+            providers,
+            provider -> provider.authenticate(authData)
+        );
+    }
+
+    @Override
+    public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession)
+        throws AuthenticationException {
+        final List<AuthenticationState> states = new ArrayList<>(providers.size());
+
+        AuthenticationException authenticationException = null;
+        try {
+            applyAuthProcessor(
+                providers,
+                provider -> {
+                    AuthenticationState state = provider.newAuthState(authData, remoteAddress, sslSession);
+                    states.add(state);
+                    return state;
+                }
+            );
+        } catch (AuthenticationException ae) {
+            authenticationException = ae;
+        }
+        if (states.isEmpty()) {
+            log.error("Failed to initialize a new auth state from {}", remoteAddress, authenticationException);
+            if (authenticationException != null) {
+                throw authenticationException;
+            } else {
+                throw new AuthenticationException("Failed to initialize a new auth state from " + remoteAddress);
+            }
+        } else {
+            return new AuthenticationListState(states);
+        }
+    }
+
+    @Override
+    public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletResponse response) throws Exception {
+        Boolean authenticated = applyAuthProcessor(
+            providers,
+            provider -> {
+                try {
+                    return provider.authenticateHttpRequest(request, response);
+                } catch (Exception e) {
+                    if (e instanceof AuthenticationException) {
+                        throw (AuthenticationException) e;
+                    } else {
+                        throw new AuthenticationException("Failed to authentication http request");
+                    }
+                }
+            }
+        );
+        return authenticated.booleanValue();
+    }
+
+    @Override
+    public void close() throws IOException {
+        for (AuthenticationProvider provider : providers) {
+            provider.close();
+        }
+    }
+}
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
index da21a8e..66971e3 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java
@@ -46,6 +46,9 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
     final static String HTTP_HEADER_VALUE_PREFIX = "Bearer ";
 
     // When symmetric key is configured
+    final static String CONF_TOKEN_SETTING_PREFIX = "";
+
+    // When symmetric key is configured
     final static String CONF_TOKEN_SECRET_KEY = "tokenSecretKey";
 
     // When public/private key pair is configured
@@ -71,6 +74,14 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
     private String audienceClaim;
     private String audience;
 
+    // config keys
+    private String confTokenSecretKeySettingName;
+    private String confTokenPublicKeySettingName;
+    private String confTokenAuthClaimSettingName;
+    private String confTokenPublicAlgSettingName;
+    private String confTokenAudienceClaimSettingName;
+    private String confTokenAudienceSettingName;
+
     @Override
     public void close() throws IOException {
         // noop
@@ -78,6 +89,17 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
 
     @Override
     public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException {
+        String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX);
+        if (null == prefix) {
+            prefix = "";
+        }
+        this.confTokenSecretKeySettingName = prefix + CONF_TOKEN_SECRET_KEY;
+        this.confTokenPublicKeySettingName = prefix + CONF_TOKEN_PUBLIC_KEY;
+        this.confTokenAuthClaimSettingName = prefix + CONF_TOKEN_AUTH_CLAIM;
+        this.confTokenPublicAlgSettingName = prefix + CONF_TOKEN_PUBLIC_ALG;
+        this.confTokenAudienceClaimSettingName = prefix + CONF_TOKEN_AUDIENCE_CLAIM;
+        this.confTokenAudienceSettingName = prefix + CONF_TOKEN_AUDIENCE;
+
         // we need to fetch the algorithm before we fetch the key
         this.publicKeyAlg = getPublicKeyAlgType(config);
         this.validationKey = getValidationKey(config);
@@ -184,14 +206,14 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
      * Try to get the validation key for tokens from several possible config options.
      */
     private Key getValidationKey(ServiceConfiguration conf) throws IOException {
-        if (conf.getProperty(CONF_TOKEN_SECRET_KEY) != null
-                && StringUtils.isNotBlank((String) conf.getProperty(CONF_TOKEN_SECRET_KEY))) {
-            final String validationKeyConfig = (String) conf.getProperty(CONF_TOKEN_SECRET_KEY);
+        if (conf.getProperty(confTokenSecretKeySettingName) != null
+                && StringUtils.isNotBlank((String) conf.getProperty(confTokenSecretKeySettingName))) {
+            final String validationKeyConfig = (String) conf.getProperty(confTokenSecretKeySettingName);
             final byte[] validationKey = AuthTokenUtils.readKeyFromUrl(validationKeyConfig);
             return AuthTokenUtils.decodeSecretKey(validationKey);
-        } else if (conf.getProperty(CONF_TOKEN_PUBLIC_KEY) != null
-                && StringUtils.isNotBlank((String) conf.getProperty(CONF_TOKEN_PUBLIC_KEY))) {
-            final String validationKeyConfig = (String) conf.getProperty(CONF_TOKEN_PUBLIC_KEY);
+        } else if (conf.getProperty(confTokenPublicKeySettingName) != null
+                && StringUtils.isNotBlank((String) conf.getProperty(confTokenPublicKeySettingName))) {
+            final String validationKeyConfig = (String) conf.getProperty(confTokenPublicKeySettingName);
             final byte[] validationKey = AuthTokenUtils.readKeyFromUrl(validationKeyConfig);
             return AuthTokenUtils.decodePublicKey(validationKey, publicKeyAlg);
         } else {
@@ -200,18 +222,18 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
     }
 
     private String getTokenRoleClaim(ServiceConfiguration conf) throws IOException {
-        if (conf.getProperty(CONF_TOKEN_AUTH_CLAIM) != null
-                && StringUtils.isNotBlank((String) conf.getProperty(CONF_TOKEN_AUTH_CLAIM))) {
-            return (String) conf.getProperty(CONF_TOKEN_AUTH_CLAIM);
+        if (conf.getProperty(confTokenAuthClaimSettingName) != null
+                && StringUtils.isNotBlank((String) conf.getProperty(confTokenAuthClaimSettingName))) {
+            return (String) conf.getProperty(confTokenAuthClaimSettingName);
         } else {
             return Claims.SUBJECT;
         }
     }
 
     private SignatureAlgorithm getPublicKeyAlgType(ServiceConfiguration conf) throws IllegalArgumentException {
-        if (conf.getProperty(CONF_TOKEN_PUBLIC_ALG) != null
-                && StringUtils.isNotBlank((String) conf.getProperty(CONF_TOKEN_PUBLIC_ALG))) {
-            String alg = (String) conf.getProperty(CONF_TOKEN_PUBLIC_ALG);
+        if (conf.getProperty(confTokenPublicAlgSettingName) != null
+                && StringUtils.isNotBlank((String) conf.getProperty(confTokenPublicAlgSettingName))) {
+            String alg = (String) conf.getProperty(confTokenPublicAlgSettingName);
             try {
                 return SignatureAlgorithm.forName(alg);
             } catch (SignatureException ex) {
@@ -224,9 +246,9 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
 
     // get Token Audience Claim from configuration, if not configured return null.
     private String getTokenAudienceClaim(ServiceConfiguration conf) throws IllegalArgumentException {
-        if (conf.getProperty(CONF_TOKEN_AUDIENCE_CLAIM) != null
-            && StringUtils.isNotBlank((String) conf.getProperty(CONF_TOKEN_AUDIENCE_CLAIM))) {
-            return (String) conf.getProperty(CONF_TOKEN_AUDIENCE_CLAIM);
+        if (conf.getProperty(confTokenAudienceClaimSettingName) != null
+            && StringUtils.isNotBlank((String) conf.getProperty(confTokenAudienceClaimSettingName))) {
+            return (String) conf.getProperty(confTokenAudienceClaimSettingName);
         } else {
             return null;
         }
@@ -234,9 +256,9 @@ public class AuthenticationProviderToken implements AuthenticationProvider {
 
     // get Token Audience that stands for this broker from configuration, if not configured return null.
     private String getTokenAudience(ServiceConfiguration conf) throws IllegalArgumentException {
-        if (conf.getProperty(CONF_TOKEN_AUDIENCE) != null
-            && StringUtils.isNotBlank((String) conf.getProperty(CONF_TOKEN_AUDIENCE))) {
-            return (String) conf.getProperty(CONF_TOKEN_AUDIENCE);
+        if (conf.getProperty(confTokenAudienceSettingName) != null
+            && StringUtils.isNotBlank((String) conf.getProperty(confTokenAudienceSettingName))) {
+            return (String) conf.getProperty(confTokenAudienceSettingName);
         } else {
             return null;
         }
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java
index 2cefad9..976ae44 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java
@@ -18,11 +18,15 @@
  */
 package org.apache.pulsar.broker.authentication;
 
+import com.google.common.collect.Maps;
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-
 import java.util.Optional;
+import java.util.stream.Collectors;
 import javax.naming.AuthenticationException;
 import javax.servlet.http.HttpServletRequest;
 
@@ -32,8 +36,6 @@ import org.apache.pulsar.broker.ServiceConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
-
 /**
  * Authentication service
  *
@@ -48,16 +50,30 @@ public class AuthenticationService implements Closeable {
         anonymousUserRole = conf.getAnonymousUserRole();
         if (conf.isAuthenticationEnabled()) {
             try {
-                AuthenticationProvider provider;
+                Map<String, List<AuthenticationProvider>> providerMap = new HashMap<>();
                 for (String className : conf.getAuthenticationProviders()) {
                     if (className.isEmpty()) {
                         continue;
                     }
-                    provider = (AuthenticationProvider) Class.forName(className).newInstance();
+                    AuthenticationProvider provider = (AuthenticationProvider) Class.forName(className).newInstance();
+
+                    List<AuthenticationProvider> providerList = providerMap.get(provider.getAuthMethodName());
+                    if (null == providerList) {
+                        providerList = new ArrayList<>(1);
+                        providerMap.put(provider.getAuthMethodName(), providerList);
+                    }
+                    providerList.add(provider);
+                }
+
+                for (Map.Entry<String, List<AuthenticationProvider>> entry : providerMap.entrySet()) {
+                    AuthenticationProviderList provider = new AuthenticationProviderList(entry.getValue());
                     provider.initialize(conf);
                     providers.put(provider.getAuthMethodName(), provider);
-                    LOG.info("{} has been loaded.", className);
+                    LOG.info("[{}] has been loaded.",
+                        entry.getValue().stream().map(
+                            p -> p.getClass().getName()).collect(Collectors.joining(",")));
                 }
+
                 if (providers.isEmpty()) {
                     LOG.warn("No authentication providers are loaded.");
                 }
diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java
new file mode 100644
index 0000000..8b034c6
--- /dev/null
+++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java
@@ -0,0 +1,191 @@
+/**
+ * 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.broker.authentication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
+import io.jsonwebtoken.SignatureAlgorithm;
+import io.jsonwebtoken.io.Decoders;
+import io.jsonwebtoken.security.Keys;
+import java.security.KeyPair;
+import java.security.PrivateKey;
+import java.util.Date;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.common.api.AuthData;
+import org.assertj.core.util.Lists;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+/**
+ * Unit test for {@link AuthenticationProviderList}.
+ */
+public class AuthenticationProviderListTest {
+
+    private static final String SUBJECT_A = "my-subject-a";
+    private static final String SUBJECT_B = "my-subject-b";
+
+    private AuthenticationProviderToken providerA;
+    private KeyPair keyPairA;
+    private AuthenticationProviderToken providerB;
+    private KeyPair keyPairB;
+    private String tokenAA;
+    private String tokenAB;
+    private String tokenBA;
+    private String tokenBB;
+    private String expiringTokenAA;
+    private String expiringTokenAB;
+    private String expiringTokenBA;
+    private String expiringTokenBB;
+
+    private AuthenticationProviderList authProvider;
+
+    @BeforeMethod
+    public void setUp() throws Exception {
+        this.keyPairA = Keys.keyPairFor(SignatureAlgorithm.ES256);
+        this.keyPairB = Keys.keyPairFor(SignatureAlgorithm.RS512);
+
+        this.providerA = new AuthenticationProviderToken();
+        this.providerB = new AuthenticationProviderToken();
+
+        Properties propertiesA = new Properties();
+        propertiesA.setProperty(AuthenticationProviderToken.CONF_TOKEN_SETTING_PREFIX, "a");
+        propertiesA.setProperty(
+            "a" + AuthenticationProviderToken.CONF_TOKEN_PUBLIC_KEY,
+            AuthTokenUtils.encodeKeyBase64(keyPairA.getPublic()));
+        propertiesA.setProperty(
+            "a" + AuthenticationProviderToken.CONF_TOKEN_PUBLIC_ALG,
+            SignatureAlgorithm.ES256.getValue()
+        );
+        ServiceConfiguration confA = new ServiceConfiguration();
+        confA.setProperties(propertiesA);
+        providerA.initialize(confA);
+
+        Properties propertiesB = new Properties();
+        propertiesB.setProperty(AuthenticationProviderToken.CONF_TOKEN_SETTING_PREFIX, "b");
+        propertiesB.setProperty(
+            "b" + AuthenticationProviderToken.CONF_TOKEN_PUBLIC_KEY,
+            AuthTokenUtils.encodeKeyBase64(keyPairB.getPublic()));
+        propertiesB.setProperty(
+            "b" + AuthenticationProviderToken.CONF_TOKEN_PUBLIC_ALG,
+            SignatureAlgorithm.RS512.getValue()
+        );
+        ServiceConfiguration confB = new ServiceConfiguration();
+        confB.setProperties(propertiesB);
+        providerB.initialize(confB);
+
+        this.authProvider = new AuthenticationProviderList(Lists.newArrayList(
+            providerA, providerB
+        ));
+
+        // generate tokens
+        PrivateKey privateKeyA = AuthTokenUtils.decodePrivateKey(
+            Decoders.BASE64.decode(AuthTokenUtils.encodeKeyBase64(keyPairA.getPrivate())),
+            SignatureAlgorithm.ES256
+        );
+        this.tokenAA = AuthTokenUtils.createToken(privateKeyA, SUBJECT_A, Optional.empty());
+        this.tokenAB = AuthTokenUtils.createToken(privateKeyA, SUBJECT_B, Optional.empty());
+        this.expiringTokenAA = AuthTokenUtils.createToken(privateKeyA, SUBJECT_A,
+            Optional.of(new Date(System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(3))));
+        this.expiringTokenAB = AuthTokenUtils.createToken(privateKeyA, SUBJECT_B,
+            Optional.of(new Date(System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(3))));
+
+        PrivateKey privateKeyB = AuthTokenUtils.decodePrivateKey(
+            Decoders.BASE64.decode(AuthTokenUtils.encodeKeyBase64(keyPairB.getPrivate())),
+            SignatureAlgorithm.RS512
+        );
+        this.tokenBA = AuthTokenUtils.createToken(privateKeyB, SUBJECT_A, Optional.empty());
+        this.tokenBB = AuthTokenUtils.createToken(privateKeyB, SUBJECT_B, Optional.empty());
+        this.expiringTokenBA = AuthTokenUtils.createToken(privateKeyB, SUBJECT_A,
+            Optional.of(new Date(System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(3))));
+        this.expiringTokenBB = AuthTokenUtils.createToken(privateKeyB, SUBJECT_B,
+            Optional.of(new Date(System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(3))));
+    }
+
+    @AfterMethod
+    public void tearDown() throws Exception {
+        this.authProvider.close();
+    }
+
+    private void testAuthenticate(String token, String expectedSubject) throws Exception {
+        String actualSubject = authProvider.authenticate(new AuthenticationDataSource() {
+            @Override
+            public boolean hasDataFromCommand() {
+                return true;
+            }
+
+            @Override
+            public String getCommandData() {
+                return token;
+            }
+        });
+        assertEquals(actualSubject, expectedSubject);
+    }
+
+    @Test
+    public void testAuthenticate() throws Exception {
+        testAuthenticate(tokenAA, SUBJECT_A);
+        testAuthenticate(tokenAB, SUBJECT_B);
+        testAuthenticate(tokenBA, SUBJECT_A);
+        testAuthenticate(tokenBB, SUBJECT_B);
+    }
+
+    private AuthenticationState newAuthState(String token, String expectedSubject) throws Exception {
+        AuthenticationState authState = authProvider.newAuthState(
+            AuthData.of(token.getBytes(UTF_8)),
+            null,
+            null
+        );
+        assertEquals(authState.getAuthRole(), expectedSubject);
+        assertTrue(authState.isComplete());
+        assertFalse(authState.isExpired());
+        return authState;
+    }
+
+    private void verifyAuthStateExpired(AuthenticationState authState, String expectedSubject)
+        throws Exception {
+        assertEquals(authState.getAuthRole(), expectedSubject);
+        assertTrue(authState.isComplete());
+        assertTrue(authState.isExpired());
+    }
+
+    @Test
+    public void testNewAuthState() throws Exception {
+        AuthenticationState authStateAA = newAuthState(expiringTokenAA, SUBJECT_A);
+        AuthenticationState authStateAB = newAuthState(expiringTokenAB, SUBJECT_B);
+        AuthenticationState authStateBA = newAuthState(expiringTokenBA, SUBJECT_A);
+        AuthenticationState authStateBB = newAuthState(expiringTokenBB, SUBJECT_B);
+
+        Thread.sleep(TimeUnit.SECONDS.toMillis(6));
+
+        verifyAuthStateExpired(authStateAA, SUBJECT_A);
+        verifyAuthStateExpired(authStateAB, SUBJECT_B);
+        verifyAuthStateExpired(authStateBA, SUBJECT_A);
+        verifyAuthStateExpired(authStateBB, SUBJECT_B);
+
+    }
+
+}