You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/09/20 21:49:33 UTC

[GitHub] [geode] jinmeiliao opened a new pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

jinmeiliao opened a new pull request #6885:
URL: https://github.com/apache/geode/pull/6885


   * add more tests
   * minor refactor
   
   Most of this PR is adding tests, with minor refactor in `ClientUserAuth`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r717742989



##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");

Review comment:
       yes, controller VM is a client VM as well. Both uses its own `UpdatableUserAuthInitialize`

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");
+    expireUserOnAllVms("user1");
+    doPutsUsingAnotherClient(locatorPort, "user3", 100);
+
+    // make sure listener still gets all the events
+    await().untilAsserted(() -> assertThat(listener.getKeys()).hasSize(100));
+    ExpirableSecurityManager securityManager = collectSecurityManagers(server1, server2);
+    assertThat(securityManager.getAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion");
+    assertThat(securityManager.getUnAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion:key0");

Review comment:
       when `key0` is denied, the server will ask the client to re-authenticate, so client will sends in `user2` credentials, so all keys are authorized under `user2`.

##########
File path: geode-junit/src/main/java/org/apache/geode/security/ExpirableSecurityManager.java
##########
@@ -90,4 +90,11 @@ private void addToMap(Map<String, List<String>> maps, Object user,
     }
     maps.put(user.toString(), list);
   }
+
+  public void close() {

Review comment:
       will do




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r715789942



##########
File path: geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyTest.java
##########
@@ -120,4 +120,12 @@ public void deliverMessageWhenSubjectIsNull() {
     verify(securityService, never()).bindSubject(subject);
     verify(securityService, never()).postProcess(any(), any(), any(), anyBoolean());
   }
+
+  @Test
+  public void replacingSubjectShouldNotLogout() {
+    proxy = spy(new CacheClientProxy(cache, notifier, socket, id, true, (byte) 1, version, 1L, true,
+        securityService, subject, clock, statsFactory, proxyStatsFactory, dispatcherFactory));
+    proxy.setSubject(mock(Subject.class));
+    verify(subject, never()).logout();
+  }

Review comment:
       +1

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationDUnitTest.java
##########
@@ -416,21 +439,304 @@ public void cqOlderClientWithClientInteractionWillDeliverEventEventually() throw
     });
   }
 
