You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2016/03/14 17:04:09 UTC

incubator-geode git commit: GEODE-17: Adding authentication-only test

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 0d98d2145 -> a646879a6


GEODE-17: Adding authentication-only test

- Make MBeanServerConnectionRule more generic
- Rename JsonAuthorizationMBeanServerStart Rule (it does nothing with
  mbeans directly)


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/a646879a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/a646879a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/a646879a

Branch: refs/heads/feature/GEODE-17-2
Commit: a646879a67dd4de41bef7d1d8992fd1bfc2a8974
Parents: 0d98d21
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Mar 14 09:01:06 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Mar 14 09:03:05 2016 -0700

----------------------------------------------------------------------
 ...CacheServerMBeanAuthenticationJUnitTest.java | 66 +++++++++++++++
 .../CacheServerMBeanAuthorizationJUnitTest.java | 81 +++++++++++++++++++
 .../CacheServerMBeanSecurityJUnitTest.java      | 85 --------------------
 .../JsonAuthorizationCacheStartRule.java        | 75 +++++++++++++++++
 .../JsonAuthorizationMBeanServerStartRule.java  | 65 ---------------
 .../security/MBeanSecurityJUnitTest.java        |  2 +-
 .../security/MBeanServerConnectionRule.java     |  4 +-
 .../security/MemberMBeanSecurityJUnitTest.java  |  4 +-
 8 files changed, 226 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java
