You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2015/12/01 17:43:47 UTC

[09/51] [abbrv] nifi git commit: NIFI-655. - Added issuer field to LoginAuthenticationToken. - Updated AccessResource to pass identity provider class name when creating LoginAuthenticationTokens. - Began refactoring JWT logic from request parsing logic i

NIFI-655. - Added issuer field to LoginAuthenticationToken. - Updated AccessResource to pass identity provider class name when creating LoginAuthenticationTokens. - Began refactoring JWT logic from request parsing logic in JwtService. - Added unit tests for JWT logic.

Signed-off-by: Matt Gilman <ma...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/3bc11e13
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/3bc11e13
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/3bc11e13

Branch: refs/heads/master
Commit: 3bc11e13d7f7dd4242881cde03c9b2ff6fc7da02
Parents: caeede5
Author: Andy LoPresto <an...@andylopresto.com>
Authored: Tue Nov 17 18:22:39 2015 -0800
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Nov 18 08:31:23 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/web/api/AccessResource.java |   6 +-
 .../nifi/web/security/jwt/JwtService.java       | 100 ++++++--
 .../token/LoginAuthenticationToken.java         |  64 ++++-
 .../nifi/web/security/jwt/JwtServiceTest.java   | 254 +++++++++++++++++++
 4 files changed, 395 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/3bc11e13/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
