You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@geode.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/12/12 19:59:02 UTC

[jira] [Commented] (GEODE-4076) Client receives a GemFireSecurityException instead of AuthenticationRequiredException

    [ https://issues.apache.org/jira/browse/GEODE-4076?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16288173#comment-16288173 ] 

ASF GitHub Bot commented on GEODE-4076:
---------------------------------------

jinmeiliao closed pull request #1147: GEODE-4076: throw AuthenticationRequiredException instead of GemfireSecurityException
URL: https://github.com/apache/geode/pull/1147
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUserAuths.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUserAuths.java
index 5933162645..d59fbe103e 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUserAuths.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUserAuths.java
@@ -51,6 +51,7 @@ public long putUserAuth(UserAuthAttributes userAuthAttr) {
   public long putSubject(Subject subject) {
     long newId = getNextID();
     uniqueIdVsSubject.put(newId, subject);
+    logger.debug("Subject of {} added.", newId);
     return newId;
   }
 
@@ -83,6 +84,7 @@ public Subject getSubject(long userId) {
 
   public boolean removeSubject(long userId) {
     Subject subject = uniqueIdVsSubject.remove(userId);
+    logger.debug("Subject of {} removed.", userId);
     if (subject == null)
       return false;
 
@@ -130,6 +132,7 @@ public boolean removeUserId(long userId, boolean keepAlive) {
     UserAuthAttributes uaa = uniqueIdVsUserAuth.get(userId);
     if (uaa != null && !(uaa.isDurable() && keepAlive)) {
       uaa = uniqueIdVsUserAuth.remove(userId);
+      logger.debug("UserAuth of {} removed.");
       if (uaa != null) {
         cleanUserAuth(uaa);
         return true;
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 368a2f8048..965e4bde5a 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
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.internal.cache.tier.sockets;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTHENTICATOR;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -779,9 +779,18 @@ protected void doNormalMsg() {
             && !isInternalMessage(this.requestMsg, allowInternalMessagesWithoutCredentials)
             && !this.communicationMode.isWAN()) {
           long uniqueId = getUniqueId();
+          String messageType = MessageType.getString(this.requestMsg.getMessageType());
           Subject subject = this.clientUserAuths.getSubject(uniqueId);
           if (subject != null) {
             threadState = securityService.bindSubject(subject);
+            logger.debug("Bound {} with uniqueId {} for message {} with {}", subject.getPrincipal(),
+                uniqueId, messageType, this.getName());
+          } else if (uniqueId == 0) {
+            logger.debug("No unique ID yet. {}, {}", messageType, this.getName());
+          } else {
+            logger.error("Failed to bind the subject of uniqueId {} for message {} with {}",
+                uniqueId, messageType, this.getName());
+            throw new AuthenticationRequiredException("Failed to find the authenticated user.");
           }
         }
 
@@ -1039,7 +1048,6 @@ public boolean removeUserAuth(Message msg, boolean keepalive) {
       if (principal instanceof Subject) {
         Subject subject = (Subject) principal;
         uniqueId = this.clientUserAuths.putSubject(subject);
-        logger.info(this.clientUserAuths);
       } else {
         // this sets principal in map as well....
         uniqueId = ServerHandShakeProcessor.getUniqueId(this, (Principal) principal);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
index 2a60723aae..393db2bfa2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
@@ -114,7 +114,7 @@ public Subject getSubject() {
     currentUser = SecurityUtils.getSubject();
 
     if (currentUser == null || currentUser.getPrincipal() == null) {
-      throw new GemFireSecurityException("Error: Anonymous User");
+      throw new AuthenticationRequiredException("Failed to find the authenticated user.");
     }
 
     return currentUser;
@@ -138,7 +138,7 @@ public Subject login(final Properties credentials) {
       logger.debug("Logging in " + token.getPrincipal());
       currentUser.login(token);
     } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
+      logger.info("error logging in: " + token.getPrincipal());
       throw new AuthenticationFailedException(
           "Authentication error. Please check your credentials.", e);
     }
@@ -155,7 +155,7 @@ public void logout() {
       logger.debug("Logging out " + currentUser.getPrincipal());
       currentUser.logout();
     } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
+      logger.info("error logging out: " + currentUser.getPrincipal());
       throw new GemFireSecurityException(e.getMessage(), e);
     }
 
@@ -233,7 +233,7 @@ public void authorize(final ResourcePermission context) {
       currentUser.checkPermission(context);
     } catch (ShiroException e) {
       String msg = currentUser.getPrincipal() + " not authorized for " + context;
-      logger.info(msg);
+      logger.info("NotAuthorizedException: {}", msg);
       throw new NotAuthorizedException(msg, e);
     }
   }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiClientDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiClientDUnitTest.java
new file mode 100644
index 0000000000..46deac10cb
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiClientDUnitTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Cache;
+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.ClientCacheFactory;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.client.ServerOperationException;
+import org.apache.geode.examples.SimpleSecurityManager;
+import org.apache.geode.security.templates.UserPasswordAuthInit;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class MultiClientDUnitTest {
+
+  private static int SESSION_COUNT = 2;
+  private static int KEY_COUNT = 20;
+
+  @ClassRule
+  public static LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  private static MemberVM locator, server1, server2;
+  private static VM client3, client4, client5, client6;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    IgnoredException.addIgnoredException("org.apache.geode.security.AuthenticationFailedException");
+    Properties locatorProps = new Properties();
+    locatorProps.setProperty(SECURITY_MANAGER, SimpleSecurityManager.class.getCanonicalName());
+    locator = lsRule.startLocatorVM(0, locatorProps);
+
+    Properties serverProps = new Properties();
+    serverProps.setProperty("security-username", "cluster");
+    serverProps.setProperty("security-password", "cluster");
+    server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+    server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
+
+    MemberVM.invokeInEveryMember(() -> {
+      Cache cache = LocatorServerStartupRule.getCache();
+      cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
+    }, server1, server2);
+
+    client3 = lsRule.getVM(3);
+    client4 = lsRule.getVM(4);
+    client5 = lsRule.getVM(5);
+    client6 = lsRule.getVM(6);
+  }
+
+  @Test
+  public void multiClient() throws ExecutionException, InterruptedException, TimeoutException {
+    int locatorPort = locator.getPort();
+    int server1Port = server1.getPort();
+    int server2Port = server2.getPort();
+
+
+    // client 3 keeps logging in, do some successful put, and log out and keep doing it for multiple
+    // times
+    AsyncInvocation vm3Invoke = client3.invokeAsync("run as data", () -> {
+      for (int i = 0; i < SESSION_COUNT; i++) {
+        ClientCache cache = createClientCache("data", "data", server1Port, server2Port);
+        Region region =
+            cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("region");
+        for (int j = 0; j < KEY_COUNT; j++) {
+          region.put(i + "" + j, i + "" + j);
+        }
+        cache.close();
+      }
+    });
+
+    // client 4 keeps logging in, do an unauthorized put, and log out
+    AsyncInvocation vm4Invoke = client4.invokeAsync("run as stranger", () -> {
+      for (int i = 0; i < SESSION_COUNT; i++) {
+        ClientCache cache = createClientCache("stranger", "stranger", server1Port, server2Port);
+        Region region =
+            cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("region");
+        for (int j = 0; j < KEY_COUNT; j++) {
+          String value = i + "" + j;
+          assertThatThrownBy(() -> region.put(value, value))
+              .isInstanceOf(ServerOperationException.class);
+        }
+        cache.close();
+      }
+    });
+
+    // client 5 keeps logging in, do some successful get, and log out
+    AsyncInvocation vm5Invoke = client5.invokeAsync("run as data", () -> {
+      for (int i = 0; i < SESSION_COUNT; i++) {
+        ClientCache cache = createClientCache("data", "data", server1Port, server2Port);
+        Region region =
+            cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("region");
+        for (int j = 0; j < KEY_COUNT; j++) {
+          region.get(i + "" + j);
+        }
+        cache.close();
+      }
+    });
+
+    // // client 6 keeps logging in with incorrect
+    AsyncInvocation vm6Invoke = client6.invokeAsync("run as invalid user", () -> {
+      for (int i = 0; i < SESSION_COUNT; i++) {
+        ClientCache cache =
+            createClientCache("dataWithWrongPswd", "password", server1Port, server2Port);
+        Region region =
+            cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("region");
+        for (int j = 0; j < 1; j++) {
+          String key = i + "" + j;
+          assertThatThrownBy(() -> region.get(key)).isInstanceOf(ServerOperationException.class);
+        }
+        cache.close();
+      }
+    });
+
+    vm3Invoke.await(60, TimeUnit.MINUTES);
+    vm4Invoke.await(60, TimeUnit.MINUTES);
+    vm5Invoke.await(60, TimeUnit.MINUTES);
+    vm6Invoke.await(60, TimeUnit.MINUTES);
+  }
+
+  private static ClientCache createClientCache(String username, String password, int locatorPort) {
+    ClientCache cache = new ClientCacheFactory(getClientCacheProperties(username, password))
+        .setPoolSubscriptionEnabled(false).addPoolLocator("localhost", locatorPort).create();
+    return cache;
+  }
+
+  private static ClientCache createClientCache(String username, String password, int server1Port,
+      int server2Port) {
+    ClientCache cache = new ClientCacheFactory(getClientCacheProperties(username, password))
+        .setPoolSubscriptionEnabled(false).addPoolServer("localhost", server1Port)
+        .addPoolServer("localhost", server2Port).create();
+    return cache;
+  }
+
+  private static Properties getClientCacheProperties(String username, String password) {
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, username);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, password);
+    props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
+    props.setProperty(LOCATORS, "");
+    props.setProperty(MCAST_PORT, "0");
+    return props;
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiGfshDUnitTest.java
similarity index 95%
rename from geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/management/internal/security/MultiGfshDUnitTest.java
index ba6f2fafcf..844d6e54bd 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiGfshDUnitTest.java
@@ -48,7 +48,7 @@
 import org.apache.geode.test.junit.rules.Member;
 
 @Category({DistributedTest.class, SecurityTest.class})
-public class MultiUserDUnitTest {
+public class MultiGfshDUnitTest {
 
   @Rule
   public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
@@ -75,11 +75,13 @@ public void testMultiUser() throws IOException, JSONException, InterruptedExcept
     // test is done.
     VM vm1 = lsRule.getVM(1);
     AsyncInvocation vm1Invoke = vm1.invokeAsync("run as data-reader", () -> {
-      GfshCommandRule gfsh = new GfshCommandRule();
-      gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManager, "dataRead", "dataRead");
+      while (true) {
+        GfshCommandRule gfsh = new GfshCommandRule();
+        gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManager, "dataRead", "dataRead");
 
-      Awaitility.waitAtMost(5, TimeUnit.MILLISECONDS);
-      gfsh.close();
+        Awaitility.waitAtMost(5, TimeUnit.MILLISECONDS);
+        gfsh.close();
+      }
     });
 
     VM vm2 = lsRule.getVM(2);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserAuthenticationDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserAuthenticationDUnitTest.java
new file mode 100644
index 0000000000..ab79d6ea64
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserAuthenticationDUnitTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.Properties;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionService;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientCacheFactory;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.client.ServerOperationException;
+import org.apache.geode.examples.SimpleSecurityManager;
+import org.apache.geode.security.templates.UserPasswordAuthInit;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+
+@Category(DistributedTest.class)
+public class MultiUserAuthenticationDUnitTest {
+
+  private static int SESSION_COUNT = 2;
+  private static int KEY_COUNT = 2;
+
+  @ClassRule
+  public static LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  @ClassRule
+  public static GfshCommandRule gfsh = new GfshCommandRule();
+
+  private static MemberVM locator, server1, server2;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    IgnoredException.addIgnoredException("org.apache.geode.security.AuthenticationFailedException");
+    Properties locatorProps = new Properties();
+    locatorProps.setProperty(SECURITY_MANAGER, SimpleSecurityManager.class.getCanonicalName());
+    locator = lsRule.startLocatorVM(0, locatorProps);
+
+    Properties serverProps = new Properties();
+    serverProps.setProperty("security-username", "cluster");
+    serverProps.setProperty("security-password", "cluster");
+    server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+    server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
+
+    // create region and put in some values
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndAssertThat("create region --name=region --type=PARTITION").statusIsSuccess();
+  }
+
+  @Test
+  public void multiAuthenticatedView() throws Exception {
+    int locatorPort = locator.getPort();
+    for (int i = 0; i < SESSION_COUNT; i++) {
+      ClientCache cache = new ClientCacheFactory(getClientCacheProperties("stranger", "stranger"))
+          .setPoolSubscriptionEnabled(true).setPoolMultiuserAuthentication(true)
+          .addPoolLocator("localhost", locatorPort).create();
+
+      RegionService regionService1 =
+          cache.createAuthenticatedView(getClientCacheProperties("data", "data"));
+      RegionService regionService2 =
+          cache.createAuthenticatedView(getClientCacheProperties("cluster", "cluster"));
+
+      cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("region");
+
+      Region region = regionService1.getRegion("/region");
+      Region region2 = regionService2.getRegion("/region");
+      for (int j = 0; j < KEY_COUNT; j++) {
+        String value = i + "" + j;
+        region.put(value, value);
+        assertThatThrownBy(() -> region2.put(value, value))
+            .isInstanceOf(ServerOperationException.class);
+      }
+      regionService1.close();
+      regionService2.close();
+      cache.close();
+    }
+  }
+
+  private static Properties getClientCacheProperties(String username, String password) {
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, username);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, password);
+    props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
+    props.setProperty(LOCATORS, "");
+    props.setProperty(MCAST_PORT, "0");
+    return props;
+  }
+
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Client receives a GemFireSecurityException instead of AuthenticationRequiredException
> -------------------------------------------------------------------------------------
>
>                 Key: GEODE-4076
>                 URL: https://issues.apache.org/jira/browse/GEODE-4076
>             Project: Geode
>          Issue Type: Bug
>          Components: security
>            Reporter: Jinmei Liao
>            Assignee: Jinmei Liao
>
> In some cases, when a client message comes in, the ServerConnection that handles this client message fails to find the authenticated user for this clientID, the older security framework throws an AuthenticationRequiredException while the new Security framework throws GemfireSecurityException. This might causes the client not do a retry connection. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)