new file mode 100644
index 0000000..cf70f43
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.security;
+
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.management.CacheServerMXBean;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class CacheServerMBeanAuthenticationJUnitTest {
+  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+
+  private CacheServerMXBean cacheServerMXBean;
+
+  @ClassRule
+  public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(
+      jmxManagerPort, "cacheServer.json", false);
+
+  @Rule
+  public MBeanServerConnectionRule<CacheServerMXBean> connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+
+  @Before
+  public void setUp() throws Exception {
+    cacheServerMXBean = connectionRule.getProxyMBean(CacheServerMXBean.class, "GemFire:service=CacheServer,*");
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
+  public void testAllAccess() throws Exception {
+    cacheServerMXBean.removeIndex("foo"); // "INDEX:DESTROY",
+    cacheServerMXBean.executeContinuousQuery("bar"); // CONTNUOUS_QUERY:EXECUTE
+    cacheServerMXBean.fetchLoadProbe(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.getActiveCQCount(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.stopContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    cacheServerMXBean.closeAllContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    cacheServerMXBean.isRunning(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.showClientQueueDetails("foo"); // DISTRIBUTED_SYSTEM:LIST_DS
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "user", password = "1234567")
+  public void testSomeAccess() throws Exception {
+    cacheServerMXBean.removeIndex("foo");
+    cacheServerMXBean.executeContinuousQuery("bar");
+    cacheServerMXBean.fetchLoadProbe();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java
new file mode 100644
index 0000000..3f2b01a
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.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 com.gemstone.gemfire.management.internal.security;
+
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.management.CacheServerMXBean;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+@Category(IntegrationTest.class)
+public class CacheServerMBeanAuthorizationJUnitTest {
+  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+
+  private CacheServerMXBean cacheServerMXBean;
+
+  @ClassRule
+  public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(
+      jmxManagerPort, "cacheServer.json");
+
+  @Rule
+  public MBeanServerConnectionRule<CacheServerMXBean> connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+
+  @Before
+  public void setUp() throws Exception {
+    cacheServerMXBean = connectionRule.getProxyMBean(CacheServerMXBean.class, "GemFire:service=CacheServer,*");
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
+  public void testAllAccess() throws Exception {
+    cacheServerMXBean.removeIndex("foo"); // "INDEX:DESTROY",
+    cacheServerMXBean.executeContinuousQuery("bar"); // CONTNUOUS_QUERY:EXECUTE
+    cacheServerMXBean.fetchLoadProbe(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.getActiveCQCount(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.stopContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    cacheServerMXBean.closeAllContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    cacheServerMXBean.isRunning(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.showClientQueueDetails("foo"); // DISTRIBUTED_SYSTEM:LIST_DS
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "user", password = "1234567")
+  public void testSomeAccess() throws Exception {
+    assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    cacheServerMXBean.fetchLoadProbe();
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
+  public void testNoAccess() throws Exception {
+    assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.fetchLoadProbe()).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.getActiveCQCount()).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.stopContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.closeAllContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.isRunning()).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.showClientQueueDetails("bar")).isInstanceOf(SecurityException.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
deleted file mode 100644
index 50bda62..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
+++ /dev/null
@@ -1,85 +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 com.gemstone.gemfire.management.internal.security;
-
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.management.CacheServerMXBean;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import javax.management.MBeanServerConnection;
-
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-@Category(IntegrationTest.class)
-public class CacheServerMBeanSecurityJUnitTest {
-  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
-  private CacheServerMXBean cacheServerMXBean;
-  private MBeanServerConnection con;
-
-  @ClassRule
-  public static JsonAuthorizationMBeanServerStartRule serverRule = new JsonAuthorizationMBeanServerStartRule(
-      jmxManagerPort, "cacheServer.json");
-
-  @Rule
-  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
-
-  @Before
-  public void setUp() throws Exception {
-    cacheServerMXBean = (CacheServerMXBean) connectionRule.getProxyMBean(CacheServerMXBean.class,
-        "GemFire:service=CacheServer,*");
-  }
-
-  @Test
-  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
-  public void testAllAccess() throws Exception {
-    cacheServerMXBean.removeIndex("foo"); // "INDEX:DESTROY",
-    cacheServerMXBean.executeContinuousQuery("bar"); // CONTNUOUS_QUERY:EXECUTE
-    cacheServerMXBean.fetchLoadProbe(); // DISTRIBUTED_SYSTEM:LIST_DS
-    cacheServerMXBean.getActiveCQCount(); // DISTRIBUTED_SYSTEM:LIST_DS
-    cacheServerMXBean.stopContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
-    cacheServerMXBean.closeAllContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
-    cacheServerMXBean.isRunning(); // DISTRIBUTED_SYSTEM:LIST_DS
-    cacheServerMXBean.showClientQueueDetails("foo"); // DISTRIBUTED_SYSTEM:LIST_DS
-  }
-
-  @Test
-  @JMXConnectionConfiguration(user = "user", password = "1234567")
-  public void testSomeAccess() throws Exception {
-    assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
-    cacheServerMXBean.fetchLoadProbe();
-  }
-
-  @Test
-  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
-  public void testNoAccess() throws Exception {
-    assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.fetchLoadProbe()).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.getActiveCQCount()).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.stopContinuousQuery("bar")).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.closeAllContinuousQuery("bar")).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.isRunning()).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.showClientQueueDetails("bar")).isInstanceOf(SecurityException.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationCacheStartRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationCacheStartRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationCacheStartRule.java
new file mode 100644
index 0000000..872c611
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationCacheStartRule.java
@@ -0,0 +1,75 @@
+/*
+ * 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 com.gemstone.gemfire.management.internal.security;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import org.junit.rules.ExternalResource;
+
+import java.util.Properties;
+
+public class JsonAuthorizationCacheStartRule extends ExternalResource {
+  private Cache cache;
+  private int jmxManagerPort;
+  private String jsonFile;
+  private boolean doAuthorization;
+
+  public JsonAuthorizationCacheStartRule(int jmxManagerPort, String jsonFile) {
+    this.jmxManagerPort = jmxManagerPort;
+    this.jsonFile = jsonFile;
+    this.doAuthorization = true;
+  }
+
+  public JsonAuthorizationCacheStartRule(int jmxManagerPort, String jsonFile, boolean doAuthorization) {
+    this.jmxManagerPort = jmxManagerPort;
+    this.jsonFile = jsonFile;
+    this.doAuthorization = doAuthorization;
+  }
+
+  protected void before() throws Throwable {
+    Properties properties = new Properties();
+    properties.put(DistributionConfig.NAME_NAME, JsonAuthorizationCacheStartRule.class.getSimpleName());
+    properties.put(DistributionConfig.LOCATORS_NAME, "");
+    properties.put(DistributionConfig.MCAST_PORT_NAME, "0");
+    properties.put(DistributionConfig.JMX_MANAGER_NAME, "true");
+    properties.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
+    properties.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxManagerPort));
+    properties.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
+    properties.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
+        JSONAuthorization.class.getName() + ".create");
+    if (doAuthorization) {
+      properties.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
+    }
+    JSONAuthorization.setUpWithJsonFile(jsonFile);
+
+    cache = new CacheFactory(properties).create();
+    cache.addCacheServer().start();
+  }
+
+  public Cache getCache(){
+    return cache;
+  }
+
+  /**
+   * Override to tear down your specific external resource.
+   */
+  protected void after() {
+    cache.close();
+    cache = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
deleted file mode 100644
index 1f4c592..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
+++ /dev/null
@@ -1,65 +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 com.gemstone.gemfire.management.internal.security;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import org.junit.rules.ExternalResource;
-
-import java.util.Properties;
-
-public class JsonAuthorizationMBeanServerStartRule extends ExternalResource {
-  private Cache cache;
-  private int jmxManagerPort;
-  private String jsonFile;
-
-  public JsonAuthorizationMBeanServerStartRule(int jmxManagerPort, String jsonFile) {
-    this.jmxManagerPort = jmxManagerPort;
-    this.jsonFile = jsonFile;
-  }
-
-  protected void before() throws Throwable {
-    Properties properties = new Properties();
-    properties.put(DistributionConfig.NAME_NAME, JsonAuthorizationMBeanServerStartRule.class.getSimpleName());
-    properties.put(DistributionConfig.LOCATORS_NAME, "");
-    properties.put(DistributionConfig.MCAST_PORT_NAME, "0");
-    properties.put(DistributionConfig.JMX_MANAGER_NAME, "true");
-    properties.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
-    properties.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxManagerPort));
-    properties.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
-    properties.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
-    properties.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
-        JSONAuthorization.class.getName() + ".create");
-    JSONAuthorization.setUpWithJsonFile(jsonFile);
-
-    cache = new CacheFactory(properties).create();
-    cache.addCacheServer().start();
-  }
-
-  public Cache getCache(){
-    return cache;
-  }
-
-  /**
-   * Override to tear down your specific external resource.
-   */
-  protected void after() {
-    cache.close();
-    cache = null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java
index d386af8..4b3b6c5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java
@@ -45,7 +45,7 @@ public class MBeanSecurityJUnitTest {
   private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
   @ClassRule
-  public static JsonAuthorizationMBeanServerStartRule serverRule = new JsonAuthorizationMBeanServerStartRule(jmxManagerPort, "cacheServer.json");
+  public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(jmxManagerPort, "cacheServer.json");
 
   @Rule
   public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
index 85c7cec..b634271 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
@@ -39,7 +39,7 @@ import java.util.Set;
  * it allows for the creation of per-test connections with different user/password combinations.
  *
  */
-public class MBeanServerConnectionRule extends DescribedExternalResource {
+public class MBeanServerConnectionRule<T> extends DescribedExternalResource {
 
   private final int jmxServerPort;
   private JMXConnector jmxConnector;
@@ -60,7 +60,7 @@ public class MBeanServerConnectionRule extends DescribedExternalResource {
    *
    * @return A new proxy MBean of the same type with which the class was constructed
    */
-  public Object getProxyMBean(Class proxyClass, String beanQueryName) throws MalformedObjectNameException, IOException {
+  public T getProxyMBean(Class<T> proxyClass, String beanQueryName) throws MalformedObjectNameException, IOException {
     ObjectName name = null;
     QueryExp query = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a646879a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
index 147dd9a..edb0bc2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
@@ -25,8 +25,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import javax.management.MBeanServerConnection;
-
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 @Category(IntegrationTest.class)
@@ -36,7 +34,7 @@ public class MemberMBeanSecurityJUnitTest {
   private MemberMXBean bean;
 
   @ClassRule
-  public static JsonAuthorizationMBeanServerStartRule serverRule = new JsonAuthorizationMBeanServerStartRule(
+  public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(
       jmxManagerPort, "cacheServer.json");
 
   @Rule