You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kh...@apache.org on 2017/03/28 16:46:54 UTC

[26/35] geode git commit: GEODE-2395: use random ports when starting the jmx manager and http services

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsPostProcessorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsPostProcessorTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsPostProcessorTest.java
index 62c0b85..e84cb56 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsPostProcessorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsPostProcessorTest.java
@@ -14,13 +14,11 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_POST_PROCESSOR;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.security.TestPostProcessor;
 import org.apache.geode.security.TestSecurityManager;
 import org.apache.geode.test.dunit.rules.ConnectionConfiguration;
@@ -34,30 +32,20 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.Properties;
-
 @Category({IntegrationTest.class, SecurityTest.class})
 public class GfshCommandsPostProcessorTest {
 
-  protected static int jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
-
-  static Properties properties = new Properties() {
-    {
-      setProperty(JMX_MANAGER_PORT, jmxPort + "");
-      setProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName());
-      setProperty(SECURITY_MANAGER, TestSecurityManager.class.getName());
-      setProperty("security-json",
-          "org/apache/geode/management/internal/security/cacheServer.json");
-    }
-  };
-
-
   @ClassRule
-  public static ServerStarterRule serverStarter = new ServerStarterRule().startServer(properties);
+  public static ServerStarterRule serverStarter = new ServerStarterRule().withJMXManager()
+      .withProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName())
+      .withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+      .withProperty("security-json",
+          "org/apache/geode/management/internal/security/cacheServer.json")
+      .startServer();
 
   @Rule
-  public GfshShellConnectionRule gfshConnection =
-      new GfshShellConnectionRule(jmxPort, GfshShellConnectionRule.PortType.jmxManger);
+  public GfshShellConnectionRule gfshConnection = new GfshShellConnectionRule(
+      serverStarter.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger);
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsSecurityTest.java
index 45d437a..348e2ea 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/GfshCommandsSecurityTest.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -24,7 +22,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.result.CommandResult;
@@ -44,41 +41,19 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.util.List;
-import java.util.Properties;
 
 @Category({IntegrationTest.class, SecurityTest.class})
-// @RunWith(Parameterized.class)
-// @Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class GfshCommandsSecurityTest {
+  @ClassRule
+  public static ServerStarterRule serverStarter = new ServerStarterRule().withJMXManager()
+      .withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+      .withProperty("security-json",
+          "org/apache/geode/management/internal/security/cacheServer.json")
+      .startServer();
 
-  protected static int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
-  protected static int jmxPort = ports[0];
-  protected static int httpPort = ports[1];
-
-  // can't do parameterized tests here since useHttp tests needs to be in geode-web project
-  // @Parameterized.Parameters
-  // public static Collection<Object> data() {
-  // return Arrays.asList(new Object[] {true, false});
-  // }
-  //
-  // @Parameterized.Parameter
-  // public boolean useHttp;
-
-  static Properties properties = new Properties() {
-    {
-      setProperty(JMX_MANAGER_PORT, jmxPort + "");
-      setProperty(HTTP_SERVICE_PORT, httpPort + "");
-      setProperty(SECURITY_MANAGER, TestSecurityManager.class.getName());
-      setProperty("security-json",
-          "org/apache/geode/management/internal/security/cacheServer.json");
-    }
-  };
   @Rule
-  public GfshShellConnectionRule gfshConnection =
-      new GfshShellConnectionRule(jmxPort, GfshShellConnectionRule.PortType.jmxManger);
-
-  @ClassRule
-  public static ServerStarterRule serverStarter = new ServerStarterRule().startServer(properties);
+  public GfshShellConnectionRule gfshConnection = new GfshShellConnectionRule(
+      serverStarter.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger);
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/management/internal/security/JavaRmiServerNameTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/JavaRmiServerNameTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/JavaRmiServerNameTest.java
index 070e905..6334e7e 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/JavaRmiServerNameTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/JavaRmiServerNameTest.java
@@ -16,44 +16,27 @@
 
 package org.apache.geode.management.internal.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
 import static org.junit.Assert.assertEquals;
 
-import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.junit.After;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.Properties;
-
 @Category(IntegrationTest.class)
 public class JavaRmiServerNameTest {
 
   private static final String JMX_HOST = "myHostname";
 
-  static Properties properties = new Properties() {
-    {
-      setProperty(JMX_MANAGER_PORT,
-          AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) + "");
-      setProperty("jmx-manager-hostname-for-clients", JMX_HOST);
-    }
-  };
-
   @ClassRule
