You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2021/07/30 19:38:13 UTC

[geode] branch expireAuthentication updated: GEODE-9456, GEODE-9452: Authentication Expiration (#6721)

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

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


The following commit(s) were added to refs/heads/expireAuthentication by this push:
     new 340e8c6  GEODE-9456, GEODE-9452: Authentication Expiration (#6721)
340e8c6 is described below

commit 340e8c627ea45f06e5edaebe823b3692de8544c3
Author: Jinmei Liao <ji...@pivotal.io>
AuthorDate: Fri Jul 30 12:33:54 2021 -0700

    GEODE-9456, GEODE-9452: Authentication Expiration (#6721)
    
    * Add tests and throw AuthenticationExpiredException
    Co-authored-by: Joris Melchior <jo...@gmail.com>
---
 .../integrationTest/resources/assembly_content.txt |   5 +-
 geode-core/build.gradle                            |   2 +-
 .../SecurityWithExpirationIntegrationTest.java     |  66 ++++++++++++
 geode-core/src/integrationTest/resources/shiro.ini |  40 ++++++++
 .../cache/client/internal/OpExecutorImpl.java      |   8 +-
 .../cache/tier/sockets/ServerConnection.java       |   6 +-
 .../security/AuthenticationExpiredException.java   |  32 ++++++
 .../org/apache/geode/security/SecurityManager.java |   5 +-
 .../sanctioned-geode-core-serializables.txt        |   2 +-
 .../geode/security/AuthExpirationDUnitTest.java    | 112 +++++++++++++++++++++
 .../geode/security/ExpirableSecurityManager.java   |  56 +++++++++++
 .../security/UpdatableUserAuthInitialize.java      |  54 ++++++++++
 .../gemstone/gemfire/OldClientSupportProvider.java |  16 ++-
 13 files changed, 394 insertions(+), 10 deletions(-)

diff --git a/geode-assembly/src/integrationTest/resources/assembly_content.txt b/geode-assembly/src/integrationTest/resources/assembly_content.txt
index c408ada..39a94f8 100644
--- a/geode-assembly/src/integrationTest/resources/assembly_content.txt
+++ b/geode-assembly/src/integrationTest/resources/assembly_content.txt
@@ -929,6 +929,7 @@ javadoc/org/apache/geode/ra/package-tree.html
 javadoc/org/apache/geode/security/AccessControl.html
 javadoc/org/apache/geode/security/AuthInitialize.html
 javadoc/org/apache/geode/security/AuthTokenEnabledComponents.html
+javadoc/org/apache/geode/security/AuthenticationExpiredException.html
 javadoc/org/apache/geode/security/AuthenticationFailedException.html
 javadoc/org/apache/geode/security/AuthenticationRequiredException.html
 javadoc/org/apache/geode/security/Authenticator.html
@@ -975,11 +976,12 @@ lib/commons-lang3-3.12.0.jar
 lib/commons-logging-1.2.jar
 lib/commons-modeler-2.0.1.jar
 lib/commons-validator-1.7.jar
+lib/fastutil-8.5.4.jar
 lib/fastutil-core-8.5.4.jar
 lib/fastutil-extra-8.5.4.jar
-lib/fastutil-8.5.4.jar
 lib/findbugs-annotations-1.3.9-1.jar
 lib/geo-0.7.7.jar
+lib/geode-apis-compatible-with-redis-0.0.0.jar
 lib/geode-common-0.0.0.jar
 lib/geode-connectors-0.0.0.jar
 lib/geode-core-0.0.0.jar
@@ -997,7 +999,6 @@ lib/geode-membership-0.0.0.jar
 lib/geode-memcached-0.0.0.jar
 lib/geode-old-client-support-0.0.0.jar
 lib/geode-rebalancer-0.0.0.jar
-lib/geode-apis-compatible-with-redis-0.0.0.jar
 lib/geode-serialization-0.0.0.jar
 lib/geode-tcp-server-0.0.0.jar
 lib/geode-unsafe-0.0.0.jar
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index ecb51b7..8035775 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -387,7 +387,7 @@ dependencies {
 
     upgradeTestRuntimeOnly(project(path: ':geode-old-versions', configuration: 'classpathsOutput'))
     upgradeTestRuntimeOnly(project(':geode-log4j'))
-
+    upgradeTestRuntimeOnly(project(':geode-old-client-support'))
 
     performanceTestImplementation(project(':geode-junit')) {
         exclude module: 'geode-core'
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/security/SecurityWithExpirationIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/security/SecurityWithExpirationIntegrationTest.java
new file mode 100644
index 0000000..cb0a24f
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/security/SecurityWithExpirationIntegrationTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.geode.management.internal.security;
+
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.security.SecurityServiceFactory;
+import org.apache.geode.security.AuthenticationExpiredException;
+import org.apache.geode.security.ExpirableSecurityManager;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({SecurityTest.class})
+public class SecurityWithExpirationIntegrationTest {
+
+  protected Properties props = new Properties();
+
+  protected SecurityService securityService;
+
+  @Before
+  public void before() throws Exception {
+    props.setProperty(SECURITY_MANAGER, ExpirableSecurityManager.class.getName());
+    securityService = SecurityServiceFactory.create(this.props);
+  }
+
+  @After
+  public void after() throws Exception {
+    securityService.logout();
+    ExpirableSecurityManager.reset();
+  }
+
+  @Test
+  public void testThrowAuthenticationExpiredException() {
+    ExpirableSecurityManager.addExpiredUser("data");
+    this.securityService.login(loginCredentials("data", "data"));
+    assertThatThrownBy(() -> this.securityService.authorize(ResourcePermissions.DATA_READ))
+        .isInstanceOf(AuthenticationExpiredException.class);
+  }
+
+  private Properties loginCredentials(String username, String password) {
+    Properties credentials = new Properties();
+    credentials.put(ResourceConstants.USER_NAME, username);
+    credentials.put(ResourceConstants.PASSWORD, password);
+    return credentials;
+  }
+}
diff --git a/geode-core/src/integrationTest/resources/shiro.ini b/geode-core/src/integrationTest/resources/shiro.ini
new file mode 100644
index 0000000..a9746a5
--- /dev/null
+++ b/geode-core/src/integrationTest/resources/shiro.ini
@@ -0,0 +1,40 @@
+# 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.
+
+# the users and roles in this file needs to be kept in sync with shiro.ini
+# since they are used by the same test to test ShiroUtil
+# -----------------------------------------------------------------------------
+# Users and their (optional) assigned roles
+# username = password, role1, role2, ..., roleN
+# -----------------------------------------------------------------------------
+[users]
+root = secret, admin
+guest = guest, guest
+regionAReader = password, readRegionA
+regionAUser = password, useRegionA
+dataReader = 12345, readData
+reader = 12345, readAll
+
+# -----------------------------------------------------------------------------
+# Roles with assigned permissions
+# roleName = perm1, perm2, ..., permN
+# -----------------------------------------------------------------------------
+[roles]
+admin = *
+guest = none
+readRegionA = DATA:READ:RegionA
+useRegionA = *:*:RegionA
+readData = DATA:READ
+readAll = *:READ
\ No newline at end of file
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java
index 69fb338..c5fe32a 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java
@@ -15,6 +15,8 @@
 
 package org.apache.geode.cache.client.internal;
 
+import static org.apache.geode.internal.cache.tier.sockets.ServerConnection.USER_NOT_FOUND;
+
 import java.io.IOException;
 import java.io.NotSerializableException;
 import java.net.SocketException;
@@ -59,6 +61,7 @@ import org.apache.geode.internal.cache.tier.sockets.MessageTooLargeException;
 import org.apache.geode.internal.cache.wan.BatchException70;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.security.AuthenticationExpiredException;
 import org.apache.geode.security.AuthenticationRequiredException;
 import org.apache.geode.security.GemFireSecurityException;
 import org.apache.geode.util.internal.GeodeGlossary;
@@ -740,8 +743,9 @@ public class OpExecutorImpl implements ExecutablePool {
     } catch (final ServerConnectivityException sce) {
       final Throwable cause = sce.getCause();
       if ((cause instanceof AuthenticationRequiredException
-          && "User authorization attributes not found.".equals(cause.getMessage()))
-          || sce.getMessage().contains("Connection error while authenticating user")) {
+          && USER_NOT_FOUND.equals(cause.getMessage()))
+          || sce.getMessage().contains("Connection error while authenticating user")
+          || cause instanceof AuthenticationExpiredException) {
         // 2nd exception-message above is from AbstractOp.sendMessage()
 
         if (pool.getMultiuserAuthentication()) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
index 8ee4993..5435f9b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
@@ -100,6 +100,8 @@ public class ServerConnection implements Runnable {
   private static final String DISALLOW_INTERNAL_MESSAGES_WITHOUT_CREDENTIALS_NAME =
       "geode.disallow-internal-messages-without-credentials";
 
+  public static final String USER_NOT_FOUND = "User authorization attributes not found.";
+
   /**
    * When true requires some formerly credential-less messages to carry credentials. See GEODE-3249
    * and ServerConnection.isInternalMessage()
@@ -1774,7 +1776,7 @@ public class ServerConnection implements Runnable {
       logger.debug("Unexpected exception {}", npe.toString());
     }
     if (uaa == null) {
-      throw new AuthenticationRequiredException("User authorization attributes not found.");
+      throw new AuthenticationRequiredException(USER_NOT_FOUND);
     }
     return uaa;
   }
@@ -1818,7 +1820,7 @@ public class ServerConnection implements Runnable {
       logger.debug("Unexpected exception", npe);
     }
     if (uaa == null) {
-      throw new AuthenticationRequiredException("User authorization attributes not found.");
+      throw new AuthenticationRequiredException(USER_NOT_FOUND);
     }
 
     return uaa.getPostAuthzRequest();
diff --git a/geode-core/src/main/java/org/apache/geode/security/AuthenticationExpiredException.java b/geode-core/src/main/java/org/apache/geode/security/AuthenticationExpiredException.java
new file mode 100644
index 0000000..d4ec79d
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/AuthenticationExpiredException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.geode.security;
+
+/**
+ * This exception is thrown by the SecurityManager's authorize method to indicate that the
+ * authentication has expired.
+ */
+public class AuthenticationExpiredException extends GemFireSecurityException {
+  private static final long serialVersionUID = 1771792091260325297L;
+
+  public AuthenticationExpiredException(String message) {
+    super(message);
+  }
+
+  public AuthenticationExpiredException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/security/SecurityManager.java b/geode-core/src/main/java/org/apache/geode/security/SecurityManager.java
index 301b3d5..7487d5f 100644
--- a/geode-core/src/main/java/org/apache/geode/security/SecurityManager.java
+++ b/geode-core/src/main/java/org/apache/geode/security/SecurityManager.java
@@ -74,8 +74,11 @@ public interface SecurityManager {
    * @param principal The principal that's requesting the permission
    * @param permission The permission requested
    * @return true if authorized, false if not
+   *
+   * @throw AuthenticationExpiredException if the principal has expired.
    */
-  default boolean authorize(Object principal, ResourcePermission permission) {
+  default boolean authorize(Object principal, ResourcePermission permission)
+      throws AuthenticationExpiredException {
     return true;
   }
 
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index 69b6ce0..ca7686b 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -462,7 +462,6 @@ org/apache/geode/management/internal/functions/RebalanceFunction,true,1
 org/apache/geode/management/internal/functions/RestoreRedundancyFunction,true,-8991672237560920252
 org/apache/geode/management/internal/operation/OperationState,true,8212319653561969588,locator:java/lang/String,opId:java/lang/String,operation:org/apache/geode/management/api/ClusterManagementOperation,operationEnd:java/util/Date,operationStart:java/util/Date,result:org/apache/geode/management/runtime/OperationResult,throwable:java/lang/Throwable
 org/apache/geode/management/internal/web/domain/QueryParameterSource,true,34131123582155,objectName:javax/management/ObjectName,queryExpression:javax/management/QueryExp
-org/apache/geode/management/internal/web/shell/MBeanAccessException,true,813768898269516238
 org/apache/geode/pdx/FieldType,false,defaultSerializedValue:java/nio/ByteBuffer,defaultValue:java/lang/Object,isFixedWidth:boolean,name:java/lang/String,width:int
 org/apache/geode/pdx/JSONFormatter$states,false
 org/apache/geode/pdx/JSONFormatterException,true,1
@@ -478,6 +477,7 @@ org/apache/geode/pdx/internal/EnumInfo$PdxInstanceEnumInfo,true,7907582104525106
 org/apache/geode/pdx/internal/PdxInputStream,false
 org/apache/geode/pdx/internal/PdxReaderImpl,true,-6094553093860427759,blobType:org/apache/geode/pdx/internal/PdxType,dis:org/apache/geode/pdx/internal/PdxInputStream
 org/apache/geode/security/AuthTokenEnabledComponents,false
+org/apache/geode/security/AuthenticationExpiredException,true,1771792091260325297
 org/apache/geode/security/AuthenticationFailedException,true,-8202866472279088879
 org/apache/geode/security/AuthenticationRequiredException,true,4675976651103154919
 org/apache/geode/security/GemFireSecurityException,true,3814254578203076926,cause:java/lang/Throwable
diff --git a/geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationDUnitTest.java b/geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationDUnitTest.java
new file mode 100644
index 0000000..117cc77
--- /dev/null
+++ b/geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationDUnitTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.geode.security;
+
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.apache.geode.test.version.VersionManager.CURRENT_VERSION;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientRegionFactory;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.test.dunit.rules.ClientVM;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@Category({SecurityTest.class})
+@RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class AuthExpirationDUnitTest {
+
+  @Parameterized.Parameter
+  public String clientVersion;
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<String> data() {
+    // only test the current version and the latest released version
+    return Arrays.asList(CURRENT_VERSION, "1.13.3");
+  }
+
+  @Rule
+  public ClusterStartupRule lsRule = new ClusterStartupRule();
+
+
+  @Rule
+  public RestoreSystemProperties restore = new RestoreSystemProperties();
+
+  @Rule
+  public ServerStarterRule server = new ServerStarterRule()
+      .withProperty(SECURITY_MANAGER, ExpirableSecurityManager.class.getName())
+      .withRegion(RegionShortcut.REPLICATE, "region");
+
+  @After
+  public void after() {
+    // make sure after each test, the values of the ExpirationManager are reset
+    ExpirableSecurityManager.reset();
+  }
+
+  @Test
+  public void clientShouldReAuthenticateWhenCredentialExpiredAndOperationSucceed()
+      throws Exception {
+    int serverPort = server.getPort();
+    ClientVM clientVM = lsRule.startClientVM(0, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withPoolSubscription(true)
+            .withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      ClientCache clientCache = ClusterStartupRule.getClientCache();
+      UpdatableUserAuthInitialize.setUser("user1");
+      ClientRegionFactory clientRegionFactory =
+          clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+      Region region = clientRegionFactory.create("region");
+      region.put(0, "value0");
+    });
+
+    // expire the current user
+    ExpirableSecurityManager.addExpiredUser("user1");
+
+    // do a second put, if this is successful, it means new credentials are provided
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user2");
+      ClientCache clientCache = ClusterStartupRule.getClientCache();
+      Region region = clientCache.getRegion("region");
+      region.put(1, "value1");
+    });
+
+    // all put operation succeeded
+    Region<Object, Object> region = server.getCache().getRegion("/region");
+    assertThat(ExpirableSecurityManager.getExpiredUsers().size()).isEqualTo(1);
+    assertThat(ExpirableSecurityManager.getExpiredUsers().contains("user1")).isTrue();
+    assertThat(region.size()).isEqualTo(2);
+  }
+
+}
diff --git a/geode-junit/src/main/java/org/apache/geode/security/ExpirableSecurityManager.java b/geode-junit/src/main/java/org/apache/geode/security/ExpirableSecurityManager.java
new file mode 100644
index 0000000..fc1021d
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/security/ExpirableSecurityManager.java
@@ -0,0 +1,56 @@
+/*
+ * 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.geode.security;
+
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.geode.examples.SimpleSecurityManager;
+
+/**
+ * this is a test security manager that will authenticate credentials when username matches the
+ * password. It will authorize all operations. It keeps a list of expired users, and will throw
+ * AuthenticationExpiredException if the user is in that list. This security manager is usually used
+ * with NewCredentialAuthInitialize.
+ *
+ * make sure to call reset after each test to clean things up.
+ */
+public class ExpirableSecurityManager extends SimpleSecurityManager {
+  // use static field for ease of testing since there is only one instance of this in each VM
+  // we only need ConcurrentHashSet here, but map is only construct available in the library
+  private static final Set<String> EXPIRED_USERS = ConcurrentHashMap.newKeySet();
+
+  @Override
+  public boolean authorize(Object principal, ResourcePermission permission) {
+    if (EXPIRED_USERS.contains(principal)) {
+      throw new AuthenticationExpiredException("User authentication expired.");
+    }
+    // always authorized
+    return true;
+  }
+
+  public static void addExpiredUser(String user) {
+    EXPIRED_USERS.add(user);
+  }
+
+  public static Set<String> getExpiredUsers() {
+    return EXPIRED_USERS;
+  }
+
+  public static void reset() {
+    EXPIRED_USERS.clear();
+  }
+}
diff --git a/geode-junit/src/main/java/org/apache/geode/security/UpdatableUserAuthInitialize.java b/geode-junit/src/main/java/org/apache/geode/security/UpdatableUserAuthInitialize.java
new file mode 100644
index 0000000..0a0e6b4
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/security/UpdatableUserAuthInitialize.java
@@ -0,0 +1,54 @@
+/*
+ * 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.geode.security;
+
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.geode.distributed.DistributedMember;
+
+/**
+ * this is used in conjunction with ExpirableSecurityManager. It will create a new set of
+ * credentials every time getCredentials are called, and they will always be authenticated
+ * and authorized by the ExpirableSecurityManager.
+ *
+ * make sure reset is called after each test to clean things up.
+ */
+public class UpdatableUserAuthInitialize implements AuthInitialize {
+  // use static field for ease of testing since there is only one instance of this in each VM
+  private static final AtomicReference<String> user = new AtomicReference<>();
+
+  @Override
+  public Properties getCredentials(Properties securityProps, DistributedMember server,
+      boolean isPeer) throws AuthenticationFailedException {
+    Properties credentials = new Properties();
+    credentials.put("security-username", user.get());
+    credentials.put("security-password", user.get());
+    return credentials;
+  }
+
+  public static String getUser() {
+    return user.get();
+  }
+
+  public static void setUser(String newValue) {
+    user.set(newValue);
+  }
+
+  public static void reset() {
+    user.set(null);
+  }
+}
diff --git a/geode-old-client-support/src/main/java/com/gemstone/gemfire/OldClientSupportProvider.java b/geode-old-client-support/src/main/java/com/gemstone/gemfire/OldClientSupportProvider.java
index ccf3c1a..581fc98 100644
--- a/geode-old-client-support/src/main/java/com/gemstone/gemfire/OldClientSupportProvider.java
+++ b/geode-old-client-support/src/main/java/com/gemstone/gemfire/OldClientSupportProvider.java
@@ -14,6 +14,8 @@
  */
 package com.gemstone.gemfire;
 
+import static org.apache.geode.internal.cache.tier.sockets.ServerConnection.USER_NOT_FOUND;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.util.Map;
@@ -27,6 +29,8 @@ import org.apache.geode.internal.cache.tier.sockets.OldClientSupportService;
 import org.apache.geode.internal.serialization.KnownVersion;
 import org.apache.geode.internal.serialization.VersionedDataOutputStream;
 import org.apache.geode.management.internal.beans.CacheServiceMBeanBase;
+import org.apache.geode.security.AuthenticationExpiredException;
+import org.apache.geode.security.AuthenticationRequiredException;
 import org.apache.geode.util.internal.GeodeGlossary;
 
 import com.gemstone.gemfire.cache.execute.EmtpyRegionFunctionException;
@@ -121,11 +125,21 @@ public class OldClientSupportProvider implements OldClientSupportService {
     if (theThrowable == null) {
       return theThrowable;
     }
+
+    String className = theThrowable.getClass().getName();
+
+    // backward compatibility for authentication expiration
+    if (clientVersion.isOlderThan(KnownVersion.GEODE_1_15_0)) {
+      if (className.equals(AuthenticationExpiredException.class.getName())) {
+        return new AuthenticationRequiredException(USER_NOT_FOUND);
+      }
+    }
+
     if (clientVersion.isNotOlderThan(KnownVersion.GFE_90)) {
       return theThrowable;
     }
 
-    String className = theThrowable.getClass().getName();
+
 
     // this class has been renamed, so it cannot be automatically translated
     // during java deserialization