-  private void startClientWithCQ() throws Exception {
+  @Test
+  public void registeredInterestForDefaultInterestPolicy() throws Exception {
+    int serverPort = server.getPort();
+    clientVM = cluster.startClientVM(0, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withCacheSetup(
+                ccf -> ccf.setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(0))
+            .withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user1");
+      Region<Object, Object> clientRegion = ClusterStartupRule.getClientCache()
+          .createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create("region");
+      clientRegion.registerInterestForAllKeys();
+    });
+
+    Region<Object, Object> region = server.getCache().getRegion("/region");
+    region.put("1", "value1");
+
+    // refresh user before we expire user1, otherwise we might still be using expired
+    // users in some client operations
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user2");
+    });
+
+    getSecurityManager().addExpiredUser("user1");
+    region.put("2", "value2");
+
+    // for new client, a message will be sent to client to trigger re-auth
+    // for old client, server close the proxy, but client have reconnect mechanism which
+    // also triggers re-auth. In both cases, no message loss since old client
+    // will re-register interests with default interest policy
+    clientVM.invoke(() -> {
+      Region<Object, Object> clientRegion =
+          ClusterStartupRule.getClientCache().getRegion("region");
+      await().untilAsserted(
+          () -> assertThat(clientRegion.keySet()).hasSize(2));
+      // but client will reconnect successfully using the 2nd user
+      clientRegion.put("2", "value2");
+    });
+
+    // user1 should not be used to put key2 to the region in any cases
+    assertThat(getSecurityManager().getAuthorizedOps().get("user1"))
+        .doesNotContain("DATA:READ:region:2");
+  }
+
+  @Test
+  public void registeredInterestForInterestPolicyNone() throws Exception {
+    int serverPort = server.getPort();
+    clientVM = cluster.startClientVM(0, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withCacheSetup(
+                ccf -> ccf.setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(0))
+            .withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user1");
+      Region<Object, Object> clientRegion = ClusterStartupRule.getClientCache()
+          .createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create("region");
+      clientRegion.registerInterestForAllKeys(InterestResultPolicy.NONE);
+    });
+
+    Region<Object, Object> region = server.getCache().getRegion("/region");
+    region.put("1", "value1");
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user2");
+    });
+    getSecurityManager().addExpiredUser("user1");
+
+    region.put("2", "value2");
+
+    // for old client, server close the proxy, client have reconnect mechanism which
+    // also triggers re-auth, clients re-register interest, but with InterestResultPolicy.NONE
+    // there would be message loss
+    if (TestVersion.compare(clientVersion, test_start_version) <= 0) {
+      clientVM.invoke(() -> {
+        Region<Object, Object> clientRegion =
+            ClusterStartupRule.getClientCache().getRegion("region");
+        await().during(10, TimeUnit.SECONDS).untilAsserted(
+            () -> assertThat(clientRegion.keySet().size()).isLessThan(2));
+        // but client will reconnect successfully using the 2nd user
+        clientRegion.put("2", "value2");
+      });
+    } else {
+      // new client would have no message loss
+      clientVM.invoke(() -> {
+        Region<Object, Object> clientRegion =
+            ClusterStartupRule.getClientCache().getRegion("region");
+        await().untilAsserted(
+            () -> assertThat(clientRegion.keySet()).hasSize(2));
+      });
+    }
+
+    // user1 should not be used to put key2 to the region in any cases
+    assertThat(getSecurityManager().getAuthorizedOps().get("user1"))
+        .doesNotContain("DATA:READ:region:2");
+  }
+
+  @Test
+  public void newClient_registeredInterest_slowReAuth_policyDefault() throws Exception {
+    // this test only test the newer client
+    if (TestVersion.compare(clientVersion, test_start_version) <= 0) {
+      return;
+    }
+
+    int serverPort = server.getPort();
+    clientVM = cluster.startClientVM(0, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withPoolSubscription(true)
+            .withServerConnection(serverPort));
+
+    ClientVM client2 = cluster.startClientVM(1, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withPoolSubscription(true)
+            .withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user1");
+      Region<Object, Object> region = ClusterStartupRule.getClientCache()
+          .createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create("region");
+
+      // this test will succeed because when clients re-connects, it will re-register inteest
+      // a new queue will be created with all the data. Old queue is destroyed.
+      region.registerInterestForAllKeys();
+      UpdatableUserAuthInitialize.setUser("user11");
+      // wait for time longer than server's max time to wait to ree-authenticate
+      UpdatableUserAuthInitialize.setWaitTime(6000);
+    });
+
+    AsyncInvocation invokePut = client2.invokeAsync(() -> {

Review comment:
       +1




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jchen21 commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jchen21 commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r717058932



##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");

Review comment:
       `UpdatableUserAuthInitialize` is used here on the controller VM, while in `doPutsUsingAnotherClient`, `UpdatableUserAuthInitialize` is used in a client VM. Is it expected?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jchen21 commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jchen21 commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r717046763



##########
File path: geode-junit/src/main/java/org/apache/geode/security/ExpirableSecurityManager.java
##########
@@ -90,4 +90,11 @@ private void addToMap(Map<String, List<String>> maps, Object user,
     }
     maps.put(user.toString(), list);
   }
+
+  public void close() {

Review comment:
       Better add annotation `@Override`

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");

Review comment:
       Is `user2` required here? 
   `UpdatableUserAuthInitialize` is used here on the controller VM, while in `doPutsUsingAnotherClient`, `UpdatableUserAuthInitialize` is used in a client VM. Is it expected?

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");
+    expireUserOnAllVms("user1");
+    doPutsUsingAnotherClient(locatorPort, "user3", 100);
+
+    // make sure listener still gets all the events
+    await().untilAsserted(() -> assertThat(listener.getKeys()).hasSize(100));
+    ExpirableSecurityManager securityManager = collectSecurityManagers(server1, server2);
+    assertThat(securityManager.getAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion");
+    assertThat(securityManager.getUnAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion:key0");

Review comment:
       Why this does not include `key1` through `key99`, since user1's authentication is expired?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r717744350



##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");
+    expireUserOnAllVms("user1");
+    doPutsUsingAnotherClient(locatorPort, "user3", 100);
+
+    // make sure listener still gets all the events
+    await().untilAsserted(() -> assertThat(listener.getKeys()).hasSize(100));
+    ExpirableSecurityManager securityManager = collectSecurityManagers(server1, server2);
+    assertThat(securityManager.getAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion");
+    assertThat(securityManager.getUnAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion:key0");

Review comment:
       when `key0` is denied, the server will ask the client to re-authenticate, so client will sends in `user2` credentials, so all keys are authorized under `user2`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r717744464



##########
File path: geode-junit/src/main/java/org/apache/geode/security/ExpirableSecurityManager.java
##########
@@ -90,4 +90,11 @@ private void addToMap(Map<String, List<String>> maps, Object user,
     }
     maps.put(user.toString(), list);
   }
+
+  public void close() {

Review comment:
       will do




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao merged pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao merged pull request #6885:
URL: https://github.com/apache/geode/pull/6885


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] kirklund commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r715790387



##########
File path: geode-junit/src/main/java/org/apache/geode/security/UpdatableUserAuthInitialize.java
##########
@@ -38,6 +40,16 @@ public Properties getCredentials(Properties securityProps, DistributedMember ser
     credentials.put("security-username", user.get());
     credentials.put("security-password", user.get());
 
+    Long timeToWait = waitTime.get();
+    if (timeToWait < 0) {
+      throw new AuthenticationFailedException("Something wrong happened.");
+    } else if (timeToWait > 0) {
+      try {
+        Thread.sleep(timeToWait);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }

Review comment:
       PS: I know it's geode-junit. I would still log it or rethrow it though.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r715790123



##########
File path: geode-junit/src/main/java/org/apache/geode/security/UpdatableUserAuthInitialize.java
##########
@@ -38,6 +40,16 @@ public Properties getCredentials(Properties securityProps, DistributedMember ser
     credentials.put("security-username", user.get());
     credentials.put("security-password", user.get());
 
+    Long timeToWait = waitTime.get();
+    if (timeToWait < 0) {
+      throw new AuthenticationFailedException("Something wrong happened.");
+    } else if (timeToWait > 0) {
+      try {
+        Thread.sleep(timeToWait);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }

Review comment:
       +1




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] pivotal-jbarrett commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r713258874



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUserAuths.java
##########
@@ -99,19 +98,20 @@ public Subject getSubject(final Long userId) {
     return uniqueIdVsSubject.get(userId);
   }
 
-  public boolean removeSubject(final Long userId) {
-    final Subject subject = uniqueIdVsSubject.remove(userId);
+  public void removeSubject(final Long userId) {
     logger.debug("Subject of {} removed.", userId);
+    removeSubject(uniqueIdVsSubject.remove(userId));
+  }
+
+  @VisibleForTesting
+  void removeSubject(Subject subject) {

Review comment:
       I would really like to see the increased use of `@Nullable`, and `@NotNull`, annotation where appropriate.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] kirklund commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r714102448



##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationDUnitTest.java
##########
@@ -416,21 +439,304 @@ public void cqOlderClientWithClientInteractionWillDeliverEventEventually() throw
     });
   }
 
-  private void startClientWithCQ() throws Exception {
+  @Test
+  public void registeredInterestForDefaultInterestPolicy() throws Exception {
+    int serverPort = server.getPort();
+    clientVM = cluster.startClientVM(0, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withCacheSetup(
+                ccf -> ccf.setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(0))
+            .withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user1");
+      Region<Object, Object> clientRegion = ClusterStartupRule.getClientCache()
+          .createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create("region");
+      clientRegion.registerInterestForAllKeys();
+    });
+
+    Region<Object, Object> region = server.getCache().getRegion("/region");
+    region.put("1", "value1");
+
+    // refresh user before we expire user1, otherwise we might still be using expired
+    // users in some client operations
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user2");
+    });
+
+    getSecurityManager().addExpiredUser("user1");
+    region.put("2", "value2");
+
+    // for new client, a message will be sent to client to trigger re-auth
+    // for old client, server close the proxy, but client have reconnect mechanism which
+    // also triggers re-auth. In both cases, no message loss since old client
+    // will re-register interests with default interest policy
+    clientVM.invoke(() -> {
+      Region<Object, Object> clientRegion =
+          ClusterStartupRule.getClientCache().getRegion("region");
+      await().untilAsserted(
+          () -> assertThat(clientRegion.keySet()).hasSize(2));
+      // but client will reconnect successfully using the 2nd user
+      clientRegion.put("2", "value2");
+    });
+
+    // user1 should not be used to put key2 to the region in any cases
+    assertThat(getSecurityManager().getAuthorizedOps().get("user1"))
+        .doesNotContain("DATA:READ:region:2");
+  }
+
+  @Test
+  public void registeredInterestForInterestPolicyNone() throws Exception {
+    int serverPort = server.getPort();
+    clientVM = cluster.startClientVM(0, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withCacheSetup(
+                ccf -> ccf.setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(0))
+            .withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user1");
+      Region<Object, Object> clientRegion = ClusterStartupRule.getClientCache()
+          .createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create("region");
+      clientRegion.registerInterestForAllKeys(InterestResultPolicy.NONE);
+    });
+
+    Region<Object, Object> region = server.getCache().getRegion("/region");
+    region.put("1", "value1");
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user2");
+    });
+    getSecurityManager().addExpiredUser("user1");
+
+    region.put("2", "value2");
+
+    // for old client, server close the proxy, client have reconnect mechanism which
+    // also triggers re-auth, clients re-register interest, but with InterestResultPolicy.NONE
+    // there would be message loss
+    if (TestVersion.compare(clientVersion, test_start_version) <= 0) {
+      clientVM.invoke(() -> {
+        Region<Object, Object> clientRegion =
+            ClusterStartupRule.getClientCache().getRegion("region");
+        await().during(10, TimeUnit.SECONDS).untilAsserted(
+            () -> assertThat(clientRegion.keySet().size()).isLessThan(2));
+        // but client will reconnect successfully using the 2nd user
+        clientRegion.put("2", "value2");
+      });
+    } else {
+      // new client would have no message loss
+      clientVM.invoke(() -> {
+        Region<Object, Object> clientRegion =
+            ClusterStartupRule.getClientCache().getRegion("region");
+        await().untilAsserted(
+            () -> assertThat(clientRegion.keySet()).hasSize(2));
+      });
+    }
+
+    // user1 should not be used to put key2 to the region in any cases
+    assertThat(getSecurityManager().getAuthorizedOps().get("user1"))
+        .doesNotContain("DATA:READ:region:2");
+  }
+
+  @Test
+  public void newClient_registeredInterest_slowReAuth_policyDefault() throws Exception {
+    // this test only test the newer client
+    if (TestVersion.compare(clientVersion, test_start_version) <= 0) {
+      return;
+    }
+
+    int serverPort = server.getPort();
+    clientVM = cluster.startClientVM(0, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withPoolSubscription(true)
+            .withServerConnection(serverPort));
+
+    ClientVM client2 = cluster.startClientVM(1, clientVersion,
+        c -> c.withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+            .withPoolSubscription(true)
+            .withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      UpdatableUserAuthInitialize.setUser("user1");
+      Region<Object, Object> region = ClusterStartupRule.getClientCache()
+          .createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create("region");
+
+      // this test will succeed because when clients re-connects, it will re-register inteest
+      // a new queue will be created with all the data. Old queue is destroyed.
+      region.registerInterestForAllKeys();
+      UpdatableUserAuthInitialize.setUser("user11");
+      // wait for time longer than server's max time to wait to ree-authenticate
+      UpdatableUserAuthInitialize.setWaitTime(6000);
+    });
+
+    AsyncInvocation invokePut = client2.invokeAsync(() -> {

Review comment:
       Typing here would be `AsyncInvocation<Void>`.

##########
File path: geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyTest.java
##########
@@ -120,4 +120,12 @@ public void deliverMessageWhenSubjectIsNull() {
     verify(securityService, never()).bindSubject(subject);
     verify(securityService, never()).postProcess(any(), any(), any(), anyBoolean());
   }
+
+  @Test
+  public void replacingSubjectShouldNotLogout() {
+    proxy = spy(new CacheClientProxy(cache, notifier, socket, id, true, (byte) 1, version, 1L, true,
+        securityService, subject, clock, statsFactory, proxyStatsFactory, dispatcherFactory));
+    proxy.setSubject(mock(Subject.class));
+    verify(subject, never()).logout();
+  }

Review comment:
       Looks like you don't need to use `spy` here. Nothing is overriding or verifying anything on `proxy` itself.

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationDUnitTest.java
##########
@@ -320,10 +333,20 @@ public void cqOlderClientWillNotReAuthenticate() throws Exception {
     clientVM.invoke(() -> {
       // even user gets refreshed, the old client wouldn't be able to send in the new credentials
       UpdatableUserAuthInitialize.setUser("user2");
-      await().during(6, TimeUnit.SECONDS)
+      await().during(10, TimeUnit.SECONDS)

Review comment:
       Quick question: why does this call need to change from 6 seconds to 10 seconds?

##########
File path: geode-junit/src/main/java/org/apache/geode/security/UpdatableUserAuthInitialize.java
##########
@@ -38,6 +40,16 @@ public Properties getCredentials(Properties securityProps, DistributedMember ser
     credentials.put("security-username", user.get());
     credentials.put("security-password", user.get());
 
+    Long timeToWait = waitTime.get();
+    if (timeToWait < 0) {
+      throw new AuthenticationFailedException("Something wrong happened.");
+    } else if (timeToWait > 0) {
+      try {
+        Thread.sleep(timeToWait);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }

Review comment:
       Would it be better for this test code to `throw new RuntimeException(e)`? Printing the stack trace is usually not a good idea unless you use a logger in some way. In general, the only thing that will `interrupt` a thread is shutting down an `Executor` and we want that to throw some runtime exception and abort out of there fast.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r715790726



##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationDUnitTest.java
##########
@@ -320,10 +333,20 @@ public void cqOlderClientWillNotReAuthenticate() throws Exception {
     clientVM.invoke(() -> {
       // even user gets refreshed, the old client wouldn't be able to send in the new credentials
       UpdatableUserAuthInitialize.setUser("user2");
-      await().during(6, TimeUnit.SECONDS)
+      await().during(10, TimeUnit.SECONDS)

Review comment:
       No particular reason, I guess I just want to wait a bit longer and 10 sounds better than 6.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jchen21 commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jchen21 commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r717046763



##########
File path: geode-junit/src/main/java/org/apache/geode/security/ExpirableSecurityManager.java
##########
@@ -90,4 +90,11 @@ private void addToMap(Map<String, List<String>> maps, Object user,
     }
     maps.put(user.toString(), list);
   }
+
+  public void close() {

Review comment:
       Better add annotation `@Override`

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");

Review comment:
       Is `user2` required here? 
   `UpdatableUserAuthInitialize` is used here on the controller VM, while in `doPutsUsingAnotherClient`, `UpdatableUserAuthInitialize` is used in a client VM. Is it expected?

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");
+    expireUserOnAllVms("user1");
+    doPutsUsingAnotherClient(locatorPort, "user3", 100);
+
+    // make sure listener still gets all the events
+    await().untilAsserted(() -> assertThat(listener.getKeys()).hasSize(100));
+    ExpirableSecurityManager securityManager = collectSecurityManagers(server1, server2);
+    assertThat(securityManager.getAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion");
+    assertThat(securityManager.getUnAuthorizedOps().get("user1"))
+        .containsExactly("DATA:READ:partitionRegion:key0");

Review comment:
       Why this does not include `key1` through `key99`, since user1's authentication is expired?

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");

Review comment:
       `UpdatableUserAuthInitialize` is used here on the controller VM, while in `doPutsUsingAnotherClient`, `UpdatableUserAuthInitialize` is used in a client VM. Is it expected?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao merged pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao merged pull request #6885:
URL: https://github.com/apache/geode/pull/6885


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6885: GEODE-9570: make sure re-authentication works with registered interests

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6885:
URL: https://github.com/apache/geode/pull/6885#discussion_r717742989



##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/security/AuthExpirationMultiServerDUnitTest.java
##########
@@ -174,10 +184,75 @@ public void clientConnectToLocatorShouldNotAllowOperationIfUserIsNotRefreshed()
             AuthenticationRequiredException.class, AuthenticationExpiredException.class);
       }
     }
-    ExpirableSecurityManager consolidated = combineSecurityManagerResults(server1, server2);
+    ExpirableSecurityManager consolidated = collectSecurityManagers(server1, server2);
     assertThat(consolidated.getAuthorizedOps().keySet()).isEmpty();
   }
 
+  @Test
+  public void cqWithMultiServer() throws Exception {
+    int locatorPort = locator.getPort();
+    UpdatableUserAuthInitialize.setUser("user1");
+    clientCacheRule
+        .withProperty(SECURITY_CLIENT_AUTH_INIT, UpdatableUserAuthInitialize.class.getName())
+        .withPoolSubscription(true)
+        .withLocatorConnection(locatorPort);
+    ClientCache cache = clientCacheRule.createCache();
+    EventsCqListner listener =
+        createAndExecuteCQ(cache.getQueryService(), "cq1", "select * from /" + PARTITION_REGION);
+
+    UpdatableUserAuthInitialize.setUser("user2");

Review comment:
       yes, controller VM is a client VM as well. Both uses its own `UpdatableUserAuthInitialize`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org