-  public static ServerStarterRule serverStarter = new ServerStarterRule();
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    serverStarter.startServer(properties);
-  }
-
-
+  public static ServerStarterRule serverStarter = new ServerStarterRule()
+      .withProperty("jmx-manager-hostname-for-clients", JMX_HOST).withJMXManager().startServer();
 
-  // https://issues.apache.org/jira/browse/GEODE-1548
+  /**
+   * this is for GEODE-1548
+   */
   @Test
   public void testThatJavaRmiServerNameGetsSet() {
     assertEquals(JMX_HOST, System.getProperty("java.rmi.server.hostname"));

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
index 97b9730..a5a88c2 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
@@ -14,43 +14,45 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.assertj.core.api.Assertions.*;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.locks.DLockService;
-import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.management.LockServiceMXBean;
+import org.apache.geode.security.TestSecurityManager;
 import org.apache.geode.test.dunit.rules.ConnectionConfiguration;
 import org.apache.geode.test.dunit.rules.MBeanServerConnectionRule;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 @Category({IntegrationTest.class, SecurityTest.class})
 public class LockServiceMBeanAuthorizationJUnitTest {
-
-  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
   private LockServiceMXBean lockServiceMBean;
 
   @ClassRule
-  public static CacheServerStartupRule serverRule =
-      CacheServerStartupRule.withDefaultSecurityJson(jmxManagerPort);
+  public static ServerStarterRule server = new ServerStarterRule().withJMXManager()
+      .withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+      .withProperty(TestSecurityManager.SECURITY_JSON,
+          "org/apache/geode/management/internal/security/cacheServer.json")
+      .startServer();
 
   @Rule
-  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+  public MBeanServerConnectionRule connectionRule =
+      new MBeanServerConnectionRule(server.getJmxPort());
 
   @BeforeClass
   public static void beforeClassSetUp() {
-    Cache cache = serverRule.getCache();
+    Cache cache = server.getCache();
     DLockService.create("test-lock-service",
         (InternalDistributedSystem) cache.getDistributedSystem(), false, true, true);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/management/internal/security/MBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/MBeanSecurityJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/MBeanSecurityJUnitTest.java
index 9561111..f97cf2b 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/MBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/MBeanSecurityJUnitTest.java
@@ -14,12 +14,28 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.assertj.core.api.Assertions.*;
-import static org.mockito.Mockito.*;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+
+import org.apache.geode.management.ManagementException;
+import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.MemberMXBean;
+import org.apache.geode.management.internal.MBeanJMXAdapter;
+import org.apache.geode.security.TestSecurityManager;
+import org.apache.geode.test.dunit.rules.ConnectionConfiguration;
+import org.apache.geode.test.dunit.rules.MBeanServerConnectionRule;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import java.io.IOException;
 import java.util.Set;
-
 import javax.management.DynamicMBean;
 import javax.management.MBeanServer;
 import javax.management.MBeanServerConnection;
@@ -28,32 +44,18 @@ import javax.management.ObjectInstance;
 import javax.management.ObjectName;
 import javax.management.ReflectionException;
 
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.internal.AvailablePort;
-import org.apache.geode.management.ManagementException;
-import org.apache.geode.management.ManagementService;
-import org.apache.geode.management.MemberMXBean;
-import org.apache.geode.management.internal.MBeanJMXAdapter;
-import org.apache.geode.test.dunit.rules.ConnectionConfiguration;
-import org.apache.geode.test.dunit.rules.MBeanServerConnectionRule;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.apache.geode.test.junit.categories.SecurityTest;
-
 @Category({IntegrationTest.class, SecurityTest.class})
 public class MBeanSecurityJUnitTest {
-
-  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
   @ClassRule
-  public static CacheServerStartupRule serverRule =
-      CacheServerStartupRule.withDefaultSecurityJson(jmxManagerPort);
+  public static ServerStarterRule server = new ServerStarterRule().withJMXManager()
+      .withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+      .withProperty(TestSecurityManager.SECURITY_JSON,
+          "org/apache/geode/management/internal/security/cacheServer.json")
+      .startServer();
 
   @Rule
-  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+  public MBeanServerConnectionRule connectionRule =
+      new MBeanServerConnectionRule(server.getJmxPort());
 
   /**
    * No user can call createBean or unregisterBean of GemFire Domain
@@ -112,7 +114,7 @@ public class MBeanSecurityJUnitTest {
   @ConnectionConfiguration(user = "stranger", password = "1234567")
   public void testServerSideCalls() {
     // calls through ManagementService is not going through authorization checks
-    ManagementService service = ManagementService.getManagementService(serverRule.getCache());
+    ManagementService service = ManagementService.getManagementService(server.getCache());
     MemberMXBean bean = service.getMemberMXBean();
     bean.compactAllDiskStores();
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/management/internal/security/ManagerMBeanAuthorizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/ManagerMBeanAuthorizationJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/ManagerMBeanAuthorizationJUnitTest.java
index e9e29fd..9be9e02 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/ManagerMBeanAuthorizationJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/ManagerMBeanAuthorizationJUnitTest.java
@@ -14,13 +14,18 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.assertj.core.api.Assertions.*;
-import static org.mockito.Mockito.*;
-
-import java.lang.management.ManagementFactory;
-
-import javax.management.ObjectName;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
 
+import org.apache.geode.management.ManagerMXBean;
+import org.apache.geode.management.internal.beans.ManagerMBean;
+import org.apache.geode.security.TestSecurityManager;
+import org.apache.geode.test.dunit.rules.ConnectionConfiguration;
+import org.apache.geode.test.dunit.rules.MBeanServerConnectionRule;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -28,27 +33,23 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.internal.AvailablePort;
-import org.apache.geode.management.ManagerMXBean;
-import org.apache.geode.management.internal.beans.ManagerMBean;
-import org.apache.geode.test.dunit.rules.ConnectionConfiguration;
-import org.apache.geode.test.dunit.rules.MBeanServerConnectionRule;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.apache.geode.test.junit.categories.SecurityTest;
+import java.lang.management.ManagementFactory;
+import javax.management.ObjectName;
 
 @Category({IntegrationTest.class, SecurityTest.class})
 public class ManagerMBeanAuthorizationJUnitTest {
-
-  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
   private ManagerMXBean managerMXBean;
 
   @ClassRule
-  public static CacheServerStartupRule serverRule =
-      CacheServerStartupRule.withDefaultSecurityJson(jmxManagerPort);
+  public static ServerStarterRule server = new ServerStarterRule().withJMXManager()
+      .withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+      .withProperty(TestSecurityManager.SECURITY_JSON,
+          "org/apache/geode/management/internal/security/cacheServer.json")
+      .startServer();
 
   @Rule
-  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+  public MBeanServerConnectionRule connectionRule =
+      new MBeanServerConnectionRule(server.getJmxPort());
 
   @BeforeClass
   public static void beforeClassSetup() throws Exception {

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/management/internal/security/MemberMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/MemberMBeanSecurityJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/MemberMBeanSecurityJUnitTest.java
index 002900c..2ee03e3 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/MemberMBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/MemberMBeanSecurityJUnitTest.java
@@ -14,13 +14,14 @@
  */
 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 org.apache.geode.internal.AvailablePort;
 import org.apache.geode.management.MemberMXBean;
+import org.apache.geode.security.TestSecurityManager;
 import org.apache.geode.test.dunit.rules.ConnectionConfiguration;
 import org.apache.geode.test.dunit.rules.MBeanServerConnectionRule;
-import org.apache.geode.test.junit.categories.FlakyTest;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
 import org.junit.Before;
@@ -28,22 +29,20 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-/**
- * GEODE-2496: Flaky -- possibly due to use of AvailablePort
- */
-@Category({IntegrationTest.class, SecurityTest.class, FlakyTest.class})
+@Category({IntegrationTest.class, SecurityTest.class})
 public class MemberMBeanSecurityJUnitTest {
-
-  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
   private MemberMXBean bean;
 
-  @Rule
-  public CacheServerStartupRule serverRule =
-      CacheServerStartupRule.withDefaultSecurityJson(jmxManagerPort);
+  @Rule // do not use a ClassRule since some test will do a shutdownMember
+  public ServerStarterRule server = new ServerStarterRule().withJMXManager()
+      .withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+      .withProperty(TestSecurityManager.SECURITY_JSON,
+          "org/apache/geode/management/internal/security/cacheServer.json")
+      .startServer();
 
   @Rule
-  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+  public MBeanServerConnectionRule connectionRule =
+      new MBeanServerConnectionRule(server.getJmxPort());
 
   @Before
   public void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/AbstractSecureServerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/AbstractSecureServerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/AbstractSecureServerDUnitTest.java
deleted file mode 100644
index f8d90db..0000000
--- a/geode-core/src/test/java/org/apache/geode/security/AbstractSecureServerDUnitTest.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.LOCATORS;
-import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
-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 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.security.templates.UserPasswordAuthInit;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.IgnoredException;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.ServerStarterRule;
-import org.assertj.core.api.ThrowableAssert.ThrowingCallable;
-import org.junit.Before;
-import org.junit.Rule;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-
-public abstract class AbstractSecureServerDUnitTest extends JUnit4DistributedTestCase {
-
-  protected static final String REGION_NAME = "AuthRegion";
-
-  protected VM client1 = null;
-  protected VM client2 = null;
-  protected VM client3 = null;
-  protected int serverPort;
-  protected boolean pdxPersistent = false;
-
-  @Rule
-  public transient ServerStarterRule serverStarter =
-      new ServerStarterRule().startServer(getProperties(), 0, pdxPersistent);
-
-  // overwrite this in child classes
-  public Properties getProperties() {
-    return new Properties() {
-      {
-        setProperty(SECURITY_MANAGER, TestSecurityManager.class.getName());
-        setProperty(TestSecurityManager.SECURITY_JSON,
-            "org/apache/geode/management/internal/security/clientServer.json");
-      }
-    };
-  }
-
-  // overwrite this if you want a different set of initial data
-  public Map<String, String> getData() {
-    Map<String, String> data = new HashMap();
-    for (int i = 0; i < 5; i++) {
-      data.put("key" + i, "value" + i);
-    }
-    return data;
-  }
-
-  @Before
-  public void before() throws Exception {
-    serverPort = serverStarter.getServer().getPort();
-    Region region =
-        serverStarter.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
-    for (Entry entry : getData().entrySet()) {
-      region.put(entry.getKey(), entry.getValue());
-    }
-
-    IgnoredException.addIgnoredException("No longer connected to localhost");
-    IgnoredException.addIgnoredException(AuthenticationFailedException.class.getName());
-    final Host host = Host.getHost(0);
-    this.client1 = host.getVM(1);
-    this.client2 = host.getVM(2);
-    this.client3 = host.getVM(3);
-  }
-
-  public static void assertNotAuthorized(ThrowingCallable shouldRaiseThrowable, String permString) {
-    assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
-  }
-
-  public static Properties createClientProperties(String userName, String password) {
-    Properties props = new Properties();
-    props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
-    props.setProperty(UserPasswordAuthInit.PASSWORD, password);
-    props.setProperty(LOG_LEVEL, "fine");
-    props.setProperty(LOCATORS, "");
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
-    return props;
-  }
-
-  public static ClientCache createClientCache(String username, String password, int serverPort) {
-    ClientCache cache = new ClientCacheFactory(createClientProperties(username, password))
-        .setPoolSubscriptionEnabled(true).addPoolServer("localhost", serverPort).create();
-
-    cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
-    return cache;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientAuthDUnitTest.java
new file mode 100644
index 0000000..27b002b
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientAuthDUnitTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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_MANAGER;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server = new ServerStarterRule()
+      .withProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName()).startServer();
+
+  @Test
+  public void authWithCorrectPasswordShouldPass() {
+    client1.invoke("logging in super-user with correct password", () -> {
+      SecurityTestUtil.createClientCache("test", "test", server.getPort());
+    });
+  }
+
+  @Test
+  public void authWithIncorrectPasswordShouldFail() {
+    IgnoredException.addIgnoredException(AuthenticationFailedException.class.getName());
+    client2.invoke("logging in super-user with wrong password", () -> {
+      assertThatThrownBy(
+          () -> SecurityTestUtil.createClientCache("test", "wrong", server.getPort()))
+              .isInstanceOf(AuthenticationFailedException.class);
+    });
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientContainsKeyAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientContainsKeyAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientContainsKeyAuthDUnitTest.java
new file mode 100644
index 0000000..f2398bf
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientContainsKeyAuthDUnitTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientContainsKeyAuthDUnitTest extends JUnit4DistributedTestCase {
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer();
+
+  @Before
+  public void before() throws Exception {
+    Region region =
+        server.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+    for (int i = 0; i < 5; i++) {
+      region.put("key" + i, "value" + i);
+    }
+  }
+
+  @Test
+  public void testContainsKey() throws Exception {
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache = createClientCache("key1User", "1234567", server.getPort());
+      final Region region = createProxyRegion(cache, REGION_NAME);
+      assertTrue(region.containsKeyOnServer("key1"));
+      SecurityTestUtil.assertNotAuthorized(() -> region.containsKeyOnServer("key3"),
+          "DATA:READ:AuthRegion:key3");
+    });
+
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionReader", "1234567", server.getPort());
+      final Region region = createProxyRegion(cache, REGION_NAME);
+      region.containsKeyOnServer("key3");
+      assertTrue(region.containsKeyOnServer("key1"));
+    });
+
+    ai1.await();
+    ai2.await();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java
new file mode 100644
index 0000000..a25900e
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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_MANAGER;
+import static org.junit.Assert.*;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientDestroyInvalidateAuthDUnitTest extends JUnit4DistributedTestCase {
+  private static String REGION_NAME = "testRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer();
+
+  @Before
+  public void before() throws Exception {
+    Region region =
+        server.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+    for (int i = 0; i < 5; i++) {
+      region.put("key" + i, "value" + i);
+    }
+  }
+
+  @Test
+  public void testDestroyInvalidate() throws Exception {
+
+    // Delete one key and invalidate another key with an authorized user.
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache =
+          SecurityTestUtil.createClientCache("dataUser", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      assertTrue(region.containsKeyOnServer("key1"));
+
+      // Destroy key1
+      region.destroy("key1");
+      assertFalse(region.containsKeyOnServer("key1"));
+
+      // Invalidate key2
+      assertNotNull("Value of key2 should not be null", region.get("key2"));
+      region.invalidate("key2");
+      assertNull("Value of key2 should have been null", region.get("key2"));
+
+    });
+
+    // Delete one key and invalidate another key with an unauthorized user.
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache =
+          SecurityTestUtil.createClientCache("authRegionReader", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+
+      assertTrue(region.containsKeyOnServer("key3"));
+
+      // Destroy key1
+      SecurityTestUtil.assertNotAuthorized(() -> region.destroy("key3"), "DATA:WRITE:AuthRegion");
+      assertTrue(region.containsKeyOnServer("key3"));
+
+      // Invalidate key2
+      assertNotNull("Value of key4 should not be null", region.get("key4"));
+      SecurityTestUtil.assertNotAuthorized(() -> region.invalidate("key4"),
+          "DATA:WRITE:AuthRegion");
+      assertNotNull("Value of key4 should not be null", region.get("key4"));
+    });
+
+    ai1.await();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientDestroyRegionAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientDestroyRegionAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientDestroyRegionAuthDUnitTest.java
new file mode 100644
index 0000000..33c8d62
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientDestroyRegionAuthDUnitTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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_MANAGER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+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.ClientRegionShortcut;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientDestroyRegionAuthDUnitTest extends JUnit4DistributedTestCase {
+  private static String REGION_NAME = "testRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+  final VM client3 = host.getVM(3);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testDestroyRegion() throws InterruptedException {
+    client1.invoke(() -> {
+      ClientCache cache =
+          SecurityTestUtil.createClientCache("dataWriter", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      SecurityTestUtil.assertNotAuthorized(() -> region.destroyRegion(), "DATA:MANAGE");
+    });
+
+    client2.invoke(() -> {
+      ClientCache cache =
+          SecurityTestUtil.createClientCache("authRegionManager", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      SecurityTestUtil.assertNotAuthorized(() -> region.destroyRegion(), "DATA:MANAGE");
+    });
+
+    client3.invoke(() -> {
+      ClientCache cache =
+          SecurityTestUtil.createClientCache("super-user", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      region.destroyRegion();
+      assertThat(region.isDestroyed()).isTrue();
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java
new file mode 100644
index 0000000..adc66d0
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.internal.cache.functions.TestFunction;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientExecuteFunctionAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "testRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  private final static Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testExecuteRegionFunctionWithClientRegistration() {
+
+    FunctionService.registerFunction(function);
+    client1.invoke("logging in with dataReader", () -> {
+      ClientCache cache = createClientCache("dataReader", "1234567", server.getPort());
+
+      FunctionService.registerFunction(function);
+
+      assertNotAuthorized(() -> FunctionService.onServer(cache.getDefaultPool())
+          .withArgs(Boolean.TRUE).execute(function.getId()), "DATA:WRITE");
+    });
+
+    client2.invoke("logging in with super-user", () -> {
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
+
+      FunctionService.registerFunction(function);
+      ResultCollector rc = FunctionService.onServer(cache.getDefaultPool()).withArgs(Boolean.TRUE)
+          .execute(function.getId());
+      rc.getResult();
+    });
+  }
+
+  @Test
+  // this would trigger the client to send a GetFunctionAttribute command before executing it
+  public void testExecuteRegionFunctionWithOutClientRegistration() {
+    FunctionService.registerFunction(function);
+    client1.invoke("logging in with dataReader", () -> {
+      ClientCache cache = createClientCache("dataReader", "1234567", server.getPort());
+      assertNotAuthorized(() -> FunctionService.onServer(cache.getDefaultPool())
+          .withArgs(Boolean.TRUE).execute(function.getId()), "DATA:WRITE");
+    });
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientExecuteRegionFunctionAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientExecuteRegionFunctionAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientExecuteRegionFunctionAuthDUnitTest.java
new file mode 100644
index 0000000..1f88161
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientExecuteRegionFunctionAuthDUnitTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.internal.cache.functions.TestFunction;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientExecuteRegionFunctionAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  private final static Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testExecuteRegionFunction() {
+
+    FunctionService.registerFunction(function);
+
+    client1.invoke("logging in with dataReader", () -> {
+      ClientCache cache = createClientCache("dataReader", "1234567", server.getPort());
+
+      Region region = createProxyRegion(cache, REGION_NAME);
+      FunctionService.registerFunction(function);
+      assertNotAuthorized(
+          () -> FunctionService.onRegion(region).withArgs(Boolean.TRUE).execute(function.getId()),
+          "DATA:WRITE");
+    });
+
+    client2.invoke("logging in with super-user", () -> {
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
+
+      Region region = createProxyRegion(cache, REGION_NAME);
+      FunctionService.registerFunction(function);
+      ResultCollector rc =
+          FunctionService.onRegion(region).withArgs(Boolean.TRUE).execute(function.getId());
+      rc.getResult();
+    });
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientGetAllAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientGetAllAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientGetAllAuthDUnitTest.java
new file mode 100644
index 0000000..8963ee2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientGetAllAuthDUnitTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.internal.Assert.assertTrue;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+import static org.jgroups.util.Util.assertEquals;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Arrays;
+import java.util.Map;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientGetAllAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testGetAll() {
+    client1.invoke("logging in Stranger", () -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+
+      Region region = createProxyRegion(cache, REGION_NAME);
+      Map emptyMap = region.getAll(Arrays.asList("key1", "key2", "key3", "key4"));
+      assertTrue(emptyMap.isEmpty());
+    });
+
+    client2.invoke("logging in authRegionReader", () -> {
+      ClientCache cache = createClientCache("authRegionReader", "1234567", server.getPort());
+
+      Region region = createProxyRegion(cache, REGION_NAME);
+      Map filledMap = region.getAll(Arrays.asList("key1", "key2", "key3", "key4"));
+      assertEquals("Map should contain 4 entries", 4, filledMap.size());
+      assertTrue(filledMap.containsKey("key1"));
+    });
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientGetEntryAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientGetEntryAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientGetEntryAuthDUnitTest.java
new file mode 100644
index 0000000..0f41cc6
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientGetEntryAuthDUnitTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+
+import org.apache.geode.cache.CacheTransactionManager;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientGetEntryAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer();
+
+  @Before
+  public void before() throws Exception {
+    Region region =
+        server.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+    for (int i = 0; i < 5; i++) {
+      region.put("key" + i, "value" + i);
+    }
+  }
+
+  @Test
+  public void testGetEntry() throws Exception {
+    // client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+
+      CacheTransactionManager transactionManager = cache.getCacheTransactionManager();
+      transactionManager.begin();
+      try {
+        Region region = createProxyRegion(cache, REGION_NAME);
+        assertNotAuthorized(() -> region.getEntry("key3"), "DATA:READ:AuthRegion:key3");
+      } finally {
+        transactionManager.commit();
+      }
+
+    });
+
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionReader", "1234567", server.getPort());
+
+      CacheTransactionManager transactionManager = cache.getCacheTransactionManager();
+      transactionManager.begin();
+      try {
+        Region region = createProxyRegion(cache, REGION_NAME);
+        region.getEntry("key3");
+      } finally {
+        transactionManager.commit();
+      }
+
+    });
+
+    ai1.await();
+    ai2.await();
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientGetPutAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientGetPutAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientGetPutAuthDUnitTest.java
new file mode 100644
index 0000000..8f63301
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientGetPutAuthDUnitTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientGetPutAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+  final VM client3 = host.getVM(3);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer();
+
+  @Before
+  public void before() throws Exception {
+    Region region =
+        server.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+    for (int i = 0; i < 5; i++) {
+      region.put("key" + i, "value" + i);
+    }
+  }
+
+  @Test
+  public void testGetPutAuthorization() throws InterruptedException {
+    Map<String, String> allValues = new HashMap<String, String>();
+    allValues.put("key1", "value1");
+    allValues.put("key2", "value2");
+
+    List<String> keys = new ArrayList<>();
+    keys.add("key1");
+    keys.add("key2");
+
+    // client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+
+      assertNotAuthorized(() -> region.put("key3", "value3"), "DATA:WRITE:AuthRegion:key3");
+      assertNotAuthorized(() -> region.get("key3"), "DATA:READ:AuthRegion:key3");
+
+      // putall
+      assertNotAuthorized(() -> region.putAll(allValues), "DATA:WRITE:AuthRegion");
+
+      // not authorized for either keys, get no record back
+      Map keyValues = region.getAll(keys);
+      assertEquals(0, keyValues.size());
+
+      assertNotAuthorized(() -> region.keySetOnServer(), "DATA:READ:AuthRegion");
+    });
+
+
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+
+      region.put("key3", "value3");
+      assertEquals("value3", region.get("key3"));
+
+      // put all
+      region.putAll(allValues);
+
+      // get all
+      Map keyValues = region.getAll(keys);
+      assertEquals(2, keyValues.size());
+
+      // keyset
+      Set keySet = region.keySetOnServer();
+      assertEquals(5, keySet.size());
+    });
+
+    // client3 connects to user as a user authorized to use key1 in AuthRegion region
+    AsyncInvocation ai3 = client3.invokeAsync(() -> {
+      ClientCache cache = createClientCache("key1User", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+
+      assertNotAuthorized(() -> region.put("key2", "value1"), "DATA:WRITE:AuthRegion:key2");
+      assertNotAuthorized(() -> region.get("key2"), "DATA:READ:AuthRegion:key2");
+
+      assertNotAuthorized(() -> region.putAll(allValues), "DATA:WRITE:AuthRegion");
+
+      // only authorized for one recrod
+      Map keyValues = region.getAll(keys);
+      assertEquals(1, keyValues.size());
+
+      // keyset
+      assertNotAuthorized(() -> region.keySetOnServer(), "DATA:READ:AuthRegion");
+    });
+
+    ai1.join();
+    ai2.join();
+    ai3.join();
+
+    ai1.checkException();
+    ai2.checkException();
+    ai3.checkException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientRegionClearAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientRegionClearAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientRegionClearAuthDUnitTest.java
new file mode 100644
index 0000000..7eae441
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientRegionClearAuthDUnitTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.SerializableRunnable;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientRegionClearAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testRegionClear() throws InterruptedException {
+    // Verify that an unauthorized user can't clear the region
+    SerializableRunnable clearUnauthorized = new SerializableRunnable() {
+      @Override
+      public void run() {
+        ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+        Region region = createProxyRegion(cache, REGION_NAME);
+        assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
+      }
+    };
+    client1.invoke(clearUnauthorized);
+
+    // Verify that an authorized user can clear the region
+    SerializableRunnable clearAuthorized = new SerializableRunnable() {
+      @Override
+      public void run() {
+        ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+        Region region = createProxyRegion(cache, REGION_NAME);
+        region.clear();
+      }
+    };
+    client2.invoke(clearAuthorized);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientRegisterInterestAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientRegisterInterestAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientRegisterInterestAuthDUnitTest.java
new file mode 100644
index 0000000..fcd2697
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientRegisterInterestAuthDUnitTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+
+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.ClientRegionShortcut;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientRegisterInterestAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+  final VM client3 = host.getVM(3);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testRegisterInterest() throws Exception {
+    // client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+      assertNotAuthorized(() -> region.registerInterest("key3"), "DATA:READ:AuthRegion:key3");
+    });
+
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+      region.registerInterest("key3"); // DATA:READ:AuthRegion:key3;
+    });
+
+    // client3 connects to user as a user authorized to use key1 in AuthRegion region
+    AsyncInvocation ai3 = client3.invokeAsync(() -> {
+      ClientCache cache = createClientCache("key1User", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+      assertNotAuthorized(() -> region.registerInterest("key2"), "DATA:READ:AuthRegion:key2");
+    });
+
+    ai1.await();
+    ai2.await();
+    ai3.await();
+  }
+
+  @Test
+  public void testRegisterInterestRegex() throws Exception {
+    // client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      assertNotAuthorized(() -> region.registerInterestRegex("key.*"), "DATA:READ:AuthRegion");
+    });
+
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      region.registerInterestRegex("key[0-9]+"); // DATA:READ:AuthRegion:key3;
+    });
+
+    // client3 connects to user as a user authorized to use key1 in AuthRegion region
+    AsyncInvocation ai3 = client3.invokeAsync(() -> {
+      ClientCache cache = createClientCache("key1User", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      assertNotAuthorized(() -> region.registerInterestRegex("key[0-9]+"), "DATA:READ:AuthRegion");
+      assertNotAuthorized(() -> region.registerInterestRegex("key1"), "DATA:READ:AuthRegion");
+    });
+
+    ai1.await();
+    ai2.await();
+    ai3.await();
+  }
+
+  @Test
+  public void testRegisterInterestList() throws Exception {
+    List<String> keys = new ArrayList<>();
+    keys.add("key1");
+    keys.add("key2");
+
+    // client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      assertNotAuthorized(() -> region.registerInterest(keys), "DATA:READ:AuthRegion");
+    });
+
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      region.registerInterest(keys); // DATA:READ:AuthRegion;
+    });
+
+    // client3 connects to user as a user authorized to use key1 in AuthRegion region
+    AsyncInvocation ai3 = client3.invokeAsync(() -> {
+      ClientCache cache = createClientCache("key1User", "1234567", server.getPort());
+
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      assertNotAuthorized(() -> region.registerInterest(keys), "DATA:READ:AuthRegion");
+    });
+
+    ai1.await();
+    ai2.await();
+    ai3.await();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientRemoveAllAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientRemoveAllAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientRemoveAllAuthDUnitTest.java
new file mode 100644
index 0000000..b649d7f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientRemoveAllAuthDUnitTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+import static org.junit.Assert.assertFalse;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Arrays;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientRemoveAllAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testRemoveAll() throws Exception {
+
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionReader", "1234567", server.getPort());
+
+      Region region = createProxyRegion(cache, REGION_NAME);
+      assertNotAuthorized(() -> region.removeAll(Arrays.asList("key1", "key2", "key3", "key4")),
+          "DATA:WRITE:AuthRegion");
+    });
+
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionWriter", "1234567", server.getPort());
+
+      Region region = createProxyRegion(cache, REGION_NAME);
+      region.removeAll(Arrays.asList("key1", "key2", "key3", "key4"));
+      assertFalse(region.containsKey("key1"));
+      assertNotAuthorized(() -> region.containsKeyOnServer("key1"), "DATA:READ:AuthRegion:key1");
+    });
+    ai1.await();
+    ai2.await();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/ClientUnregisterInterestAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClientUnregisterInterestAuthDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClientUnregisterInterestAuthDUnitTest.java
new file mode 100644
index 0000000..4802c80
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/ClientUnregisterInterestAuthDUnitTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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_MANAGER;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientUnregisterInterestAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testUnregisterInterest() throws Exception {
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai1 = client2.invokeAsync(() -> {
+      ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+
+      Region region = createProxyRegion(cache, REGION_NAME);
+      region.registerInterest("key3");
+      region.unregisterInterest("key3"); // DATA:READ:AuthRegion:key3;
+    });
+    ai1.await();
+  }
+}