index 8f4cd5a..b4778ad 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
@@ -191,7 +191,7 @@ public class AccessResource extends ApplicationResource {
                     final String principal = jwtService.getAuthentication(httpServletRequest);
 
                     // TODO - catch jwt exception?
-                    // ensure we have something we can work with (certificate or crendentials)
+                    // ensure we have something we can work with (certificate or credentials)
                     if (principal == null) {
                         throw new IllegalArgumentException("The specific token is not valid.");
                     } else {
@@ -334,7 +334,7 @@ public class AccessResource extends ApplicationResource {
                 }
                 
                 // create the authentication token
-                loginAuthenticationToken = new LoginAuthenticationToken(authenticationResponse.getIdentity(), expiration);
+                loginAuthenticationToken = new LoginAuthenticationToken(authenticationResponse.getIdentity(), expiration, loginIdentityProvider.getClass().getSimpleName());
             } catch (final InvalidLoginCredentialsException ilce) {
                 throw new IllegalArgumentException("The supplied username and password are not valid.", ilce);
             } catch (final IdentityAccessException iae) {
@@ -355,7 +355,7 @@ public class AccessResource extends ApplicationResource {
             authorizeProxyIfNecessary(proxyChain);
 
             // create the authentication token
-            loginAuthenticationToken = new LoginAuthenticationToken(proxyChain.get(0), authenticationResponse.getExpiration());
+            loginAuthenticationToken = new LoginAuthenticationToken(proxyChain.get(0), authenticationResponse.getExpiration(), certificateIdentityProvider.getClass().getSimpleName());
         }
 
         // generate JWT for response

http://git-wip-us.apache.org/repos/asf/nifi/blob/3bc11e13/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
index 25e52d6..acbbcfe 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/jwt/JwtService.java
@@ -20,25 +20,30 @@ import io.jsonwebtoken.Claims;
 import io.jsonwebtoken.ExpiredJwtException;
 import io.jsonwebtoken.Jws;
 import io.jsonwebtoken.JwsHeader;
+import io.jsonwebtoken.JwtException;
 import io.jsonwebtoken.Jwts;
 import io.jsonwebtoken.MalformedJwtException;
 import io.jsonwebtoken.SignatureAlgorithm;
 import io.jsonwebtoken.SignatureException;
 import io.jsonwebtoken.SigningKeyResolverAdapter;
 import io.jsonwebtoken.UnsupportedJwtException;
-import java.nio.charset.StandardCharsets;
-import java.util.Calendar;
-import javax.servlet.http.HttpServletRequest;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.AdministrationException;
 import org.apache.nifi.admin.service.KeyService;
 import org.apache.nifi.web.security.token.LoginAuthenticationToken;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import java.nio.charset.StandardCharsets;
+import java.util.Calendar;
 
 /**
  *
  */
 public class JwtService {
+    private static final org.slf4j.Logger logger = LoggerFactory.getLogger(JwtService.class);
 
+    private static final SignatureAlgorithm SIGNATURE_ALGORITHM = SignatureAlgorithm.HS256;
     private final static String AUTHORIZATION = "Authorization";
 
     private final KeyService keyService;
@@ -54,51 +59,104 @@ public class JwtService {
      * @return The user identifier from the token
      */
     public String getAuthentication(final HttpServletRequest request) {
+        // TODO: Refactor request token extraction out of this service
         // extract/verify token from incoming request
         final String authorization = request.getHeader(AUTHORIZATION);
-        final String token = StringUtils.substringAfterLast(authorization, " ");
+        final String base64EncodedToken = StringUtils.substringAfterLast(authorization, " ");
+
+        return getAuthenticationFromToken(base64EncodedToken);
+    }
+
+    public String getAuthenticationFromToken(final String base64EncodedToken) {
+        // The library representations of the JWT should be kept internal to this service.
+        try {
+            final Jws<Claims> jws = parseTokenFromBase64EncodedString(base64EncodedToken);
+            return jws.getBody().getSubject();
+        } catch (JwtException e) {
+            logger.debug("The Base64 encoded JWT: " + base64EncodedToken);
+            final String errorMessage = "There was an error parsing the Base64-encoded JWT";
+            logger.error(errorMessage, e);
+            throw new JwtException(errorMessage, e);
+        }
+    }
 
+    private Jws<Claims> parseTokenFromBase64EncodedString(final String base64EncodedToken) throws JwtException {
         try {
-            final Jws<Claims> jwt = Jwts.parser().setSigningKeyResolver(new SigningKeyResolverAdapter() {
+            // TODO: Check algorithm for validity
+            // TODO: Ensure signature verification occurs
+            return Jwts.parser().setSigningKeyResolver(new SigningKeyResolverAdapter() {
                 @Override
                 public byte[] resolveSigningKeyBytes(JwsHeader header, Claims claims) {
                     final String identity = claims.getSubject();
+
+                    // The key is unique per identity and should be retrieved from the key service
                     final String key = keyService.getKey(identity);
 
-                    // ensure we were able to find a key that was previously issued by this key service for this user
+                    // Ensure we were able to find a key that was previously issued by this key service for this user
                     if (key == null) {
                         throw new UnsupportedJwtException("Unable to determine signing key for " + identity);
                     }
 
                     return key.getBytes(StandardCharsets.UTF_8);
                 }
-            }).parseClaimsJws(token);
-            return jwt.getBody().getSubject();
+            }).parseClaimsJws(base64EncodedToken);
         } catch (final MalformedJwtException | UnsupportedJwtException | SignatureException | ExpiredJwtException | IllegalArgumentException | AdministrationException e) {
-            return null;
+            // TODO: Exercise all exceptions to ensure none leak key material to logs
+            final String errorMessage = "There was an error parsing the Base64-encoded JWT";
+            logger.error(errorMessage, e);
+            throw new JwtException(errorMessage, e);
         }
     }
 
     /**
      * Generates a signed JWT token from the provided (Spring Security) login authentication token.
      *
-     * @param authenticationToken the authentication token
+     * @param authenticationToken
      * @return a signed JWT containing the user identity and the identity provider, Base64-encoded
+     * @throws JwtException
      */
-    public String generateSignedToken(final LoginAuthenticationToken authenticationToken) {
-        // set expiration to one day from now
-        final Calendar calendar = Calendar.getInstance();
-        calendar.setTimeInMillis(calendar.getTimeInMillis() + authenticationToken.getExpiration());
+    public String generateSignedToken(final LoginAuthenticationToken authenticationToken) throws JwtException {
+        if (authenticationToken == null) {
+            throw new IllegalArgumentException("Cannot generate a JWT for a null authentication token");
+        }
+
+        // Set expiration from the token
+        final Calendar expiration = Calendar.getInstance();
+        expiration.setTimeInMillis(authenticationToken.getExpiration());
 
-        // create a token the specified authentication
-        final String identity = authenticationToken.getPrincipal().toString();
+        final Object principal = authenticationToken.getPrincipal();
+        if (principal == null || StringUtils.isEmpty(principal.toString())) {
+            final String errorMessage = "Cannot generate a JWT for a token with an empty identity issued by " + authenticationToken.getIssuer();
+            logger.error(errorMessage);
+            throw new JwtException(errorMessage);
+        }
+
+        // Create a JWT with the specified authentication
+        final String identity = principal.toString();
         final String username = authenticationToken.getName();
 
-        // get/create the key for this user
-        final String key = keyService.getOrCreateKey(identity);
-        final byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+        try {
+            // Get/create the key for this user
+            final String key = keyService.getOrCreateKey(identity);
+            final byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+
+            logger.trace("Generating JWT for " + authenticationToken);
 
-        // build the token
-        return Jwts.builder().setSubject(identity).claim("preferred_username", username).setExpiration(calendar.getTime()).signWith(SignatureAlgorithm.HS512, keyBytes).compact();
+            // TODO: Implement "jti" claim with nonce to prevent replay attacks and allow blacklisting of revoked tokens
+
+            // Build the token
+            return Jwts.builder().setSubject(identity)
+                    .setIssuer(authenticationToken.getIssuer())
+                    .setAudience(authenticationToken.getIssuer())
+                    .claim("preferred_username", username)
+                    .setExpiration(expiration.getTime())
+                    .setIssuedAt(Calendar.getInstance().getTime())
+                    .signWith(SIGNATURE_ALGORITHM, keyBytes).compact();
+        } catch (NullPointerException | AdministrationException e) {
+            // TODO: Remove exception handling and pass through
+            final String errorMessage = "Could not retrieve the signing key for JWT";
+            logger.error(errorMessage, e);
+            throw new JwtException(errorMessage, e);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/3bc11e13/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java
index 0bb0932..a06ee89 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/token/LoginAuthenticationToken.java
@@ -19,25 +19,46 @@ package org.apache.nifi.web.security.token;
 import org.apache.nifi.security.util.CertificateUtils;
 import org.springframework.security.authentication.AbstractAuthenticationToken;
 
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+
 /**
- * This is an Authentication Token for logging in. Once a user is authenticated, they can be issues an ID token.
+ * This is an Authentication Token for logging in. Once a user is authenticated, they can be issued an ID token.
  */
 public class LoginAuthenticationToken extends AbstractAuthenticationToken {
 
     private final String identity;
     private final String username;
     private final long expiration;
+    private final String issuer;
 
-    public LoginAuthenticationToken(final String identity, final long expiration) {
-        this(identity, null, expiration);
+    /**
+     * Creates a representation of the authentication token for a user.
+     *
+     * @param identity   The unique identifier for this user
+     * @param expiration The relative time to expiration in milliseconds
+     * @param issuer     The IdentityProvider implementation that generated this token
+     */
+    public LoginAuthenticationToken(final String identity, final long expiration, final String issuer) {
+        this(identity, null, expiration, issuer);
     }
 
-    public LoginAuthenticationToken(final String identity, final String username, final long expiration) {
+    /**
+     * Creates a representation of the authentication token for a user.
+     *
+     * @param identity   The unique identifier for this user (cannot be null or empty)
+     * @param username   The preferred username for this user
+     * @param expiration The relative time to expiration in milliseconds
+     * @param issuer     The IdentityProvider implementation that generated this token
+     */
+    public LoginAuthenticationToken(final String identity, final String username, final long expiration, final String issuer) {
         super(null);
         setAuthenticated(true);
         this.identity = identity;
         this.username = username;
-        this.expiration = expiration;
+        this.issuer = issuer;
+        Calendar now = Calendar.getInstance();
+        this.expiration = now.getTimeInMillis() + expiration;
     }
 
     @Override
@@ -50,10 +71,19 @@ public class LoginAuthenticationToken extends AbstractAuthenticationToken {
         return identity;
     }
 
+    /**
+     * Returns the expiration instant in milliseconds. This value is an absolute point in time (i.e. Nov 16, 2015 11:30:00.000 GMT), not a relative time (i.e. 60 minutes). It is calculated by adding the relative expiration from the constructor to the timestamp at object creation.
+     *
+     * @return the expiration in millis
+     */
     public long getExpiration() {
         return expiration;
     }
 
+    public String getIssuer() {
+        return issuer;
+    }
+
     @Override
     public String getName() {
         if (username == null) {
@@ -64,4 +94,28 @@ public class LoginAuthenticationToken extends AbstractAuthenticationToken {
         }
     }
 
+    @Override
+    public String toString() {
+        Calendar expirationTime = Calendar.getInstance();
+        expirationTime.setTimeInMillis(getExpiration());
+        long remainingTime = expirationTime.getTimeInMillis() - Calendar.getInstance().getTimeInMillis();
+
+        SimpleDateFormat dateFormat = new SimpleDateFormat("dd-MM-yyyy HH:mm:ss.SSS");
+        dateFormat.setTimeZone(expirationTime.getTimeZone());
+        String expirationTimeString = dateFormat.format(expirationTime.getTime());
+
+        return new StringBuilder("LoginAuthenticationToken for ")
+                .append(getName())
+                .append(" issued by ")
+                .append(getIssuer())
+                .append(" expiring at ")
+                .append(expirationTimeString)
+                .append(" [")
+                .append(getExpiration())
+                .append(" ms, ")
+                .append(remainingTime)
+                .append(" ms remaining]")
+                .toString();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/3bc11e13/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
new file mode 100644
index 0000000..a7e763e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
@@ -0,0 +1,254 @@
+package org.apache.nifi.web.security.jwt;
+
+import io.jsonwebtoken.JwtException;
+import org.apache.commons.codec.CharEncoding;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.nifi.admin.service.AdministrationException;
+import org.apache.nifi.admin.service.KeyService;
+import org.apache.nifi.web.security.token.LoginAuthenticationToken;
+import org.codehaus.jettison.json.JSONObject;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+import java.io.UnsupportedEncodingException;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.when;
+
+/**
+ * Created by alopresto on 11/11/15.
+ */
+public class JwtServiceTest {
+
+    private static final Logger logger = LoggerFactory.getLogger(JwtServiceTest.class);
+
+    /**
+     * These constant strings were generated using the tool at http://jwt.io
+     */
+
+    private static final String VALID_SIGNED_TOKEN = "";
+    private static final String INVALID_SIGNED_TOKEN = "";
+    private static final String VALID_UNSIGNED_TOKEN = "";
+    private static final String INVALID_UNSIGNED_TOKEN = "";
+    private static final String VALID_MALSIGNED_TOKEN = "";
+    private static final String INVALID_MALSIGNED_TOKEN = "";
+
+    private static final String DEFAULT_HEADER = "{\"alg\":\"HS256\"}";
+
+    private static final String TOKEN_DELIMITER = ".";
+
+    private static final String HMAC_SECRET = "test_hmac_shared_secret";
+
+    private KeyService mockKeyService;
+
+    // Class under test
+    private JwtService jwtService;
+
+
+    private String generateHS256Token(String rawHeader, String rawPayload, boolean isValid, boolean isSigned) {
+        return generateHS256Token(rawHeader, rawPayload, HMAC_SECRET, isValid, isSigned);
+    }
+
+    private String generateHS256Token(String rawHeader, String rawPayload, String hmacSecret, boolean isValid, boolean isSigned) {
+        try {
+            logger.info("Generating token for " + rawHeader + " + " + rawPayload);
+
+            String base64Header = Base64.encodeBase64URLSafeString(rawHeader.getBytes(CharEncoding.UTF_8));
+            String base64Payload = Base64.encodeBase64URLSafeString(rawPayload.getBytes(CharEncoding.UTF_8));
+            // TODO: Support valid/invalid manipulation
+
+            final String body = base64Header + TOKEN_DELIMITER + base64Payload;
+
+            String signature = generateHMAC(hmacSecret, body);
+
+            return body + TOKEN_DELIMITER + signature;
+        } catch (NoSuchAlgorithmException | InvalidKeyException | UnsupportedEncodingException e) {
+            final String errorMessage = "Could not generate the token";
+            logger.error(errorMessage, e);
+            fail(errorMessage);
+            return null;
+        }
+    }
+
+    private String generateHMAC(String hmacSecret, String body) throws NoSuchAlgorithmException, UnsupportedEncodingException, InvalidKeyException {
+        Mac hmacSHA256 = Mac.getInstance("HmacSHA256");
+        SecretKeySpec secret_key = new SecretKeySpec(hmacSecret.getBytes("UTF-8"), "HmacSHA256");
+        hmacSHA256.init(secret_key);
+        return Base64.encodeBase64URLSafeString(hmacSHA256.doFinal(body.getBytes("UTF-8")));
+    }
+
+
+    @Before
+    public void setUp() throws Exception {
+        mockKeyService = Mockito.mock(KeyService.class);
+        when(mockKeyService.getOrCreateKey(anyString())).thenReturn(HMAC_SECRET);
+        jwtService = new JwtService(mockKeyService);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+
+    }
+
+    @Test
+    public void testShouldGetAuthenticationForValidToken() throws Exception {
+
+    }
+
+    @Test
+    public void testShouldNotGetAuthenticationForInvalidToken() throws Exception {
+        // Arrange
+        String token = INVALID_SIGNED_TOKEN;
+
+        String header = "{" +
+                "  \"alg\":\"HS256\"" +
+                "}";
+        String payload = "{" +
+                "  \"sub\":\"alopresto\"," +
+                "  \"preferred_username\":\"alopresto\"," +
+                "  \"exp\":2895419760" +
+                "}";
+
+        // Act
+        logger.info("Test token: " + generateHS256Token(header, payload, true, true));
+
+
+        // Assert
+
+
+    }
+
+    @Test
+    public void testShouldNotGetAuthenticationForEmptyToken() throws Exception {
+
+    }
+
+    @Test
+    public void testShouldNotGetAuthenticationForUnsignedToken() throws Exception {
+
+    }
+
+    @Test
+    public void testShouldNotGetAuthenticationForTokenWithWrongAlgorithm() throws Exception {
+
+    }
+
+    @Test
+    public void testShouldGenerateSignedToken() throws Exception {
+        // Arrange
+
+        // Token expires in 60 seconds
+        final int EXPIRATION_MILLIS = 60000;
+        LoginAuthenticationToken loginAuthenticationToken = new LoginAuthenticationToken("alopresto", EXPIRATION_MILLIS, "MockIdentityProvider");
+        logger.debug("Generating token for " + loginAuthenticationToken);
+
+        final String EXPECTED_HEADER = DEFAULT_HEADER;
+
+        // Convert the expiration time from ms to s
+        final long TOKEN_EXPIRATION_SEC = (long) (loginAuthenticationToken.getExpiration() / 1000.0);
+
+        // Act
+        String token = jwtService.generateSignedToken(loginAuthenticationToken);
+        logger.debug("Generated JWT: " + token);
+
+        // Run after the SUT generates the token to ensure the same issued at time
+
+        // Split the token, decode the middle section, and form a new String
+        final String DECODED_PAYLOAD = new String(Base64.decodeBase64(token.split("\\.")[1].getBytes()));
+        final long ISSUED_AT_SEC = Long.valueOf(DECODED_PAYLOAD.substring(DECODED_PAYLOAD.lastIndexOf(":") + 1, DECODED_PAYLOAD.length() - 1));
+        logger.trace("Actual token was issued at " + ISSUED_AT_SEC);
+
+        // Always use LinkedHashMap to enforce order of the keys because the signature depends on order
+        Map<String, Object> claims = new LinkedHashMap<>();
+        claims.put("sub", "alopresto");
+        claims.put("iss", "MockIdentityProvider");
+        claims.put("aud", "MockIdentityProvider");
+        claims.put("preferred_username", "alopresto");
+        claims.put("exp", TOKEN_EXPIRATION_SEC);
+        claims.put("iat", ISSUED_AT_SEC);
+        logger.trace("JSON Object to String: " + new JSONObject(claims).toString());
+
+        final String EXPECTED_PAYLOAD = new JSONObject(claims).toString();
+        final String EXPECTED_TOKEN_STRING = generateHS256Token(EXPECTED_HEADER, EXPECTED_PAYLOAD, true, true);
+        logger.debug("Expected JWT: " + EXPECTED_TOKEN_STRING);
+
+        // Assert
+        assertEquals("JWT token", EXPECTED_TOKEN_STRING, token);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testShouldNotGenerateTokenWithNullAuthenticationToken() throws Exception {
+        // Arrange
+        LoginAuthenticationToken nullLoginAuthenticationToken = null;
+        logger.debug("Generating token for " + nullLoginAuthenticationToken);
+
+        // Act
+        jwtService.generateSignedToken(nullLoginAuthenticationToken);
+
+        // Assert
+
+        // Should throw exception
+    }
+
+    @Test(expected = JwtException.class)
+    public void testShouldNotGenerateTokenWithEmptyIdentity() throws Exception {
+        // Arrange
+        final int EXPIRATION_MILLIS = 60000;
+        LoginAuthenticationToken emptyIdentityLoginAuthenticationToken = new LoginAuthenticationToken("", EXPIRATION_MILLIS, "MockIdentityProvider");
+        logger.debug("Generating token for " + emptyIdentityLoginAuthenticationToken);
+
+        // Act
+        jwtService.generateSignedToken(emptyIdentityLoginAuthenticationToken);
+
+        // Assert
+
+        // Should throw exception
+    }
+
+    @Test(expected = JwtException.class)
+    public void testShouldNotGenerateTokenWithNullIdentity() throws Exception {
+        // Arrange
+        final int EXPIRATION_MILLIS = 60000;
+        LoginAuthenticationToken nullIdentityLoginAuthenticationToken = new LoginAuthenticationToken(null, EXPIRATION_MILLIS, "MockIdentityProvider");
+        logger.debug("Generating token for " + nullIdentityLoginAuthenticationToken);
+
+        // Act
+        jwtService.generateSignedToken(nullIdentityLoginAuthenticationToken);
+
+        // Assert
+
+        // Should throw exception
+    }
+
+    @Test(expected = JwtException.class)
+    public void testShouldNotGenerateTokenWithMissingKey() throws Exception {
+        // Arrange
+        final int EXPIRATION_MILLIS = 60000;
+        LoginAuthenticationToken loginAuthenticationToken = new LoginAuthenticationToken("alopresto", EXPIRATION_MILLIS, "MockIdentityProvider");
+        logger.debug("Generating token for " + loginAuthenticationToken);
+
+        // Set up the bad key service
+        KeyService missingKeyService = Mockito.mock(KeyService.class);
+        when(missingKeyService.getOrCreateKey(anyString())).thenThrow(new AdministrationException("Could not find a key for that user"));
+        jwtService = new JwtService(missingKeyService);
+
+        // Act
+        jwtService.generateSignedToken(loginAuthenticationToken);
+
+        // Assert
+
+        // Should throw exception
+    }
+}
\ No newline at end of file