You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by dh...@apache.org on 2020/04/24 18:41:45 UTC

[geode] branch develop updated: GEODE-7851: Pulse refreshes expired access tokens (#4977)

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

dhemery pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 2999414  GEODE-7851: Pulse refreshes expired access tokens (#4977)
2999414 is described below

commit 2999414d6004b7725fd9652f75dbfdb549a2544d
Author: Dale Emery <de...@pivotal.io>
AuthorDate: Fri Apr 24 11:41:10 2020 -0700

    GEODE-7851: Pulse refreshes expired access tokens (#4977)
    
    If a user's access token expires, Pulse attempts to refresh it. If the
    refresh fails, Pulse logs the user out and redirects the browser to
    /pulse/clusterLogout.
    
    Changes in Repository:
    - When OAuth is configured, before returning the user's cluster,
      getCluster() checks whether the user's access token has expired.
    - If the access token has expired, the repository attempts to refresh
      it.  If the refresh succeeds, the repository reconnects the user's
      cluster to JMX and returns it.
    - If the refresh fails, the repository disconnects the user's cluster
      from JMX, removes the cluster from the repository, and throws an
      authentication or authorization exception.
    
    Changes in PulseController:
    - If the service call throws an authentication or authorization
      exception, PulseController.  getPulseUpdate() returns a 401 status.
    
    Changes in pulsescript/common.js:
    - If a Pulse ajax call returns a 401 status, ajaxPost() redirects the
      browser to /pulse/clusterLogout to log the user out and request
      re-authorization.
    
    Co-authored-by: Joris Melchior <jo...@gmail.com>
    Co-authored-by: Dale Emery <de...@pivotal.io>
    Co-authored-by: Jinmei Liao <ji...@pivotal.io>
    
    Co-authored-by: Kirk Lund <kl...@apache.org>
    Co-authored-by: Joris Melchior <jo...@gmail.com>
    Co-authored-by: Jinmei Liao <ji...@pivotal.io>
---
 .../security/OAuthSecurityTokenHandoffTest.java    |  80 ++++---
 .../internal/controllers/PulseController.java      |   7 +
 .../geode/tools/pulse/internal/data/Cluster.java   |  12 +
 .../tools/pulse/internal/data/Repository.java      | 262 ++++++++++++++-------
 .../src/main/webapp/scripts/pulsescript/common.js  |   3 +
 5 files changed, 251 insertions(+), 113 deletions(-)

diff --git a/geode-pulse/src/integrationTest/java/org/apache/geode/tools/pulse/security/OAuthSecurityTokenHandoffTest.java b/geode-pulse/src/integrationTest/java/org/apache/geode/tools/pulse/security/OAuthSecurityTokenHandoffTest.java
index bf10319..6a51343 100644
--- a/geode-pulse/src/integrationTest/java/org/apache/geode/tools/pulse/security/OAuthSecurityTokenHandoffTest.java
+++ b/geode-pulse/src/integrationTest/java/org/apache/geode/tools/pulse/security/OAuthSecurityTokenHandoffTest.java
@@ -29,7 +29,6 @@ import static org.springframework.test.web.servlet.request.MockMvcRequestBuilder
 import java.time.Duration;
 import java.time.Instant;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -45,8 +44,9 @@ import org.springframework.security.oauth2.client.OAuth2AuthorizedClientService;
 import org.springframework.security.oauth2.client.authentication.OAuth2AuthenticationToken;
 import org.springframework.security.oauth2.client.registration.ClientRegistration;
 import org.springframework.security.oauth2.core.OAuth2AccessToken;
-import org.springframework.security.oauth2.core.user.DefaultOAuth2User;
-import org.springframework.security.oauth2.core.user.OAuth2User;
+import org.springframework.security.oauth2.core.oidc.OidcIdToken;
+import org.springframework.security.oauth2.core.oidc.user.DefaultOidcUser;
+import org.springframework.security.oauth2.core.oidc.user.OidcUser;
 import org.springframework.security.oauth2.core.user.OAuth2UserAuthority;
 import org.springframework.test.context.ActiveProfiles;
 import org.springframework.test.context.ContextConfiguration;
@@ -85,53 +85,67 @@ public class OAuthSecurityTokenHandoffTest {
   }
 
   @Test
-  public void usesCurrentSessionAccessTokenAsCredentialToConnectToGemFire() throws Exception {
-    String userName = "some-user-name";
-    String accessTokenValue = "the-access-token-value";
-    String urlThatTriggersPulseToConnectToGemFire = "/dataBrowserRegions";
+  public void usesCurrentSessionAccessTokenValueAsCredentialToConnectToGemFire() throws Exception {
+    String subject = "some-subject";
 
     Cluster clusterForUser = mock(Cluster.class);
-    when(clusterFactory.create(any(), any(), eq(userName), any(), any()))
-        .thenReturn(clusterForUser);
+    when(clusterFactory.create(any(), any(), eq(subject), any(), any())).thenReturn(clusterForUser);
 
-    MockHttpSession session = sessionWithAuthenticatedUser(userName, accessTokenValue);
+    String tokenValue = "the-token-value";
+    MockHttpSession session = sessionWithAuthenticatedUser("some-user-name", subject, tokenValue);
 
+    String urlThatTriggersPulseToConnectToGemFire = "/dataBrowserRegions";
     mvc.perform(get(urlThatTriggersPulseToConnectToGemFire).session(session));
 
-    verify(clusterForUser).connectToGemFire(accessTokenValue);
+    verify(clusterForUser).connectToGemFire(tokenValue);
+  }
+
+  private MockHttpSession sessionWithAuthenticatedUser(String userName, String subject,
+      String tokenValue) {
+    OAuth2AccessToken accessToken = accessToken(tokenValue);
+    OidcIdToken idToken = idToken(userName, subject, accessToken);
+    OAuth2AuthenticationToken authenticationToken = authenticationToken(idToken);
+    authorizeClient(authenticationToken, accessToken);
+    return sessionWithAuthenticationToken(authenticationToken);
   }
 
-  private void authorizeClient(
-      OAuth2AuthenticationToken authenticationToken, OAuth2AccessToken accessToken) {
+  private static OAuth2AuthenticationToken authenticationToken(OidcIdToken idToken) {
+    List<GrantedAuthority> userAuthorities = allGeodeAuthorities(idToken.getClaims());
+    OidcUser user = new DefaultOidcUser(userAuthorities, idToken);
+    return new OAuth2AuthenticationToken(user, userAuthorities, AUTHENTICATION_PROVIDER_ID);
+  }
+
+  private void authorizeClient(OAuth2AuthenticationToken authenticationToken,
+      OAuth2AccessToken accessToken) {
+    String userName = authenticationToken.getPrincipal().getName();
     OAuth2AuthorizedClient authorizedClient =
-        new OAuth2AuthorizedClient(clientRegistration(),
-            authenticationToken.getPrincipal().getName(), accessToken);
+        new OAuth2AuthorizedClient(clientRegistration(), userName, accessToken);
     authorizedClientService.saveAuthorizedClient(authorizedClient, authenticationToken);
   }
 
-  private MockHttpSession sessionWithAuthenticatedUser(String username, String tokenValue) {
-    OAuth2AuthenticationToken authenticationToken = authenticationToken(username);
-    authorizeClient(authenticationToken, accessToken(tokenValue));
-    return sessionWithAuthenticationToken(authenticationToken);
+  private static OidcIdToken idToken(String userName, String subject,
+      OAuth2AccessToken accessToken) {
+    return OidcIdToken.withTokenValue(accessToken.getTokenValue())
+        .subject(subject)
+        .claim("user_name", userName)
+        .issuedAt(accessToken.getIssuedAt())
+        .expiresAt(accessToken.getExpiresAt())
+        .build();
   }
 
   private static OAuth2AccessToken accessToken(String tokenValue) {
-    return new OAuth2AccessToken(BEARER, tokenValue, Instant.now(),
-        Instant.now().plus(Duration.ofHours(1)));
+    Instant issuedAt = Instant.now();
+    Instant expiresAt = issuedAt.plus(Duration.ofDays(12));
+    return new OAuth2AccessToken(BEARER, tokenValue, issuedAt, expiresAt);
   }
 
-  private static OAuth2AuthenticationToken authenticationToken(String userName) {
-    Map<String, Object> attributes = new HashMap<>();
-    attributes.put("sub", userName);
-
-    List<GrantedAuthority> authorities = Arrays.asList(
-        new OAuth2UserAuthority("ROLE_USER", attributes),
-        new OAuth2UserAuthority("SCOPE_CLUSTER:READ", attributes),
-        new OAuth2UserAuthority("SCOPE_CLUSTER:WRITE", attributes),
-        new OAuth2UserAuthority("SCOPE_DATA:READ", attributes),
-        new OAuth2UserAuthority("SCOPE_DATA:WRITE", attributes));
-    OAuth2User user = new DefaultOAuth2User(authorities, attributes, "sub");
-    return new OAuth2AuthenticationToken(user, authorities, AUTHENTICATION_PROVIDER_ID);
+  private static List<GrantedAuthority> allGeodeAuthorities(Map<String, Object> userAttributes) {
+    return Arrays.asList(
+        new OAuth2UserAuthority("ROLE_USER", userAttributes),
+        new OAuth2UserAuthority("SCOPE_CLUSTER:READ", userAttributes),
+        new OAuth2UserAuthority("SCOPE_CLUSTER:WRITE", userAttributes),
+        new OAuth2UserAuthority("SCOPE_DATA:READ", userAttributes),
+        new OAuth2UserAuthority("SCOPE_DATA:WRITE", userAttributes));
   }
 
   private static ClientRegistration clientRegistration() {
diff --git a/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/controllers/PulseController.java b/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/controllers/PulseController.java
index 709ac08..b478a91 100644
--- a/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/controllers/PulseController.java
+++ b/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/controllers/PulseController.java
@@ -34,6 +34,9 @@ import org.apache.commons.text.StringEscapeUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.http.HttpStatus;
+import org.springframework.security.oauth2.core.OAuth2AuthenticationException;
+import org.springframework.security.oauth2.core.OAuth2AuthorizationException;
 import org.springframework.stereotype.Controller;
 import org.springframework.web.bind.annotation.RequestMapping;
 import org.springframework.web.bind.annotation.RequestMethod;
@@ -106,6 +109,10 @@ public class PulseController {
         try {
           PulseService pulseService = pulseServiceFactory.getPulseServiceInstance(serviceName);
           responseMap.set(serviceName, pulseService.execute(request));
+        } catch (OAuth2AuthenticationException | OAuth2AuthorizationException e) {
+          logger.warn("serviceException [for service {}] = {}", serviceName, e.getMessage());
+          response.setStatus(HttpStatus.UNAUTHORIZED.value());
+          return;
         } catch (Exception serviceException) {
           logger.warn("serviceException [for service {}] = {}", serviceName,
               serviceException.getMessage());
diff --git a/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Cluster.java b/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Cluster.java
index dbc3b56..48134f3 100644
--- a/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Cluster.java
+++ b/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Cluster.java
@@ -17,6 +17,7 @@
 
 package org.apache.geode.tools.pulse.internal.data;
 
+import java.io.IOException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -2759,6 +2760,17 @@ public class Cluster extends Thread {
     return getDataBrowser().deleteQueryById(userId, queryId);
   }
 
+  public void reconnectToGemFire(Object credentials) {
+    if (jmxConnector != null) {
+      try {
+        jmxConnector.close();
+      } catch (IOException e) {
+        logger.info("Could not close old connection on reconnect attempt", e);
+      }
+      jmxConnector = updater.connect(credentials);
+    }
+  }
+
   public void connectToGemFire(Object credentials) {
     jmxConnector = updater.connect(credentials);
 
diff --git a/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Repository.java b/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Repository.java
index 9e320bc..585cd06 100644
--- a/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Repository.java
+++ b/geode-pulse/src/main/java/org/apache/geode/tools/pulse/internal/data/Repository.java
@@ -17,13 +17,15 @@
 
 package org.apache.geode.tools.pulse.internal.data;
 
+import static java.time.Instant.now;
+
+import java.time.Instant;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
 import java.util.ResourceBundle;
+import java.util.Set;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -33,8 +35,15 @@ import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.security.oauth2.client.OAuth2AuthorizedClient;
 import org.springframework.security.oauth2.client.OAuth2AuthorizedClientService;
 import org.springframework.security.oauth2.client.authentication.OAuth2AuthenticationToken;
-import org.springframework.security.oauth2.core.OAuth2AccessToken;
-import org.springframework.security.oauth2.core.user.OAuth2User;
+import org.springframework.security.oauth2.client.endpoint.DefaultRefreshTokenTokenResponseClient;
+import org.springframework.security.oauth2.client.endpoint.OAuth2RefreshTokenGrantRequest;
+import org.springframework.security.oauth2.core.AbstractOAuth2Token;
+import org.springframework.security.oauth2.core.OAuth2AuthenticationException;
+import org.springframework.security.oauth2.core.OAuth2AuthorizationException;
+import org.springframework.security.oauth2.core.OAuth2Error;
+import org.springframework.security.oauth2.core.OAuth2RefreshToken;
+import org.springframework.security.oauth2.core.endpoint.OAuth2AccessTokenResponse;
+import org.springframework.security.oauth2.core.oidc.user.OidcUser;
 import org.springframework.stereotype.Component;
 
 /**
@@ -45,6 +54,8 @@ import org.springframework.stereotype.Component;
 @Component
 public class Repository {
   private static final Logger logger = LogManager.getLogger();
+  private static final Locale LOCALE =
+      new Locale(PulseConstants.APPLICATION_LANGUAGE, PulseConstants.APPLICATION_COUNTRY);
 
   private final OAuth2AuthorizedClientService authorizedClientService;
   private final ClusterFactory clusterFactory;
@@ -56,11 +67,9 @@ public class Repository {
   private boolean useSSLManager = false;
   private Properties javaSslProperties;
 
-  Locale locale =
-      new Locale(PulseConstants.APPLICATION_LANGUAGE, PulseConstants.APPLICATION_COUNTRY);
 
   private final ResourceBundle resourceBundle =
-      ResourceBundle.getBundle(PulseConstants.LOG_MESSAGES_FILE, locale);
+      ResourceBundle.getBundle(PulseConstants.LOG_MESSAGES_FILE, LOCALE);
 
   private final PulseConfig pulseConfig = new PulseConfig();
 
@@ -80,8 +89,99 @@ public class Repository {
     this.clusterFactory = clusterFactory;
   }
 
+  /**
+   * this will return a cluster already connected to the geode jmx manager for the user in the
+   * request
+   * <p>
+   * But for multi-user connections to gemfireJMX, i.e pulse that uses gemfire integrated security,
+   * we will need to get the username from the context
+   */
+  public Cluster getCluster() {
+    Authentication authentication = SecurityContextHolder.getContext().getAuthentication();
+    if (authentication == null) {
+      return null;
+    }
+
+    if (authentication instanceof OAuth2AuthenticationToken) {
+      return getClusterWithAuthenticationToken((OAuth2AuthenticationToken) authentication);
+    }
+
+    return getClusterWithUserNameAndPassword(authentication.getName(), null);
+  }
+
+  public Cluster getClusterWithUserNameAndPassword(String userName, String password) {
+    String[] credentials = {userName, password};
+    return getClusterWithCredentials(userName, credentials);
+  }
+
+  public Cluster getClusterWithCredentials(String userName, Object credentials) {
+    synchronized (clusterMap) {
+      Cluster cluster = clusterMap.get(userName);
+      if (cluster == null) {
+        logger.info(resourceBundle.getString("LOG_MSG_CREATE_NEW_THREAD") + " : " + userName);
+        cluster = clusterFactory.create(host, port, userName, resourceBundle, this);
+        // Assign name to thread created
+        cluster.setName(PulseConstants.APP_NAME + "-" + host + ":" + port + ":" + userName);
+        cluster.connectToGemFire(credentials);
+        if (cluster.isConnectedFlag()) {
+          clusterMap.put(userName, cluster);
+        }
+      }
+      return cluster;
+    }
+  }
+
+  /**
+   * Returns the cluster for the user associated with the given authentication. If the user's
+   * access token is expired, it is refreshed and the cluster is reconnected to JMX using the fresh
+   * token. If the refresh fails, the user's cluster is disconnected from JMX and removed from the
+   * repository.
+   */
+  private Cluster getClusterWithAuthenticationToken(OAuth2AuthenticationToken authentication) {
+    OAuth2AuthorizedClient authorizedClient = getAuthorizedClient(authentication);
+    if (isExpired(authorizedClient.getAccessToken())) {
+      return reconnectedClusterForExpiredClient(authentication, authorizedClient);
+    }
+
+    // When the cluster connects to JMX, it will include this access token as the
+    // jmx.remote.credentials attribute in the connection environment. The JMX server will then pass
+    // the access token to the security manager for authentication.
+    String credentials = authorizedClient.getAccessToken().getTokenValue();
+    String subject = getSubject(authentication);
+    return getClusterWithCredentials(subject, credentials);
+  }
+
+  private static String getSubject(Authentication authentication) {
+    OAuth2AuthenticationToken oauth2Authentication = (OAuth2AuthenticationToken) authentication;
+    OidcUser oidcUser = (OidcUser) oauth2Authentication.getPrincipal();
+    return oidcUser.getIdToken().getSubject();
+  }
+
+  public void logoutUser(String userName) {
+    Cluster cluster = clusterMap.remove(userName);
+    if (cluster != null) {
+      try {
+        cluster.setStopUpdates(true);
+        cluster.getJMXConnector().close();
+      } catch (Exception e) {
+        // We're logging out so this can be ignored
+      }
+    }
+  }
+
+  public void removeAllClusters() {
+    Set<String> keySet = clusterMap.keySet();
+
+    for (String key : keySet) {
+      Cluster c = clusterMap.get(key);
+      c.stopThread();
+      clusterMap.remove(key);
+      logger.info("{} : {}", resourceBundle.getString("LOG_MSG_REMOVE_THREAD"), key);
+    }
+  }
+
   public Boolean getJmxUseLocator() {
-    return this.jmxUseLocator;
+    return jmxUseLocator;
   }
 
   public void setJmxUseLocator(Boolean jmxUseLocator) {
@@ -90,19 +190,19 @@ public class Repository {
   }
 
   public String getHost() {
-    return this.host;
+    return host;
   }
 
   public void setHost(String jmxHost) {
-    this.host = jmxHost;
+    host = jmxHost;
   }
 
   public String getPort() {
-    return this.port;
+    return port;
   }
 
   public void setPort(String jmxPort) {
-    this.port = jmxPort;
+    port = jmxPort;
   }
 
   public boolean isUseSSLLocator() {
@@ -122,7 +222,7 @@ public class Repository {
   }
 
   public PulseConfig getPulseConfig() {
-    return this.pulseConfig;
+    return pulseConfig;
   }
 
   public Properties getJavaSslProperties() {
@@ -133,86 +233,88 @@ public class Repository {
     this.javaSslProperties = javaSslProperties;
   }
 
-  /**
-   * this will return a cluster already connected to the geode jmx manager for the user in the
-   * request
-   *
-   * But for multi-user connections to gemfireJMX, i.e pulse that uses gemfire integrated security,
-   * we will need to get the username from the context
-   */
-  public Cluster getCluster() {
-    Authentication authentication = SecurityContextHolder.getContext().getAuthentication();
-    if (authentication == null) {
-      return null;
-    }
-
-    if (authentication instanceof OAuth2AuthenticationToken) {
-      OAuth2AuthenticationToken authenticationToken = (OAuth2AuthenticationToken) authentication;
-      OAuth2AuthorizedClient authorizedClient = authorizedClientService.loadAuthorizedClient(
-          authenticationToken.getAuthorizedClientRegistrationId(),
-          authenticationToken.getName());
-
-      OAuth2User authenticatedPrincipal = authenticationToken.getPrincipal();
-      String authenticatedPrincipalName = authenticatedPrincipal.getName();
-      OAuth2AccessToken accessToken = authorizedClient.getAccessToken();
-      String accessTokenValue = accessToken.getTokenValue();
-      return getClusterWithCredentials(authenticatedPrincipalName, accessTokenValue);
-    }
-
-    return getClusterWithUserNameAndPassword(authentication.getName(), null);
+  public ResourceBundle getResourceBundle() {
+    return resourceBundle;
   }
 
-  public Cluster getClusterWithUserNameAndPassword(String userName, String password) {
-    return getClusterWithCredentials(userName, new String[] {userName, password});
+  private OAuth2AuthorizedClient getAuthorizedClient(
+      OAuth2AuthenticationToken authenticationToken) {
+    return authorizedClientService.loadAuthorizedClient(
+        authenticationToken.getAuthorizedClientRegistrationId(), authenticationToken.getName());
   }
 
-  public Cluster getClusterWithCredentials(String username, Object credentials) {
-    synchronized (this.clusterMap) {
-      Cluster data = clusterMap.get(username);
-      if (data == null) {
-        logger.info(resourceBundle.getString("LOG_MSG_CREATE_NEW_THREAD") + " : " + username);
-        data = clusterFactory.create(host, port, username, resourceBundle, this);
-        // Assign name to thread created
-        data.setName(PulseConstants.APP_NAME + "-" + host + ":" + port + ":" + username);
-        data.connectToGemFire(credentials);
-        if (data.isConnectedFlag()) {
-          this.clusterMap.put(username, data);
-        }
-      }
-      return data;
-    }
+  private static boolean isExpired(AbstractOAuth2Token token) {
+    Instant tokenExpiration = token.getExpiresAt();
+    return tokenExpiration != null && tokenExpiration.isBefore(now());
   }
 
-  public void logoutUser(String username) {
-    Cluster data = clusterMap.remove(username);
-    if (data != null) {
-      try {
-        data.setStopUpdates(true);
-        data.getJMXConnector().close();
-      } catch (Exception e) {
-        // We're logging out so this can be ignored
-      }
+  private OAuth2AuthorizedClient refreshExpiredClient(Authentication authentication,
+      OAuth2AuthorizedClient expiredClient) {
+    OAuth2RefreshToken refreshToken = expiredClient.getRefreshToken();
+    String subject = getSubject(authentication);
+    if (refreshToken == null) {
+      throw new OAuth2AuthenticationException(new OAuth2Error("401"),
+          "User " + subject + " has no refresh token.");
+    }
+    if (isExpired(refreshToken)) {
+      throw new OAuth2AuthenticationException(new OAuth2Error("401"),
+          "The refresh token for " + subject + " has expired.");
     }
-  }
 
-  // This method is used to remove all cluster threads
-  public void removeAllClusters() {
+    OAuth2AccessTokenResponse freshToken = getFreshToken(expiredClient);
 
-    Iterator<Map.Entry<String, Cluster>> iter = clusterMap.entrySet().iterator();
+    OAuth2AuthorizedClient freshClient = new OAuth2AuthorizedClient(
+        expiredClient.getClientRegistration(), expiredClient.getPrincipalName(),
+        freshToken.getAccessToken(), freshToken.getRefreshToken());
 
-    while (iter.hasNext()) {
-      Map.Entry<String, Cluster> entry = iter.next();
-      Cluster c = entry.getValue();
-      String clusterKey = entry.getKey();
-      c.stopThread();
-      iter.remove();
-      logger.info("{} : {}", resourceBundle.getString("LOG_MSG_REMOVE_THREAD"), clusterKey);
-    }
+    authorizedClientService.saveAuthorizedClient(freshClient, authentication);
+
+    return freshClient;
   }
 
-  public ResourceBundle getResourceBundle() {
-    return this.resourceBundle;
+  /**
+   * Refreshes the expired client's access token, reconnects the associated user's cluster using
+   * the new token, and returns the reconnected cluster. If the access token cannot be refreshed,
+   * the user's cluster is disconnected from JMX and removed from the repository.
+   */
+  private Cluster reconnectedClusterForExpiredClient(OAuth2AuthenticationToken authentication,
+      OAuth2AuthorizedClient expiredClient) {
+    String subject = getSubject(authentication);
+
+    logger.info("Attempting to refresh the expired access token for {}.", subject);
+
+    OAuth2AuthorizedClient freshClient;
+    try {
+      freshClient = refreshExpiredClient(authentication, expiredClient);
+    } catch (OAuth2AuthenticationException | OAuth2AuthorizationException authException) {
+      String message = "Failed to refresh the access token for " + subject +
+          ". Disconnecting and removing the user's cluster.";
+      logger.info(message);
+      logoutUser(subject);
+      throw authException;
+    }
+
+    logger.info("Refreshed the access token for {}. Reconnecting the user's cluster.", subject);
+    synchronized (clusterMap) {
+      Cluster cluster = clusterMap.get(subject);
+      if (cluster != null) {
+        // When the cluster reconnects to JMX, it will include this access token as the
+        // jmx.remote.credentials attribute in the connection environment. The JMX server will then
+        // pass the access token to the security manager for authentication.
+        String credentials = freshClient.getAccessToken().getTokenValue();
+        cluster.reconnectToGemFire(credentials);
+      }
+      return cluster;
+    }
   }
 
+  private static OAuth2AccessTokenResponse getFreshToken(OAuth2AuthorizedClient expiredClient) {
+    OAuth2RefreshTokenGrantRequest refreshRequest = new OAuth2RefreshTokenGrantRequest(
+        expiredClient.getClientRegistration(),
+        expiredClient.getAccessToken(),
+        expiredClient.getRefreshToken());
 
+    return new DefaultRefreshTokenTokenResponseClient()
+        .getTokenResponse(refreshRequest);
+  }
 }
diff --git a/geode-pulse/src/main/webapp/scripts/pulsescript/common.js b/geode-pulse/src/main/webapp/scripts/pulsescript/common.js
index b993483..07fa255 100644
--- a/geode-pulse/src/main/webapp/scripts/pulsescript/common.js
+++ b/geode-pulse/src/main/webapp/scripts/pulsescript/common.js
@@ -1343,6 +1343,9 @@ function ajaxPost(pulseUrl, pulseData, pulseCallBackName) {
       console.log("The following error occurred: " + textStatus, errorThrown);
       $('#connectionStatusDiv').show();
       $('#connectionErrorMsgDiv').html("Pulse server is not connected");
+      if(errorThrown === "Unauthorized") {
+        window.location.replace("/pulse/clusterLogout");
+      }
     },
     // callback handler that will be called on completion
     // which means, either on success or error