You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2021/04/26 16:50:18 UTC

[geode] branch support/1.12 updated: GEODE-9064: Configure serial filter during ManagementAgent start (#6196) (#6341)

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

klund pushed a commit to branch support/1.12
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/support/1.12 by this push:
     new 20feeff  GEODE-9064: Configure serial filter during ManagementAgent start (#6196) (#6341)
20feeff is described below

commit 20feeff9220a844ed5ff970ce83c203d0b5468ac
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Apr 26 09:47:17 2021 -0700

    GEODE-9064: Configure serial filter during ManagementAgent start (#6196) (#6341)
    
    The Geode JMX layer will configure the system property
    “jmx.remote.rmi.server.serial.filter.pattern" to only accept
    JDK classes identified as open-types required for JMX.
    
    If the system property already has a value then Geode will log
    a statement and leave the system property alone.
    
    (cherry picked from commit 8b2ee6d99ab16a11a129ccd3d4f3cc517997e8c8)
---
 ...gerConfiguresJmxSerialFilterAcceptanceTest.java | 115 +++++++
 ...gerConfiguresJmxSerialFilterAcceptanceTest.java | 111 +++++++
 .../apache/geode/util/internal/UncheckedUtils.java |  34 ++
 .../geode/util/internal/UncheckedUtilsTest.java    |  60 ++++
 ...erConfiguresJmxSerialFilterIntegrationTest.java | 204 ++++++++++++
 ...erConfiguresJmxSerialFilterIntegrationTest.java | 201 ++++++++++++
 .../internal/JmxRmiOpenTypesSerialFilter.java      | 148 +++++++++
 .../management/internal/JmxRmiSerialFilter.java    |  21 ++
 .../geode/management/internal/ManagementAgent.java |   7 +-
 .../internal/ManagementAgentFactory.java           |  25 ++
 .../internal/SystemManagementService.java          |  25 +-
 .../cache/client/internal/GetAllOpJUnitTest.java   |  10 +-
 .../internal/JmxRmiOpenTypesSerialFilterTest.java  | 351 +++++++++++++++++++++
 .../internal/SystemManagementServiceTest.java      |   6 +-
 geode-junit/build.gradle                           |   1 +
 .../geode/test/junit/rules/CloseableReference.java | 209 ++++++++++++
 .../test/junit/rules/CloseableReferenceTest.java   | 204 ++++++++++++
 17 files changed, 1712 insertions(+), 20 deletions(-)

diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/LocatorManagerConfiguresJmxSerialFilterAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/LocatorManagerConfiguresJmxSerialFilterAcceptanceTest.java
new file mode 100644
index 0000000..de0b8c8
--- /dev/null
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/LocatorManagerConfiguresJmxSerialFilterAcceptanceTest.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management;
+
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.management.internal.JmxRmiOpenTypesSerialFilter.PROPERTY_NAME;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.nio.file.Path;
+
+import org.apache.commons.lang3.JavaVersion;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.RequiresGeodeHome;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+
+public class LocatorManagerConfiguresJmxSerialFilterAcceptanceTest {
+
+  private static final String NAME = "the-locator";
+
+  private Path workingDir;
+  private int locatorPort;
+  private int jmxPort;
+  private Path locatorLogFile;
+
+  @Rule
+  public RequiresGeodeHome requiresGeodeHome = new RequiresGeodeHome();
+  @Rule
+  public GfshRule gfshRule = new GfshRule();
+
+  @Before
+  public void setUpOutputFiles() {
+    TemporaryFolder temporaryFolder = gfshRule.getTemporaryFolder();
+
+    workingDir = temporaryFolder.getRoot().toPath().toAbsolutePath();
+    locatorLogFile = workingDir.resolve(NAME + ".log");
+  }
+
+  @Before
+  public void setUpRandomPorts() {
+    int[] ports = getRandomAvailableTCPPorts(2);
+
+    locatorPort = ports[0];
+    jmxPort = ports[1];
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_configuresSerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    String startLocatorCommand = String.join(" ",
+        "start locator",
+        "--name=" + NAME,
+        "--dir=" + workingDir,
+        "--port=" + locatorPort,
+        "--J=-Dgemfire.enable-cluster-configuration=false",
+        "--J=-Dgemfire.http-service-port=0",
+        "--J=-Dgemfire.jmx-manager=true",
+        "--J=-Dgemfire.jmx-manager-port=" + jmxPort,
+        "--J=-Dgemfire.jmx-manager-start=true");
+
+    gfshRule.execute(startLocatorCommand);
+
+    await().untilAsserted(() -> {
+      LogFileAssert.assertThat(locatorLogFile.toFile())
+          .as(locatorLogFile.toFile().getAbsolutePath())
+          .exists()
+          .contains("System property " + PROPERTY_NAME + " is now configured with");
+    });
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_configuresSerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    String startLocatorCommand = String.join(" ",
+        "start locator",
+        "--name=" + NAME,
+        "--dir=" + workingDir,
+        "--port=" + locatorPort,
+        "--J=-Dgemfire.enable-cluster-configuration=false",
+        "--J=-Dgemfire.http-service-port=0",
+        "--J=-Dgemfire.jmx-manager=true",
+        "--J=-Dgemfire.jmx-manager-port=" + jmxPort,
+        "--J=-Dgemfire.jmx-manager-start=true");
+
+    gfshRule.execute(startLocatorCommand);
+
+    await().untilAsserted(() -> {
+      LogFileAssert.assertThat(locatorLogFile.toFile())
+          .as(locatorLogFile.toFile().getAbsolutePath())
+          .exists()
+          .doesNotContain("System property " + PROPERTY_NAME + " is now configured with");
+    });
+  }
+}
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/ServerManagerConfiguresJmxSerialFilterAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/ServerManagerConfiguresJmxSerialFilterAcceptanceTest.java
new file mode 100644
index 0000000..7754dec
--- /dev/null
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/ServerManagerConfiguresJmxSerialFilterAcceptanceTest.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management;
+
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.management.internal.JmxRmiOpenTypesSerialFilter.PROPERTY_NAME;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.nio.file.Path;
+
+import org.apache.commons.lang3.JavaVersion;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.RequiresGeodeHome;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+
+public class ServerManagerConfiguresJmxSerialFilterAcceptanceTest {
+
+  private static final String NAME = "the-server";
+
+  private Path workingDir;
+  private int jmxPort;
+  private Path serverLogFile;
+
+  @Rule
+  public RequiresGeodeHome requiresGeodeHome = new RequiresGeodeHome();
+  @Rule
+  public GfshRule gfshRule = new GfshRule();
+
+  @Before
+  public void setUpOutputFiles() {
+    TemporaryFolder temporaryFolder = gfshRule.getTemporaryFolder();
+
+    workingDir = temporaryFolder.getRoot().toPath().toAbsolutePath();
+    serverLogFile = workingDir.resolve(NAME + ".log");
+  }
+
+  @Before
+  public void setUpRandomPorts() {
+    jmxPort = getRandomAvailableTCPPort();
+  }
+
+  @Test
+  public void startingServerWithJmxManager_configuresSerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    String startServerCommand = String.join(" ",
+        "start server",
+        "--name=" + NAME,
+        "--dir=" + workingDir,
+        "--disable-default-server",
+        "--J=-Dgemfire.enable-cluster-configuration=false",
+        "--J=-Dgemfire.http-service-port=0",
+        "--J=-Dgemfire.jmx-manager=true",
+        "--J=-Dgemfire.jmx-manager-port=" + jmxPort,
+        "--J=-Dgemfire.jmx-manager-start=true");
+
+    gfshRule.execute(startServerCommand);
+
+    await().untilAsserted(() -> {
+      LogFileAssert.assertThat(serverLogFile.toFile())
+          .as(serverLogFile.toFile().getAbsolutePath())
+          .exists()
+          .contains("System property " + PROPERTY_NAME + " is now configured with");
+    });
+  }
+
+  @Test
+  public void startingServerWithJmxManager_configuresSerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    String startServerCommand = String.join(" ",
+        "start server",
+        "--name=" + NAME,
+        "--dir=" + workingDir,
+        "--disable-default-server",
+        "--J=-Dgemfire.enable-cluster-configuration=false",
+        "--J=-Dgemfire.http-service-port=0",
+        "--J=-Dgemfire.jmx-manager=true",
+        "--J=-Dgemfire.jmx-manager-port=" + jmxPort,
+        "--J=-Dgemfire.jmx-manager-start=true");
+
+    gfshRule.execute(startServerCommand);
+
+    await().untilAsserted(() -> {
+      LogFileAssert.assertThat(serverLogFile.toFile())
+          .as(serverLogFile.toFile().getAbsolutePath())
+          .exists()
+          .doesNotContain("System property " + PROPERTY_NAME + " is now configured with");
+    });
+  }
+}
diff --git a/geode-common/src/main/java/org/apache/geode/util/internal/UncheckedUtils.java b/geode-common/src/main/java/org/apache/geode/util/internal/UncheckedUtils.java
new file mode 100644
index 0000000..61dbd8d
--- /dev/null
+++ b/geode-common/src/main/java/org/apache/geode/util/internal/UncheckedUtils.java
@@ -0,0 +1,34 @@
+/*
+ * 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.util.internal;
+
+/**
+ * Utilities for casting and working with unchecked raw types.
+ */
+@SuppressWarnings({"unchecked", "unused"})
+public class UncheckedUtils {
+
+  protected UncheckedUtils() {
+    // do not instantiate
+  }
+
+  /**
+   * Casts an instance of a raw type to a parameterized type. Preference should be given to
+   * converting all code from using raw types to using parameterized types when possible.
+   */
+  public static <T> T uncheckedCast(Object object) {
+    return (T) object;
+  }
+}
diff --git a/geode-common/src/test/java/org/apache/geode/util/internal/UncheckedUtilsTest.java b/geode-common/src/test/java/org/apache/geode/util/internal/UncheckedUtilsTest.java
new file mode 100644
index 0000000..7c282b7
--- /dev/null
+++ b/geode-common/src/test/java/org/apache/geode/util/internal/UncheckedUtilsTest.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.util.internal;
+
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+@SuppressWarnings("unchecked")
+public class UncheckedUtilsTest {
+
+  @Test
+  public void uncheckedCast_rawList_empty() {
+    List rawList = new ArrayList();
+
+    List<String> value = uncheckedCast(rawList);
+
+    assertThat(value).isSameAs(rawList);
+  }
+
+  @Test
+  public void uncheckedCast_rawList_nonEmpty() {
+    List rawList = new ArrayList();
+    rawList.add("1");
+    rawList.add("2");
+
+    List<String> value = uncheckedCast(rawList);
+
+    assertThat(value).isSameAs(rawList);
+  }
+
+  @Test
+  public void uncheckedCast_rawList_wrongTypes() {
+    List rawList = new ArrayList();
+    rawList.add(1);
+    rawList.add(2);
+    List<String> wrongType = uncheckedCast(rawList);
+
+    Throwable thrown = catchThrowable(() -> wrongType.get(0));
+
+    assertThat(thrown).isInstanceOf(ClassCastException.class);
+  }
+}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/LocatorManagerConfiguresJmxSerialFilterIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/LocatorManagerConfiguresJmxSerialFilterIntegrationTest.java
new file mode 100644
index 0000000..47f196f
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/LocatorManagerConfiguresJmxSerialFilterIntegrationTest.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal;
+
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.management.internal.JmxRmiOpenTypesSerialFilter.PROPERTY_NAME;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+
+import org.apache.commons.lang3.JavaVersion;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorManagerConfiguresJmxSerialFilterIntegrationTest {
+
+  private static final String NAME = "locator";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+  private String expectedSerialFilter;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    locatorPort = ports[0];
+    jmxPort = ports[1];
+    expectedSerialFilter = new JmxRmiOpenTypesSerialFilter().createSerialFilterPattern();
+  }
+
+  @After
+  public void tearDown() {
+    System.clearProperty(PROPERTY_NAME);
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_configuresSerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(expectedSerialFilter);
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_changesEmptySerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    System.setProperty(PROPERTY_NAME, "");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(expectedSerialFilter);
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_skipsNonEmptySerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    String existingSerialFilter = "!*";
+    System.setProperty(PROPERTY_NAME, existingSerialFilter);
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(existingSerialFilter);
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_skipsSerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isNull();
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_skipsEmptySerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    System.setProperty(PROPERTY_NAME, "");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo("");
+  }
+
+  @Test
+  public void startingLocatorWithJmxManager_skipsNonEmptySerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    String existingSerialFilter = "!*";
+    System.setProperty(PROPERTY_NAME, existingSerialFilter);
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(existingSerialFilter);
+  }
+}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/ServerManagerConfiguresJmxSerialFilterIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/ServerManagerConfiguresJmxSerialFilterIntegrationTest.java
new file mode 100644
index 0000000..f77438e
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/ServerManagerConfiguresJmxSerialFilterIntegrationTest.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal;
+
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.management.internal.JmxRmiOpenTypesSerialFilter.PROPERTY_NAME;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+
+import org.apache.commons.lang3.JavaVersion;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class ServerManagerConfiguresJmxSerialFilterIntegrationTest {
+
+  private static final String NAME = "server";
+
+  private File workingDirectory;
+  private int jmxPort;
+  private String expectedSerialFilter;
+
+  @Rule
+  public CloseableReference<ServerLauncher> server = new CloseableReference<>();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    jmxPort = getRandomAvailableTCPPort();
+    expectedSerialFilter = new JmxRmiOpenTypesSerialFilter().createSerialFilterPattern();
+  }
+
+  @After
+  public void tearDown() {
+    System.clearProperty(PROPERTY_NAME);
+  }
+
+  @Test
+  public void startingServerWithJmxManager_configuresSerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    server.set(new ServerLauncher.Builder()
+        .setMemberName(NAME)
+        .setDisableDefaultServer(true)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(expectedSerialFilter);
+  }
+
+  @Test
+  public void startingServerWithJmxManager_changesEmptySerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    System.setProperty(PROPERTY_NAME, "");
+
+    server.set(new ServerLauncher.Builder()
+        .setMemberName(NAME)
+        .setDisableDefaultServer(true)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(expectedSerialFilter);
+  }
+
+  @Test
+  public void startingServerWithJmxManager_skipsNonEmptySerialFilter_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    String existingSerialFilter = "!*";
+    System.setProperty(PROPERTY_NAME, existingSerialFilter);
+
+    server.set(new ServerLauncher.Builder()
+        .setMemberName(NAME)
+        .setDisableDefaultServer(true)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(existingSerialFilter);
+  }
+
+  @Test
+  public void startingServerWithJmxManager_skipsSerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    server.set(new ServerLauncher.Builder()
+        .setMemberName(NAME)
+        .setDisableDefaultServer(true)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isNull();
+  }
+
+  @Test
+  public void startingServerWithJmxManager_skipsEmptySerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    System.setProperty(PROPERTY_NAME, "");
+
+    server.set(new ServerLauncher.Builder()
+        .setMemberName(NAME)
+        .setDisableDefaultServer(true)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo("");
+  }
+
+  @Test
+  public void startingServerWithJmxManager_skipsNonEmptySerialFilter_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    String existingSerialFilter = "!*";
+    System.setProperty(PROPERTY_NAME, existingSerialFilter);
+
+    server.set(new ServerLauncher.Builder()
+        .setMemberName(NAME)
+        .setDisableDefaultServer(true)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    String serialFilter = System.getProperty(PROPERTY_NAME);
+    assertThat(serialFilter).isEqualTo(existingSerialFilter);
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/JmxRmiOpenTypesSerialFilter.java b/geode-core/src/main/java/org/apache/geode/management/internal/JmxRmiOpenTypesSerialFilter.java
new file mode 100644
index 0000000..803bc03
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/JmxRmiOpenTypesSerialFilter.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal;
+
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.rmi.MarshalledObject;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import javax.management.ObjectName;
+import javax.management.openmbean.ArrayType;
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.CompositeType;
+import javax.management.openmbean.OpenType;
+import javax.management.openmbean.SimpleType;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularType;
+
+import org.apache.commons.lang3.JavaVersion;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.logging.internal.log4j.api.LogService;
+
+/**
+ * Configure the “jmx.remote.rmi.server.serial.filter.pattern” system property if Java version is
+ * greater than Java 8. The serial pattern will be configured to accept only standard JMX
+ * open-types. If the system property already has a non-null value, then leave it as is.
+ */
+public class JmxRmiOpenTypesSerialFilter implements JmxRmiSerialFilter {
+
+  private static final Logger logger = LogService.getLogger();
+
+  @VisibleForTesting
+  public static final String PROPERTY_NAME = "jmx.remote.rmi.server.serial.filter.pattern";
+
+  private final Consumer<String> infoLogger;
+  private final Supplier<Boolean> supportsSerialFilter;
+
+  JmxRmiOpenTypesSerialFilter() {
+    this(logger::info, () -> isJavaVersionAtLeast(JavaVersion.JAVA_9));
+  }
+
+  @VisibleForTesting
+  JmxRmiOpenTypesSerialFilter(Consumer<String> infoLogger, Supplier<Boolean> supportsSerialFilter) {
+    this.infoLogger = infoLogger;
+    this.supportsSerialFilter = supportsSerialFilter;
+  }
+
+  @Override
+  public void configureSerialFilter() {
+    if (supportsDedicatedSerialFilter()) {
+      setPropertyValueUnlessExists(createSerialFilterPattern());
+    }
+  }
+
+  @VisibleForTesting
+  boolean supportsDedicatedSerialFilter() {
+    return supportsSerialFilter.get();
+  }
+
+  /**
+   * Sets the value of the system property {@code jmx.remote.rmi.server.serial.filter.pattern}
+   * unless it exists with a value that is not null or empty ("").
+   */
+  @VisibleForTesting
+  void setPropertyValueUnlessExists(String value) {
+    String existingValue = System.getProperty(PROPERTY_NAME);
+    if (StringUtils.isNotEmpty(existingValue)) {
+      infoLogger.accept("System property " + PROPERTY_NAME + " is already configured.");
+      return;
+    }
+
+    System.setProperty(PROPERTY_NAME, value);
+    infoLogger.accept("System property " + PROPERTY_NAME + " is now configured with '"
+        + value + "'.");
+  }
+
+  /**
+   * Returns a serial filter pattern that accepts all open MBean data types and rejects everything
+   * not included in the pattern.
+   */
+  @VisibleForTesting
+  String createSerialFilterPattern() {
+    // note: java.util.* may also be needed
+    return new StringBuilder()
+        // accept all open MBean data types
+        .append(Boolean.class.getName())
+        .append(';')
+        .append(Byte.class.getName())
+        .append(';')
+        .append(Character.class.getName())
+        .append(';')
+        .append(Short.class.getName())
+        .append(';')
+        .append(Integer.class.getName())
+        .append(';')
+        .append(Long.class.getName())
+        .append(';')
+        .append(Float.class.getName())
+        .append(';')
+        .append(Double.class.getName())
+        .append(';')
+        .append(String.class.getName())
+        .append(';')
+        .append(BigInteger.class.getName())
+        .append(';')
+        .append(BigDecimal.class.getName())
+        .append(';')
+        .append(ObjectName.class.getName())
+        .append(';')
+        .append(OpenType.class.getName())
+        .append(';')
+        .append(CompositeData.class.getName())
+        .append(';')
+        .append(TabularData.class.getName())
+        .append(';')
+        .append(SimpleType.class.getName())
+        .append(';')
+        .append(CompositeType.class.getName())
+        .append(';')
+        .append(TabularType.class.getName())
+        .append(';')
+        .append(ArrayType.class.getName())
+        .append(';')
+        .append(MarshalledObject.class.getName())
+        .append(';')
+        // reject all other classes
+        .append("!*")
+        .toString();
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/JmxRmiSerialFilter.java b/geode-core/src/main/java/org/apache/geode/management/internal/JmxRmiSerialFilter.java
new file mode 100644
index 0000000..4a187e0
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/JmxRmiSerialFilter.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal;
+
+@FunctionalInterface
+public interface JmxRmiSerialFilter {
+
+  void configureSerialFilter();
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
index f8d5319..dbd96d6 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
@@ -105,6 +105,8 @@ public class ManagementAgent {
   private int port;
   private RemoteStreamExporter remoteStreamExporter = null;
 
+  private final JmxRmiSerialFilter serialFilter;
+
   /**
    * This system property is set to true when the embedded HTTP server is started so that the
    * embedded pulse webapp can use a local MBeanServer instead of a remote JMX connection.
@@ -115,10 +117,12 @@ public class ManagementAgent {
   private static final String PULSE_USESSL_MANAGER = "pulse.useSSL.manager";
   private static final String PULSE_USESSL_LOCATOR = "pulse.useSSL.locator";
 
-  public ManagementAgent(DistributionConfig config, InternalCache cache) {
+  public ManagementAgent(DistributionConfig config, InternalCache cache,
+      JmxRmiSerialFilter serialFilter) {
     this.config = config;
     this.cache = cache;
     this.securityService = cache.getSecurityService();
+    this.serialFilter = serialFilter;
   }
 
   public synchronized boolean isRunning() {
@@ -127,6 +131,7 @@ public class ManagementAgent {
 
 
   public synchronized void startAgent() {
+    serialFilter.configureSerialFilter();
     loadWebApplications();
 
     if (!this.running && this.config.getJmxManagerPort() != 0) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgentFactory.java b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgentFactory.java
new file mode 100644
index 0000000..bff205e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgentFactory.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal;
+
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.cache.InternalCache;
+
+@FunctionalInterface
+interface ManagementAgentFactory {
+
+  ManagementAgent create(DistributionConfig config, InternalCache cache,
+      JmxRmiSerialFilter serialFilter);
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
index 2e9403e..a5bf2f4 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.ExecutorService;
-import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.function.Supplier;
 
@@ -41,7 +40,6 @@ import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ResourceEvent;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -115,7 +113,7 @@ public class SystemManagementService extends BaseManagementService {
   private final StatisticsFactory statisticsFactory;
   private final StatisticsClock statisticsClock;
   private final FederatingManagerFactory federatingManagerFactory;
-
+  private final Function<SystemManagementService, LocalManager> localManagerFactory;
 
   /**
    * whether the service is closed or not if cache is closed automatically this service will be
@@ -137,30 +135,34 @@ public class SystemManagementService extends BaseManagementService {
    * Managing node.
    */
   private ManagementMembershipListener listener;
-  private final Function<SystemManagementService, LocalManager> localManagerFactory;
 
   static BaseManagementService newSystemManagementService(
       InternalCacheForClientAccess cache) {
-    return newSystemManagementService(cache, NotificationHub::new,
+    return newSystemManagementService(
+        cache,
+        NotificationHub::new,
         SystemManagementService::createLocalManager,
-        createFederatingManagerFactory(), ManagementAgent::new);
+        createFederatingManagerFactory(),
+        ManagementAgent::new);
   }
 
   @VisibleForTesting
-  static BaseManagementService newSystemManagementService(InternalCacheForClientAccess cache,
+  static BaseManagementService newSystemManagementService(
+      InternalCacheForClientAccess cache,
       Function<ManagementResourceRepo, NotificationHub> notificationHubFactory,
       Function<SystemManagementService, LocalManager> localManagerFactory,
       FederatingManagerFactory federatingManagerFactory,
-      BiFunction<DistributionConfig, InternalCacheForClientAccess, ManagementAgent> managementAgentFactory) {
+      ManagementAgentFactory managementAgentFactory) {
     return new SystemManagementService(cache, notificationHubFactory, localManagerFactory,
         federatingManagerFactory, managementAgentFactory).init();
   }
 
-  private SystemManagementService(InternalCacheForClientAccess cache,
+  private SystemManagementService(
+      InternalCacheForClientAccess cache,
       Function<ManagementResourceRepo, NotificationHub> notificationHubFactory,
       Function<SystemManagementService, LocalManager> localManagerFactory,
       FederatingManagerFactory federatingManagerFactory,
-      BiFunction<DistributionConfig, InternalCacheForClientAccess, ManagementAgent> managementAgentFactory) {
+      ManagementAgentFactory managementAgentFactory) {
     this.cache = cache;
     system = cache.getInternalDistributedSystem();
     this.localManagerFactory = localManagerFactory;
@@ -177,7 +179,8 @@ public class SystemManagementService extends BaseManagementService {
     notificationHub = notificationHubFactory.apply(repo);
 
     if (system.getConfig().getJmxManager()) {
-      agent = managementAgentFactory.apply(system.getConfig(), cache);
+      agent = managementAgentFactory.create(system.getConfig(), cache,
+          new JmxRmiOpenTypesSerialFilter());
     } else {
       agent = null;
     }
diff --git a/geode-core/src/test/java/org/apache/geode/cache/client/internal/GetAllOpJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/client/internal/GetAllOpJUnitTest.java
index f905225..0f80d16 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/client/internal/GetAllOpJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/client/internal/GetAllOpJUnitTest.java
@@ -40,10 +40,12 @@ import org.apache.geode.internal.cache.tier.sockets.VersionedObjectList;
 import org.apache.geode.test.fake.Fakes;
 
 public class GetAllOpJUnitTest {
-  private ExecutablePool pool = mock(PoolImpl.class);
-  private GemFireCacheImpl cache = Fakes.cache();
-  private LocalRegion region = mock(LocalRegion.class);
-  ArrayList<Integer> keys;
+
+  private final ExecutablePool pool = mock(PoolImpl.class);
+  private final GemFireCacheImpl cache = Fakes.cache();
+  private final LocalRegion region = mock(LocalRegion.class);
+
+  private ArrayList<Integer> keys;
 
   @Before
   public void setup() {
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/JmxRmiOpenTypesSerialFilterTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/JmxRmiOpenTypesSerialFilterTest.java
new file mode 100644
index 0000000..c8f8ea9
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/JmxRmiOpenTypesSerialFilterTest.java
@@ -0,0 +1,351 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal;
+
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.rmi.MarshalledObject;
+import java.util.function.Consumer;
+
+import javax.management.ObjectName;
+import javax.management.openmbean.ArrayType;
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.CompositeType;
+import javax.management.openmbean.SimpleType;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularType;
+
+import org.apache.commons.lang3.JavaVersion;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+
+public class JmxRmiOpenTypesSerialFilterTest {
+
+  private Consumer<String> infoLogger;
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  @Before
+  public void setUp() {
+    infoLogger = uncheckedCast(mock(Consumer.class));
+  }
+
+  @Test
+  public void propertyValueIsNullByDefault() {
+    String propertyValue = System.getProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME);
+
+    assertThat(propertyValue).isNull();
+  }
+
+  @Test
+  public void supportsDedicatedSerialFilter_returnsTrue_atLeastJava9() {
+    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    boolean result = jmxRmiOpenTypesSerialFilter.supportsDedicatedSerialFilter();
+
+    assertThat(result).isTrue();
+  }
+
+  @Test
+  public void supportsDedicatedSerialFilter_returnsFalse_atMostJava8() {
+    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    boolean result = jmxRmiOpenTypesSerialFilter.supportsDedicatedSerialFilter();
+
+    assertThat(result).isFalse();
+  }
+
+  @Test
+  public void setPropertyValue_setsValue_ifExistingValueIsNull() {
+    String value = "value-of-property";
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    jmxRmiOpenTypesSerialFilter.setPropertyValueUnlessExists(value);
+
+    String propertyValue = System.getProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME);
+    assertThat(propertyValue).isEqualTo(value);
+  }
+
+  @Test
+  public void setPropertyValue_setsValue_ifExistingValueIsEmpty() {
+    System.setProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME, "");
+    String value = "value-of-property";
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    jmxRmiOpenTypesSerialFilter.setPropertyValueUnlessExists(value);
+
+    String propertyValue = System.getProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME);
+    assertThat(propertyValue).isEqualTo(value);
+  }
+
+  @Test
+  public void setPropertyValue_logsMessage_ifExistingValueIsEmpty() {
+    System.setProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME, "");
+    String value = "value-of-property";
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter =
+        new JmxRmiOpenTypesSerialFilter(infoLogger, () -> true);
+
+    jmxRmiOpenTypesSerialFilter.setPropertyValueUnlessExists(value);
+
+    String expectedLogMessage = "System property " + JmxRmiOpenTypesSerialFilter.PROPERTY_NAME +
+        " is now configured with '" + value + "'.";
+    verify(infoLogger).accept(expectedLogMessage);
+  }
+
+  @Test
+  public void setPropertyValue_leavesExistingValue_ifExistingValueIsNotEmpty() {
+    String existingValue = "existing-value-of-property";
+    System.setProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME, existingValue);
+    String value = "value-of-property";
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    jmxRmiOpenTypesSerialFilter.setPropertyValueUnlessExists(value);
+
+    String propertyValue = System.getProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME);
+    assertThat(propertyValue).isEqualTo(existingValue);
+  }
+
+  @Test
+  public void setPropertyValue_logsMessage_ifExistingValueIsNotEmpty() {
+    String existingValue = "existing-value-of-property";
+    System.setProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME, existingValue);
+    String value = "value-of-property";
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter =
+        new JmxRmiOpenTypesSerialFilter(infoLogger, () -> true);
+
+    jmxRmiOpenTypesSerialFilter.setPropertyValueUnlessExists(value);
+
+    String expectedLogMessage = "System property " + JmxRmiOpenTypesSerialFilter.PROPERTY_NAME +
+        " is already configured.";
+    verify(infoLogger).accept(expectedLogMessage);
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesBoolean() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Boolean.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesByte() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Byte.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesCharacter() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Character.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesShort() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Short.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesInteger() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Integer.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesLong() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Long.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesFloat() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Float.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesDouble() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(Double.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesString() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(String.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesBigInteger() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(BigInteger.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesBigDecimal() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(BigDecimal.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesObjectName() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(ObjectName.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesCompositeData() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(CompositeData.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesTabularData() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(TabularData.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesSimpleType() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(SimpleType.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesCompositeType() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(CompositeType.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesTabularType() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(TabularType.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesArrayType() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(ArrayType.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_includesMarshalledObject() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).contains(MarshalledObject.class.getName());
+  }
+
+  @Test
+  public void createSerialFilterPattern_rejectsAllOtherTypes() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter = new JmxRmiOpenTypesSerialFilter();
+
+    String result = jmxRmiOpenTypesSerialFilter.createSerialFilterPattern();
+
+    assertThat(result).endsWith("!*");
+  }
+
+  @Test
+  public void configureSerialFilterIfEmpty_setsPropertyValue_atLeastJava9() {
+    JmxRmiOpenTypesSerialFilter jmxRmiOpenTypesSerialFilter =
+        new JmxRmiOpenTypesSerialFilter(infoLogger, () -> true);
+
+    jmxRmiOpenTypesSerialFilter.configureSerialFilter();
+
+    String propertyValue = System.getProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME);
+    assertThat(propertyValue).isNotEmpty();
+  }
+
+  @Test
+  public void configureSerialFilterIfEmpty_setsPropertyValue_atMostJava8() {
+    JmxRmiSerialFilter jmxRmiOpenTypesSerialFilter =
+        new JmxRmiOpenTypesSerialFilter(infoLogger, () -> false);
+
+    jmxRmiOpenTypesSerialFilter.configureSerialFilter();
+
+    String propertyValue = System.getProperty(JmxRmiOpenTypesSerialFilter.PROPERTY_NAME);
+    assertThat(propertyValue).isNull();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/SystemManagementServiceTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/SystemManagementServiceTest.java
index 627a5b4..e5e792e 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/SystemManagementServiceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/SystemManagementServiceTest.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.management.internal;
 
 import static org.apache.geode.distributed.internal.ResourceEvent.MANAGER_START;
@@ -29,7 +28,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 import static org.mockito.quality.Strictness.LENIENT;
 
-import java.util.function.BiFunction;
 import java.util.function.Function;
 
 import org.junit.Before;
@@ -66,7 +64,7 @@ public class SystemManagementServiceTest {
   @Mock
   private ManagementAgent managementAgent;
   @Mock
-  private BiFunction<DistributionConfig, InternalCacheForClientAccess, ManagementAgent> managementAgentFactory;
+  private ManagementAgentFactory managementAgentFactory;
   @Mock
   private Function<ManagementResourceRepo, NotificationHub> notificationHubFactory;
   @Mock
@@ -87,7 +85,7 @@ public class SystemManagementServiceTest {
         .create(any(), any(), any(), any(), any(), any(), any(), any(), any()))
             .thenReturn(federatingManager);
 
-    when(managementAgentFactory.apply(any(), any())).thenReturn(managementAgent);
+    when(managementAgentFactory.create(any(), any(), any())).thenReturn(managementAgent);
     when(notificationHubFactory.apply(any())).thenReturn(mock(NotificationHub.class));
     when(localManagerFactory.apply(any())).thenReturn(mock(LocalManager.class));
   }
diff --git a/geode-junit/build.gradle b/geode-junit/build.gradle
index 8f55211..8eeacee 100755
--- a/geode-junit/build.gradle
+++ b/geode-junit/build.gradle
@@ -23,6 +23,7 @@ apply from: "${project.projectDir}/../gradle/publish-java.gradle"
 dependencies {
   compile(platform(project(':boms:geode-all-bom')))
 
+  testImplementation(project(':geode-common'))
   compileOnly(project(':geode-core'))
   compileOnly(project(':geode-logging'))
   compileOnly(project(':geode-serialization'))
diff --git a/geode-junit/src/main/java/org/apache/geode/test/junit/rules/CloseableReference.java b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/CloseableReference.java
new file mode 100644
index 0000000..f2bbd64
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/CloseableReference.java
@@ -0,0 +1,209 @@
+/*
+ * 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.test.junit.rules;
+
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.geode.test.junit.rules.serializable.SerializableExternalResource;
+
+/**
+ * CloseableReference is a JUnit Rule that provides automated tearDown for an atomic reference. If
+ * the referenced value is an {@code AutoCloseable} or {@code Closeable} then it will be auto-closed
+ * and set to null during tear down.
+ *
+ * <p>
+ * If the referenced value is not an {@code AutoCloseable} or {@code Closeable}, the
+ * {@code CloseableReference} will use reflection to invoke any method named {@code close},
+ * {@code disconnect}, or {@code stop} regardless of what interfaces are implemented unless
+ * {@code autoClose} is set to false.
+ *
+ * <p>
+ * If the referenced value is null then it will be ignored during tear down.
+ *
+ * <p>
+ * In the following example, the test has a {@code ServerLauncher} which will be
+ * auto-stopped and set to null during tear down:
+ *
+ * <pre>
+ * {@literal @}Rule
+ * public CloseableReference&lt;ServerLauncher&gt; server = new CloseableReference&lt;&gt;();
+ *
+ * {@literal @}Before
+ * public void setUp() throws IOException {
+ *   server.set(new ServerLauncher.Builder()
+ *       .setMemberName("server1")
+ *       .setDisableDefaultServer(true)
+ *       .reinitializeWithWorkingDirectory(temporaryFolder.newFolder("server1").getAbsolutePath())
+ *       .build());
+ *
+ *   server.get().start();
+ * }
+ *
+ * {@literal @}Test
+ * public void serverHasCache() {
+ *   assertThat(server.get().getCache()).isNotNull();
+ * }
+ * </pre>
+ *
+ * <p>
+ * In the following example, the test has a {@code Cache} which will be auto-closed and set to null
+ * during tear down:
+ *
+ * <pre>
+ * {@literal @}Rule
+ * public CloseableReference&lt;Cache&gt; cache = new CloseableReference&lt;&gt;();
+ *
+ * {@literal @}Before
+ * public void setUp() {
+ *   cache.set(new CacheFactory().create());
+ * }
+ *
+ * {@literal @}Test
+ * public void cacheExists() {
+ *   assertThat(cache.get()).isNotNull();
+ * }
+ * </pre>
+ *
+ * <p>
+ * In the following example, the test has a {@code DistributedSystem} which will be
+ * auto-disconnected and set to null during tear down:
+ *
+ * <pre>
+ * {@literal @}Rule
+ * public CloseableReference&lt;DistributedSystem&gt; system = new CloseableReference&lt;&gt;();
+ *
+ * {@literal @}Before
+ * public void setUp() {
+ *   system.set(DistributedSystem.connect());
+ * }
+ *
+ * {@literal @}Test
+ * public void distributedSystemExists() {
+ *   assertThat(system.get()).isNotNull();
+ * }
+ * </pre>
+ *
+ * <p>
+ * To disable auto-closing in a test, specify {@code autoClose(false)}:
+ *
+ * <pre>
+ * {@literal @}Rule
+ * public CloseableReference&lt;ServerLauncher&gt; serverLauncher =
+ *     new CloseableReference&lt;&gt;().autoClose(false);
+ * </pre>
+ *
+ * <p>
+ * The {@code CloseableReference} value will still be set to null during tear down even if
+ * auto-closing is disabled.
+ */
+@SuppressWarnings({"serial", "WeakerAccess"})
+public class CloseableReference<V> extends SerializableExternalResource {
+
+  private static final AtomicReference<Object> reference = new AtomicReference<>();
+
+  private final AtomicBoolean autoClose = new AtomicBoolean(true);
+
+  /**
+   * Set false to disable autoClose during tearDown. Default is true.
+   */
+  public CloseableReference<V> autoClose(boolean value) {
+    autoClose.set(value);
+    return this;
+  }
+
+  /**
+   * Gets the current value.
+   *
+   * @return the current value
+   */
+  public V get() {
+    return uncheckedCast(reference.get());
+  }
+
+  /**
+   * Sets to the given value.
+   *
+   * @param newValue the new value
+   */
+  public CloseableReference<V> set(V newValue) {
+    reference.set(newValue);
+    return this;
+  }
+
+  @Override
+  protected void after() {
+    V value = get();
+    if (value == null) {
+      return;
+    }
+    reference.set(null);
+
+    if (autoClose.get()) {
+      autoClose(value);
+    }
+  }
+
+  private void autoClose(V value) {
+    if (value instanceof AutoCloseable) {
+      close((AutoCloseable) value);
+
+    } else if (hasMethod(value.getClass(), "close")) {
+      invokeMethod(value, "close");
+
+    } else if (hasMethod(value.getClass(), "disconnect")) {
+      invokeMethod(value, "disconnect");
+
+    } else if (hasMethod(value.getClass(), "stop")) {
+      invokeMethod(value, "stop");
+    }
+  }
+
+  private static void close(AutoCloseable autoCloseable) {
+    try {
+      autoCloseable.close();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static boolean hasMethod(Class<?> objectClass, String methodName) {
+    try {
+      Method method = objectClass.getMethod(methodName);
+      // currently only supports public method with zero parameters
+      if (method.getParameterCount() == 0 &&
+          Modifier.isPublic(method.getModifiers())) {
+        return true;
+      }
+    } catch (NoSuchMethodException e) {
+      // ignore
+    }
+    return false;
+  }
+
+  private static void invokeMethod(Object object, String methodName) {
+    try {
+      Method method = object.getClass().getMethod(methodName);
+      method.invoke(object);
+    } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
diff --git a/geode-junit/src/test/java/org/apache/geode/test/junit/rules/CloseableReferenceTest.java b/geode-junit/src/test/java/org/apache/geode/test/junit/rules/CloseableReferenceTest.java
new file mode 100644
index 0000000..ddcfc94
--- /dev/null
+++ b/geode-junit/src/test/java/org/apache/geode/test/junit/rules/CloseableReferenceTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.test.junit.rules;
+
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.Result;
+
+import org.apache.geode.test.junit.runners.TestRunner;
+
+public class CloseableReferenceTest {
+
+  private static final AtomicReference<CloseableReference<?>> RULE = new AtomicReference<>();
+  private static final AtomicReference<Object> VALUE = new AtomicReference<>();
+
+  @After
+  public void tearDown() {
+    RULE.set(null);
+    VALUE.set(null);
+  }
+
+  @Test
+  public void closesAutoCloseable() throws Exception {
+    VALUE.set(mock(AutoCloseable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    AutoCloseable value = uncheckedCast(VALUE.get());
+    verify(value).close();
+  }
+
+  @Test
+  public void nullsOutAutoCloseable() {
+    VALUE.set(mock(AutoCloseable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    CloseableReference<AutoCloseable> reference = uncheckedCast(RULE.get());
+    assertThat(reference.get()).isNull();
+  }
+
+  @Test
+  public void closesCloseable() throws IOException {
+    VALUE.set(mock(Closeable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    Closeable value = uncheckedCast(VALUE.get());
+    verify(value).close();
+  }
+
+  @Test
+  public void nullsOutCloseable() {
+    VALUE.set(mock(Closeable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    CloseableReference<Closeable> reference = uncheckedCast(RULE.get());
+    assertThat(reference.get()).isNull();
+  }
+
+  @Test
+  public void disconnectDisconnectable() {
+    VALUE.set(mock(Disconnectable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    Disconnectable value = uncheckedCast(VALUE.get());
+    verify(value).disconnect();
+  }
+
+  @Test
+  public void nullsOutDisconnectable() {
+    VALUE.set(mock(Disconnectable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    CloseableReference<Disconnectable> reference = uncheckedCast(RULE.get());
+    assertThat(reference.get()).isNull();
+  }
+
+  @Test
+  public void disconnectStoppable() {
+    VALUE.set(mock(Stoppable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    Stoppable value = uncheckedCast(VALUE.get());
+    verify(value).stop();
+  }
+
+  @Test
+  public void nullsOutStoppable() {
+    VALUE.set(mock(Stoppable.class));
+
+    Result result = TestRunner.runTest(WithCloseableReference.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    CloseableReference<Stoppable> reference = uncheckedCast(RULE.get());
+    assertThat(reference.get()).isNull();
+  }
+
+  @Test
+  public void skipsCloseIfAutoCloseIsFalse() {
+    VALUE.set(mock(AutoCloseable.class));
+
+    Result result = TestRunner.runTest(WithAutoCloseFalse.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    AutoCloseable value = uncheckedCast(VALUE.get());
+    verifyZeroInteractions(value);
+  }
+
+  @Test
+  public void nullsOutValueEvenIfAutoCloseIsFalse() {
+    VALUE.set(mock(AutoCloseable.class));
+
+    Result result = TestRunner.runTest(WithAutoCloseFalse.class);
+    assertThat(result.wasSuccessful()).isTrue();
+
+    CloseableReference<AutoCloseable> reference = uncheckedCast(RULE.get());
+    assertThat(reference.get()).isNull();
+  }
+
+  private static void capture(CloseableReference<?> closeableReference) {
+    RULE.set(closeableReference);
+    closeableReference.set(uncheckedCast(VALUE.get()));
+  }
+
+  public static class WithCloseableReference {
+
+    @Rule
+    public CloseableReference<?> closeableReference = new CloseableReference<>();
+
+    @Before
+    public void setUp() {
+      capture(closeableReference);
+    }
+
+    @Test
+    public void notClosed() {
+      verifyZeroInteractions(closeableReference.get());
+    }
+  }
+
+  public static class WithAutoCloseFalse {
+
+    @Rule
+    public CloseableReference<?> closeableReference = new CloseableReference<>().autoClose(false);
+
+    @Before
+    public void setUp() {
+      capture(closeableReference);
+    }
+
+    @Test
+    public void notClosed() {
+      verifyZeroInteractions(closeableReference.get());
+    }
+  }
+
+  @FunctionalInterface
+  private interface Disconnectable {
+
+    void disconnect();
+  }
+
+  @FunctionalInterface
+  private interface Stoppable {
+
+    void stop();
+  }
+}