You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/10/25 23:42:30 UTC

[GitHub] [geode] kirklund commented on a change in pull request #6962: GEODE-9758: Configure locator serialization filtering by default on Java 8

kirklund commented on a change in pull request #6962:
URL: https://github.com/apache/geode/pull/6962#discussion_r735994727



##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.launchers;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.nio.file.Path;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+
+public class LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest {
+
+  @Rule
+  public GfshRule gfshRule = new GfshRule();
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Rule
+  public ServiceJarRule serviceJarRule = new ServiceJarRule();
+
+  private File locatorFolder;
+
+  @Test
+  public void gfshStartLocatorJava8() {

Review comment:
       Rename this test to `startLocatorConfiguresGlobalSerialFilter_onJava8`

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/ServerManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -33,9 +33,10 @@
 import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
-public class ServerManagerConfiguresJmxSerialFilterAcceptanceTest {
+public class ServerManagerConfiguresJmxGlobalSerialFilterAcceptanceTest {

Review comment:
       Rename this class to `ServerLauncherJmxSerialFilterAcceptanceTest`

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/ServerManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -60,8 +61,8 @@ public void setUpRandomPorts() {
   }
 
   @Test
-  public void startingServerWithJmxManager_configuresSerialFilter_atLeastJava9() {
-    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+  public void startingServerWithJmxManager_configuresSerialFilter_onJava9orGreater() {

Review comment:
       Rename this test to `startServerWithJmxManager_configuresJmxSerialFilter_onJava9orGreater`

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/io/LocatorLauncherGlobalSerialFilterDistributedTest.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.internal.io;
+
+import static java.util.Objects.hash;
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+import static org.apache.commons.lang3.SerializationUtils.serialize;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+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.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class LocatorLauncherGlobalSerialFilterDistributedTest implements Serializable {
+
+  private VM locatorVM;
+  private File locatorDir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<LocatorLauncher> locator = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    locatorVM = getVM(0).bounce();
+    locatorDir = temporaryFolder.newFolder("locator");
+    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    locatorVM.invoke(() -> {
+      locator.set(startLocator("locator", locatorDir, locatorPort));
+    });
+  }
+
+  @Test
+  public void stringIsAllowed() {
+    locatorVM.invoke(() -> {
+      Serializable object = "hello";
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(object))) {
+        assertThat(inputStream.readObject()).isEqualTo(object);
+      }
+    });
+  }
+
+  @Test
+  public void primitiveIsAllows() {
+    locatorVM.invoke(() -> {
+      Integer integerObject = 1;
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(integerObject))) {
+        assertThat(inputStream.readObject()).isEqualTo(integerObject);
+      }
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesThrowJava8() {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+    addIgnoredException(InvalidClassException.class);
+
+    locatorVM.invoke(() -> {
+      Throwable thrown = catchThrowable(() -> {
+        Serializable object = new SerializableClass("hello");
+        try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(object))) {
+          assertThat(inputStream.readObject()).isEqualTo(object);
+        }
+      });
+      assertThat(thrown).isInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void allowedDoesNotThrowJava9() {
+    assumeThat(isJavaVersionAtLeast(JAVA_9)).isTrue();

Review comment:
       Add a line break after the line with `assumeThat`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/Java9LocatorSerializableObjectFilterIntegrationTest.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyBlankIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyBlankIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()

Review comment:
       Rename this test to `configuresGlobalSerialFilter_whenPropertyIsSetToBlank_onJava8`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorJmxSerialFilterIntegrationTest.java
##########
@@ -12,8 +12,10 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal;
+package org.apache.geode.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/ServerManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -14,16 +14,16 @@
  */
 package org.apache.geode.management;

Review comment:
       Move this class to package `org.apache.geode.serialization.filter` (leave out `internal` only for the AcceptanceTests. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterExistsIntegrationTest.java
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.io;
+
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsSameAs;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+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.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterExistsIntegrationTest {

Review comment:
       Rename this class to `LocatorLauncherGlobalSerialFilterExistsIntegrationTest`

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.launchers;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.nio.file.Path;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+
+public class LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest {
+
+  @Rule
+  public GfshRule gfshRule = new GfshRule();
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Rule
+  public ServiceJarRule serviceJarRule = new ServiceJarRule();
+
+  private File locatorFolder;
+
+  @Test
+  public void gfshStartLocatorJava8() {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    locatorFolder = temporaryFolder.getRoot();
+
+    int[] ports = getRandomAvailableTCPPorts(2);
+
+    int locatorPort = ports[0];
+    int locatorJmxPort = ports[1];
+
+    String startLocatorCommand = String.join(" ",
+        "start locator",
+        "--name=locator",
+        "--dir=" + locatorFolder.getAbsolutePath(),
+        "--port=" + locatorPort,
+        "--J=-Dgemfire.jmx-manager-port=" + locatorJmxPort);
+
+    gfshRule.execute(startLocatorCommand);
+
+    try {
+      Path locatorLogFile = locatorFolder.toPath().resolve("locator.log");
+      await().untilAsserted(() -> {
+        LogFileAssert.assertThat(locatorLogFile.toFile()).exists()
+            .contains("Global serial filter is now configured.")
+            .doesNotContain("jdk.serialFilter");
+      });
+    } finally {
+      String stopLocatorCommand = "stop locator --dir=" + locatorFolder.getAbsolutePath();
+      gfshRule.execute(stopLocatorCommand);
+    }
+  }
+
+  // another test for java 9 that does not create global serial filter
+  @Test
+  public void gfshStartLocatorJava9AndAbove() {

Review comment:
       Rename this test to `startLocatorDoesNotConfigureGlobalSerialFilter_onJava9orGreater`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterIntegrationTest.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+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 LocatorGlobalSerialFilterIntegrationTest {

Review comment:
       Rename this class to `LocatorLauncherGlobalSerialFilterIntegrationTest`

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/ServerManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -80,13 +81,13 @@ public void startingServerWithJmxManager_configuresSerialFilter_atLeastJava9() {
       LogFileAssert.assertThat(serverLogFile.toFile())
           .as(serverLogFile.toFile().getAbsolutePath())
           .exists()
-          .contains("System property " + PROPERTY_NAME + " is now configured with");
+          .contains("System property " + JMX_FILTER_PATTERN + " is now configured with");
     });
   }
 
   @Test
   public void startingServerWithJmxManager_configuresSerialFilter_atMostJava8() {

Review comment:
       Rename this test to `startServerWithJmxManager_doesNotConfigureJmxSerialFilter_onJava8`

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/io/LocatorLauncherGlobalSerialFilterDistributedTest.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.internal.io;
+
+import static java.util.Objects.hash;
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+import static org.apache.commons.lang3.SerializationUtils.serialize;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+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.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class LocatorLauncherGlobalSerialFilterDistributedTest implements Serializable {
+
+  private VM locatorVM;
+  private File locatorDir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<LocatorLauncher> locator = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    locatorVM = getVM(0).bounce();
+    locatorDir = temporaryFolder.newFolder("locator");
+    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    locatorVM.invoke(() -> {
+      locator.set(startLocator("locator", locatorDir, locatorPort));
+    });
+  }
+
+  @Test
+  public void stringIsAllowed() {
+    locatorVM.invoke(() -> {
+      Serializable object = "hello";
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(object))) {
+        assertThat(inputStream.readObject()).isEqualTo(object);
+      }
+    });
+  }
+
+  @Test
+  public void primitiveIsAllows() {
+    locatorVM.invoke(() -> {
+      Integer integerObject = 1;
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(integerObject))) {
+        assertThat(inputStream.readObject()).isEqualTo(integerObject);
+      }
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesThrowJava8() {

Review comment:
       Rename this test to `nonAllowedThrowsInvalidClassException_onJava8`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/distributed/LocatorIntegrationTest.java
##########
@@ -123,105 +96,162 @@ public void testThatLocatorDoesNotCreateFileWithZeroPort() throws Exception {
     assertThat(viewFile).doesNotExist();
   }
 
+  @Test
+  public void locatorStartsOnSpecifiedPort() throws IOException {
+    Properties configProperties = new Properties();
+    configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER, "false");
+    configProperties.setProperty(LOG_FILE, "");
+    configProperties.setProperty(MCAST_PORT, "0");
+
+    port = getRandomAvailableTCPPort();
+    locator = Locator.startLocatorAndDS(port, null, configProperties);
+
+    port = locator.getPort();
+    assertThat(port).isEqualTo(port);
+  }
+
+  @Test
+  public void locatorStartsOnEphemeralPort() throws IOException {
+    Properties configProperties = new Properties();
+    configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER, "false");
+    configProperties.setProperty(LOG_FILE, "");
+    configProperties.setProperty(MCAST_PORT, "0");
+
+    locator = Locator.startLocatorAndDS(0, null, configProperties);
+
+    port = locator.getPort();
+    assertThat(port).isNotZero();
+  }
+
   /**
    * Fix: if jmx-manager-start is true in a locator then gfsh connect will fail
    */
   @Test
-  public void testGfshConnectShouldSucceedIfJmxManagerStartIsTrueInLocator() throws Exception {
+  public void gfshConnectsIfJmxManagerStartIsTrue() throws Exception {
     int jmxPort = getRandomAvailableTCPPort();
 
     Properties configProperties = new Properties();
-    configProperties.setProperty(MCAST_PORT, "0");
-    configProperties.setProperty(JMX_MANAGER_PORT, "" + jmxPort);
-    configProperties.setProperty(JMX_MANAGER_START, "true");
-    configProperties.setProperty(JMX_MANAGER_HTTP_PORT, "0");
     configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER_PORT, String.valueOf(jmxPort));
+    configProperties.setProperty(JMX_MANAGER_START, "true");
     configProperties.setProperty(LOG_FILE, "");
-
-    // not needed
-    System.setProperty(GEMFIRE_PREFIX + "disableManagement", "false");
+    configProperties.setProperty(MCAST_PORT, "0");
 
     locator = Locator.startLocatorAndDS(port, null, configProperties);
-    List<JmxManagerProfile> alreadyManaging =
-        GemFireCacheImpl.getInstance().getJmxManagerAdvisor().adviseAlreadyManaging();
+    InternalLocator internalLocator = (InternalLocator) locator;
+    InternalCache cache = internalLocator.getCache();
+    List<JmxManagerProfile> alreadyManaging = cache.getJmxManagerAdvisor().adviseAlreadyManaging();
 
     assertThat(alreadyManaging).hasSize(1);
-    assertThat(alreadyManaging.get(0).getDistributedMember())
-        .isEqualTo(GemFireCacheImpl.getInstance().getMyId());
+    assertThat(alreadyManaging.get(0).getDistributedMember()).isEqualTo(cache.getMyId());
   }
 
   @Test
-  public void testHandlersAreWaitedOn() throws Exception {
+  public void hasHandlerForSharedConfigurationStatusRequest() throws Exception {
     Properties configProperties = new Properties();
-    configProperties.setProperty(MCAST_PORT, "0");
     configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
-    configProperties.setProperty(LOCATOR_WAIT_TIME, "1"); // seconds
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER, "false");
     configProperties.setProperty(LOG_FILE, "");
+    configProperties.setProperty(MCAST_PORT, "0");
 
     locator = Locator.startLocatorAndDS(port, null, configProperties);
 
     InternalLocator internalLocator = (InternalLocator) locator;
 
     // the locator should always install a SharedConfigurationStatusRequest handler
-    assertThat(internalLocator.hasHandlerForClass(SharedConfigurationStatusRequest.class)).isTrue();
+    boolean hasHandler = internalLocator.hasHandlerForClass(SharedConfigurationStatusRequest.class);
+    assertThat(hasHandler).isTrue();
+  }
+
+  @Test
+  public void infoRequestIncludesActualPortWhenSpecifiedPortIsZero() throws Exception {
+    locator = Locator.startLocator(0, logFile);
+    port = locator.getPort();
+    TcpClient client = new TcpClient(
+        SocketCreatorFactory.getSocketCreatorForComponent(LOCATOR),
+        InternalDataSerializer.getDSFIDSerializer().getObjectSerializer(),
+        InternalDataSerializer.getDSFIDSerializer().getObjectDeserializer(),
+        TcpSocketFactory.DEFAULT);
+
+    String[] info = client.getInfo(new HostAndPort("localhost", port));
+
+    assertThat(info).isNotNull();
+    assertThat(info.length).isGreaterThanOrEqualTo(1);
   }
 
   @Test
-  public void testBasicInfo() throws Exception {
-    locator = Locator.startLocator(port, tmpFile);
-    int boundPort = port == 0 ? locator.getPort() : port;
-    TcpClient client = new TcpClient(SocketCreatorFactory
-        .getSocketCreatorForComponent(SecurableCommunicationChannel.LOCATOR),
+  public void infoRequestIncludesActualPortWhenSpecifiedIsNonZero() throws Exception {
+    locator = Locator.startLocator(getRandomAvailableTCPPort(), logFile);
+    port = locator.getPort();
+    TcpClient client = new TcpClient(
+        SocketCreatorFactory.getSocketCreatorForComponent(LOCATOR),
         InternalDataSerializer.getDSFIDSerializer().getObjectSerializer(),
         InternalDataSerializer.getDSFIDSerializer().getObjectDeserializer(),
         TcpSocketFactory.DEFAULT);
-    String[] info = client.getInfo(new HostAndPort("localhost", boundPort));
+
+    String[] info = client.getInfo(new HostAndPort("localhost", port));
 
     assertThat(info).isNotNull();
     assertThat(info.length).isGreaterThanOrEqualTo(1);
   }
 
   @Test
-  public void testNoThreadLeftBehind() throws Exception {
+  public void threadsAreCleanedUpWhenStartFails() throws Exception {
     Properties configProperties = new Properties();
+    configProperties.setProperty(LOG_FILE, "");
     configProperties.setProperty(MCAST_PORT, "0");
-    configProperties.setProperty(JMX_MANAGER_START, "false");
-    configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
 
     int threadCount = Thread.activeCount();
 
     Throwable thrown = catchThrowable(
-        () -> locator = Locator.startLocatorAndDS(-2, new File(""), configProperties));
+        () -> locator = Locator.startLocatorAndDS(-2, null, configProperties));
 
     assertThat(thrown).isInstanceOf(IllegalArgumentException.class);
 
+    // TODO: switch to Awaitility.onTimeout to await without throwing when it releases
     for (int i = 0; i < 10; i++) {
       if (threadCount < Thread.activeCount()) {
         Thread.sleep(1000);
       }
     }
 
+    // TODO: since AssertJ supports withThreadDumpOnError, should we delete OSProcess.printStacks?

Review comment:
       Resolve TODO

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/LocatorManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -64,8 +65,8 @@ public void setUpRandomPorts() {
   }
 
   @Test
-  public void startingLocatorWithJmxManager_configuresSerialFilter_atLeastJava9() {
-    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+  public void startingLocatorWithJmxManager_configuresSerialFilter_onJava9orGreater() {

Review comment:
       Rename this test to `startLocatorConfiguresJmxSerialFilter_onJava9orGreater`

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.cache;
+
+import static java.util.Arrays.asList;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.rules.DistributedRule.getLocatorPort;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.NotSerializableException;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.InternalGemFireException;
+import org.apache.geode.SerializationException;
+import org.apache.geode.cache.util.CacheListenerAdapter;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class ValidateSerializableObjectsDistributedTest implements Serializable {
+
+  private VM server1;
+  private VM server2;
+
+  private File server1Dir;
+  private File server2Dir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<ServerLauncher> server = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    server1 = getVM(0);
+    server2 = getVM(1);
+
+    server1Dir = temporaryFolder.newFolder("server1");
+    server2Dir = temporaryFolder.newFolder("server2");
+
+    locatorPort = getLocatorPort();
+
+    server1.invoke(() -> {
+      server.set(startServer("server1", server1Dir));
+    });
+    server2.invoke(() -> {
+      server.set(startServer("server2", server2Dir));
+    });
+
+    asList(server1, server2).forEach(vm -> vm.invoke(() -> {
+      server.get().getCache()
+          .createRegionFactory(REPLICATE)
+          .addCacheListener(new CacheListenerAdapter<Object, Object>() {
+            @Override
+            public void afterCreate(EntryEvent<Object, Object> event) {
+              assertThat(event.getNewValue()).isNotNull();
+            }
+          })
+          .create("region");
+    }));

Review comment:
       Add a comment to this block explaining:
   ```
   // cache listener afterCreate causes all creates to deserialize the value
   // which causes the tests to pass if serialization filter is configured
   ```

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/LocatorManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -84,13 +85,13 @@ public void startingLocatorWithJmxManager_configuresSerialFilter_atLeastJava9()
       LogFileAssert.assertThat(locatorLogFile.toFile())
           .as(locatorLogFile.toFile().getAbsolutePath())
           .exists()
-          .contains("System property " + PROPERTY_NAME + " is now configured with");
+          .contains("System property " + JMX_FILTER_PATTERN + " is now configured with");
     });
   }
 
   @Test
-  public void startingLocatorWithJmxManager_configuresSerialFilter_atMostJava8() {
-    assumeThat(isJavaVersionAtMost(JavaVersion.JAVA_1_8)).isTrue();
+  public void startingLocatorWithJmxManager_configuresSerialFilter_onJava8() {

Review comment:
       Rename this test to `startLocatorDoesNotConfigureJmxSerialFilter_onJava8`

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.cache;
+
+import static java.util.Arrays.asList;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.rules.DistributedRule.getLocatorPort;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.NotSerializableException;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.InternalGemFireException;
+import org.apache.geode.SerializationException;
+import org.apache.geode.cache.util.CacheListenerAdapter;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class ValidateSerializableObjectsDistributedTest implements Serializable {
+
+  private VM server1;
+  private VM server2;
+
+  private File server1Dir;
+  private File server2Dir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<ServerLauncher> server = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    server1 = getVM(0);
+    server2 = getVM(1);
+
+    server1Dir = temporaryFolder.newFolder("server1");
+    server2Dir = temporaryFolder.newFolder("server2");
+
+    locatorPort = getLocatorPort();
+
+    server1.invoke(() -> {
+      server.set(startServer("server1", server1Dir));
+    });
+    server2.invoke(() -> {
+      server.set(startServer("server2", server2Dir));
+    });
+
+    asList(server1, server2).forEach(vm -> vm.invoke(() -> {
+      server.get().getCache()
+          .createRegionFactory(REPLICATE)
+          .addCacheListener(new CacheListenerAdapter<Object, Object>() {
+            @Override
+            public void afterCreate(EntryEvent<Object, Object> event) {
+              assertThat(event.getNewValue()).isNotNull();
+            }
+          })
+          .create("region");
+    }));
+
+  }
+
+  @Test
+  public void stringIsAllowed() {
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", "value");
+    });
+  }
+
+  @Test
+  public void primitiveIsAllows() {

Review comment:
       Rename this test to `primitiveIsAllowed`

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/io/LocatorLauncherGlobalSerialFilterDistributedTest.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.internal.io;
+
+import static java.util.Objects.hash;
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+import static org.apache.commons.lang3.SerializationUtils.serialize;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+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.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class LocatorLauncherGlobalSerialFilterDistributedTest implements Serializable {
+
+  private VM locatorVM;
+  private File locatorDir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<LocatorLauncher> locator = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    locatorVM = getVM(0).bounce();
+    locatorDir = temporaryFolder.newFolder("locator");
+    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    locatorVM.invoke(() -> {
+      locator.set(startLocator("locator", locatorDir, locatorPort));
+    });
+  }
+
+  @Test
+  public void stringIsAllowed() {
+    locatorVM.invoke(() -> {
+      Serializable object = "hello";
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(object))) {
+        assertThat(inputStream.readObject()).isEqualTo(object);
+      }
+    });
+  }
+
+  @Test
+  public void primitiveIsAllows() {
+    locatorVM.invoke(() -> {
+      Integer integerObject = 1;
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(integerObject))) {
+        assertThat(inputStream.readObject()).isEqualTo(integerObject);
+      }
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesThrowJava8() {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+    addIgnoredException(InvalidClassException.class);
+
+    locatorVM.invoke(() -> {
+      Throwable thrown = catchThrowable(() -> {
+        Serializable object = new SerializableClass("hello");
+        try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(object))) {
+          assertThat(inputStream.readObject()).isEqualTo(object);
+        }
+      });
+      assertThat(thrown).isInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void allowedDoesNotThrowJava9() {

Review comment:
       Rename this test to `allowedDoesNotThrow_onJava9orGreater`

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/logging/internal/LoggingWithReconnectDistributedTest.java
##########
@@ -104,7 +105,7 @@ public void setUp() throws Exception {
     locatorDir = temporaryFolder.newFolder(locatorName);
     server1Dir = temporaryFolder.newFolder(server1Name);
     server2Dir = temporaryFolder.newFolder(server2Name);
-
+    Invoke.invokeInEveryVM(() -> System.setProperty("jdk.serialFilter", "*"));

Review comment:
       Add line break before and after this line

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/management/JmxLocatorReconnectDistributedTest.java
##########
@@ -117,7 +118,7 @@ public void setUp() throws Exception {
 
     File locatorDir = temporaryFolder.newFolder(LOCATOR_NAME);
     File serverDir = temporaryFolder.newFolder(SERVER_NAME);
-
+    Invoke.invokeInEveryVM(() -> System.setProperty("jdk.serialFilter", "*"));

Review comment:
       Add line break before and after this line

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/cache/snapshot/GFSnapshotDUnitTest.java
##########
@@ -77,7 +78,7 @@ public void testDataExportAndIterate() throws IOException, ClassNotFoundExceptio
     String serverHostName = NetworkUtils.getServerHostName(host);
 
     Properties properties = configureCommonProperties(new Properties());
-
+    Invoke.invokeInEveryVM(() -> System.setProperty("jdk.serialFilter", "*"));

Review comment:
       Add line break before and after this line

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/io/LocatorLauncherGlobalSerialFilterDistributedTest.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.internal.io;
+
+import static java.util.Objects.hash;
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+import static org.apache.commons.lang3.SerializationUtils.serialize;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+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.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class LocatorLauncherGlobalSerialFilterDistributedTest implements Serializable {
+
+  private VM locatorVM;
+  private File locatorDir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<LocatorLauncher> locator = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    locatorVM = getVM(0).bounce();
+    locatorDir = temporaryFolder.newFolder("locator");
+    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    locatorVM.invoke(() -> {
+      locator.set(startLocator("locator", locatorDir, locatorPort));
+    });
+  }
+
+  @Test
+  public void stringIsAllowed() {
+    locatorVM.invoke(() -> {
+      Serializable object = "hello";
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(object))) {
+        assertThat(inputStream.readObject()).isEqualTo(object);
+      }
+    });
+  }
+
+  @Test
+  public void primitiveIsAllows() {
+    locatorVM.invoke(() -> {
+      Integer integerObject = 1;
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(integerObject))) {
+        assertThat(inputStream.readObject()).isEqualTo(integerObject);
+      }
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesThrowJava8() {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();

Review comment:
       Add a line break after the line with `assumeThat`

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/LocatorManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -33,9 +33,10 @@
 import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
-public class LocatorManagerConfiguresJmxSerialFilterAcceptanceTest {
+public class LocatorManagerConfiguresJmxGlobalSerialFilterAcceptanceTest {

Review comment:
       Rename this class to `LocatorLauncherJmxSerialFilterAcceptanceTest`.
   
   Our terminology is "JmxSerialFilter" versus "GlobalSerialFilter", never a combination of the two.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/Java8LocatorSerializableObjectFilterIntegrationTest.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionSingleHopDUnitTest.java
##########
@@ -142,7 +143,7 @@ public void setUp() {
     vm1 = getVM(1);
     vm2 = getVM(2);
     vm3 = getVM(3);
-
+    Invoke.invokeInEveryVM(() -> System.setProperty("jdk.serialFilter", "*"));

Review comment:
       Add line break before and after this line

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterBlankIntegrationTest.java
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/io/LocatorLauncherGlobalSerialFilterDistributedTest.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.internal.io;
+
+import static java.util.Objects.hash;
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+import static org.apache.commons.lang3.SerializationUtils.serialize;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtMost;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+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.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class LocatorLauncherGlobalSerialFilterDistributedTest implements Serializable {
+
+  private VM locatorVM;
+  private File locatorDir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<LocatorLauncher> locator = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    locatorVM = getVM(0).bounce();
+    locatorDir = temporaryFolder.newFolder("locator");
+    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    locatorVM.invoke(() -> {
+      locator.set(startLocator("locator", locatorDir, locatorPort));
+    });
+  }
+
+  @Test
+  public void stringIsAllowed() {
+    locatorVM.invoke(() -> {
+      Serializable object = "hello";
+      try (ObjectInput inputStream = new ObjectInputStream(byteArrayInputStream(object))) {
+        assertThat(inputStream.readObject()).isEqualTo(object);
+      }
+    });
+  }
+
+  @Test
+  public void primitiveIsAllows() {

Review comment:
       Rename this test to `primitiveIsAllowed`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterIntegrationTest.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/distributed/LocatorIntegrationTest.java
##########
@@ -123,105 +96,162 @@ public void testThatLocatorDoesNotCreateFileWithZeroPort() throws Exception {
     assertThat(viewFile).doesNotExist();
   }
 
+  @Test
+  public void locatorStartsOnSpecifiedPort() throws IOException {
+    Properties configProperties = new Properties();
+    configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER, "false");
+    configProperties.setProperty(LOG_FILE, "");
+    configProperties.setProperty(MCAST_PORT, "0");
+
+    port = getRandomAvailableTCPPort();
+    locator = Locator.startLocatorAndDS(port, null, configProperties);
+
+    port = locator.getPort();
+    assertThat(port).isEqualTo(port);
+  }
+
+  @Test
+  public void locatorStartsOnEphemeralPort() throws IOException {
+    Properties configProperties = new Properties();
+    configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER, "false");
+    configProperties.setProperty(LOG_FILE, "");
+    configProperties.setProperty(MCAST_PORT, "0");
+
+    locator = Locator.startLocatorAndDS(0, null, configProperties);
+
+    port = locator.getPort();
+    assertThat(port).isNotZero();
+  }
+
   /**
    * Fix: if jmx-manager-start is true in a locator then gfsh connect will fail
    */
   @Test
-  public void testGfshConnectShouldSucceedIfJmxManagerStartIsTrueInLocator() throws Exception {
+  public void gfshConnectsIfJmxManagerStartIsTrue() throws Exception {
     int jmxPort = getRandomAvailableTCPPort();
 
     Properties configProperties = new Properties();
-    configProperties.setProperty(MCAST_PORT, "0");
-    configProperties.setProperty(JMX_MANAGER_PORT, "" + jmxPort);
-    configProperties.setProperty(JMX_MANAGER_START, "true");
-    configProperties.setProperty(JMX_MANAGER_HTTP_PORT, "0");
     configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER_PORT, String.valueOf(jmxPort));
+    configProperties.setProperty(JMX_MANAGER_START, "true");
     configProperties.setProperty(LOG_FILE, "");
-
-    // not needed
-    System.setProperty(GEMFIRE_PREFIX + "disableManagement", "false");
+    configProperties.setProperty(MCAST_PORT, "0");
 
     locator = Locator.startLocatorAndDS(port, null, configProperties);
-    List<JmxManagerProfile> alreadyManaging =
-        GemFireCacheImpl.getInstance().getJmxManagerAdvisor().adviseAlreadyManaging();
+    InternalLocator internalLocator = (InternalLocator) locator;
+    InternalCache cache = internalLocator.getCache();
+    List<JmxManagerProfile> alreadyManaging = cache.getJmxManagerAdvisor().adviseAlreadyManaging();
 
     assertThat(alreadyManaging).hasSize(1);
-    assertThat(alreadyManaging.get(0).getDistributedMember())
-        .isEqualTo(GemFireCacheImpl.getInstance().getMyId());
+    assertThat(alreadyManaging.get(0).getDistributedMember()).isEqualTo(cache.getMyId());
   }
 
   @Test
-  public void testHandlersAreWaitedOn() throws Exception {
+  public void hasHandlerForSharedConfigurationStatusRequest() throws Exception {
     Properties configProperties = new Properties();
-    configProperties.setProperty(MCAST_PORT, "0");
     configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
-    configProperties.setProperty(LOCATOR_WAIT_TIME, "1"); // seconds
+    configProperties.setProperty(HTTP_SERVICE_PORT, "0");
+    configProperties.setProperty(JMX_MANAGER, "false");
     configProperties.setProperty(LOG_FILE, "");
+    configProperties.setProperty(MCAST_PORT, "0");
 
     locator = Locator.startLocatorAndDS(port, null, configProperties);
 
     InternalLocator internalLocator = (InternalLocator) locator;
 
     // the locator should always install a SharedConfigurationStatusRequest handler
-    assertThat(internalLocator.hasHandlerForClass(SharedConfigurationStatusRequest.class)).isTrue();
+    boolean hasHandler = internalLocator.hasHandlerForClass(SharedConfigurationStatusRequest.class);
+    assertThat(hasHandler).isTrue();
+  }
+
+  @Test
+  public void infoRequestIncludesActualPortWhenSpecifiedPortIsZero() throws Exception {
+    locator = Locator.startLocator(0, logFile);
+    port = locator.getPort();
+    TcpClient client = new TcpClient(
+        SocketCreatorFactory.getSocketCreatorForComponent(LOCATOR),
+        InternalDataSerializer.getDSFIDSerializer().getObjectSerializer(),
+        InternalDataSerializer.getDSFIDSerializer().getObjectDeserializer(),
+        TcpSocketFactory.DEFAULT);
+
+    String[] info = client.getInfo(new HostAndPort("localhost", port));
+
+    assertThat(info).isNotNull();
+    assertThat(info.length).isGreaterThanOrEqualTo(1);
   }
 
   @Test
-  public void testBasicInfo() throws Exception {
-    locator = Locator.startLocator(port, tmpFile);
-    int boundPort = port == 0 ? locator.getPort() : port;
-    TcpClient client = new TcpClient(SocketCreatorFactory
-        .getSocketCreatorForComponent(SecurableCommunicationChannel.LOCATOR),
+  public void infoRequestIncludesActualPortWhenSpecifiedIsNonZero() throws Exception {
+    locator = Locator.startLocator(getRandomAvailableTCPPort(), logFile);
+    port = locator.getPort();
+    TcpClient client = new TcpClient(
+        SocketCreatorFactory.getSocketCreatorForComponent(LOCATOR),
         InternalDataSerializer.getDSFIDSerializer().getObjectSerializer(),
         InternalDataSerializer.getDSFIDSerializer().getObjectDeserializer(),
         TcpSocketFactory.DEFAULT);
-    String[] info = client.getInfo(new HostAndPort("localhost", boundPort));
+
+    String[] info = client.getInfo(new HostAndPort("localhost", port));
 
     assertThat(info).isNotNull();
     assertThat(info.length).isGreaterThanOrEqualTo(1);
   }
 
   @Test
-  public void testNoThreadLeftBehind() throws Exception {
+  public void threadsAreCleanedUpWhenStartFails() throws Exception {
     Properties configProperties = new Properties();
+    configProperties.setProperty(LOG_FILE, "");
     configProperties.setProperty(MCAST_PORT, "0");
-    configProperties.setProperty(JMX_MANAGER_START, "false");
-    configProperties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
 
     int threadCount = Thread.activeCount();
 
     Throwable thrown = catchThrowable(
-        () -> locator = Locator.startLocatorAndDS(-2, new File(""), configProperties));
+        () -> locator = Locator.startLocatorAndDS(-2, null, configProperties));
 
     assertThat(thrown).isInstanceOf(IllegalArgumentException.class);
 
+    // TODO: switch to Awaitility.onTimeout to await without throwing when it releases

Review comment:
       Resolve TODO

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyBlankIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyBlankIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, "");

Review comment:
       Change this to set the value to `" "` containing a space

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyEmptyIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyEmptyIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToEmpty_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, "");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThatSerialFilterIsNotNull();
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenJdkSerialFilterPropertyIsSetToEmpty_onJava9orGreater()

Review comment:
       Rename this test to `doesNotConfigureGlobalSerialFilter_whenPropertyIsSetToEmpty_onJava9orGreater`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyEmptyIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyBlankIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyBlankIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, "");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThatSerialFilterIsNotNull();
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava9orGreater()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtLeast(JAVA_9)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, "");

Review comment:
       Change this to set the value to `" "` containing a space

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.cache;
+
+import static java.util.Arrays.asList;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.rules.DistributedRule.getLocatorPort;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidClassException;
+import java.io.NotSerializableException;
+import java.io.Serializable;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.InternalGemFireException;
+import org.apache.geode.SerializationException;
+import org.apache.geode.cache.util.CacheListenerAdapter;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedReference;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@SuppressWarnings("serial")
+public class ValidateSerializableObjectsDistributedTest implements Serializable {
+
+  private VM server1;
+  private VM server2;
+
+  private File server1Dir;
+  private File server2Dir;
+  private int locatorPort;
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+  @Rule
+  public DistributedReference<ServerLauncher> server = new DistributedReference<>();
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    server1 = getVM(0);
+    server2 = getVM(1);
+
+    server1Dir = temporaryFolder.newFolder("server1");
+    server2Dir = temporaryFolder.newFolder("server2");
+
+    locatorPort = getLocatorPort();
+
+    server1.invoke(() -> {
+      server.set(startServer("server1", server1Dir));
+    });
+    server2.invoke(() -> {
+      server.set(startServer("server2", server2Dir));
+    });
+
+    asList(server1, server2).forEach(vm -> vm.invoke(() -> {
+      server.get().getCache()
+          .createRegionFactory(REPLICATE)
+          .addCacheListener(new CacheListenerAdapter<Object, Object>() {
+            @Override
+            public void afterCreate(EntryEvent<Object, Object> event) {
+              assertThat(event.getNewValue()).isNotNull();
+            }
+          })
+          .create("region");
+    }));
+
+  }
+
+  @Test
+  public void stringIsAllowed() {
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", "value");
+    });
+  }
+
+  @Test
+  public void primitiveIsAllows() {
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put(1, 1);
+    });
+  }
+
+  @Test
+  public void nonSerializableThrowsNotSerializableException() {
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.put(new Object(), new Object());
+      });
+      assertThat(thrown).hasCauseExactlyInstanceOf(NotSerializableException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedIsNotPutInOtherServer() {

Review comment:
       Rename this test to `nonAllowedIsNotPropagatedToOtherServer`

##########
File path: geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ManagementRequestLoggingDistributedTest.java
##########
@@ -83,7 +84,7 @@ public void setUp() throws Exception {
     int[] ports = getRandomAvailableTCPPorts(2);
     httpPort = ports[0];
     jmxManagerPort = ports[1];
-
+    Invoke.invokeInEveryVM(() -> System.setProperty("jdk.serialFilter", "*"));

Review comment:
       Add line break before and after this line.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyEmptyIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyEmptyIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToEmpty_onJava8()

Review comment:
       Rename this test to `configuresGlobalSerialFilter_whenPropertyIsSetToEmpty_onJava8`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorJmxSerialFilterIntegrationTest.java
##########
@@ -22,34 +24,41 @@
 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.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.serialization.filter.OpenMBeanFilterPattern;
+import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.internal.SystemManagementService;
 import org.apache.geode.test.junit.rules.CloseableReference;
 
-public class LocatorManagerConfiguresJmxSerialFilterIntegrationTest {
+public class LocatorJmxSerialFilterIntegrationTest {

Review comment:
       Rename this test class to `LocatorLauncherJmxSerialFilterIntegrationTest`

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.launchers;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.nio.file.Path;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+
+public class LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest {
+
+  @Rule
+  public GfshRule gfshRule = new GfshRule();
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Rule
+  public ServiceJarRule serviceJarRule = new ServiceJarRule();
+
+  private File locatorFolder;
+
+  @Test
+  public void gfshStartLocatorJava8() {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    locatorFolder = temporaryFolder.getRoot();
+
+    int[] ports = getRandomAvailableTCPPorts(2);
+
+    int locatorPort = ports[0];
+    int locatorJmxPort = ports[1];
+
+    String startLocatorCommand = String.join(" ",
+        "start locator",
+        "--name=locator",
+        "--dir=" + locatorFolder.getAbsolutePath(),
+        "--port=" + locatorPort,
+        "--J=-Dgemfire.jmx-manager-port=" + locatorJmxPort);
+
+    gfshRule.execute(startLocatorCommand);
+
+    try {
+      Path locatorLogFile = locatorFolder.toPath().resolve("locator.log");
+      await().untilAsserted(() -> {
+        LogFileAssert.assertThat(locatorLogFile.toFile()).exists()
+            .contains("Global serial filter is now configured.")
+            .doesNotContain("jdk.serialFilter");
+      });
+    } finally {
+      String stopLocatorCommand = "stop locator --dir=" + locatorFolder.getAbsolutePath();
+      gfshRule.execute(stopLocatorCommand);
+    }

Review comment:
       This is ok, but I typically just put the lines of this finally block in a `tearDown` method annotated with `@After`. Both tests end up looking a bit cleaner and more brief.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/SerialFilterAssertions.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 org.apache.geode.internal.io;

Review comment:
       Move this class to package org.apache.geode.internal.serialization. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyExistsIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, " ");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThat(OBJECT_INPUT_FILTER_API.getSerialFilter())
+        .as("ObjectInputFilter$Config.getSerialFilter()")
+        .isNotNull();
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenSerialFilterIsNull_onJava9orGreater()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtLeast(JAVA_9)).isTrue();
+

Review comment:
       Add this line with line breaks before and after:
   ```
   
   System.setProperty(JDK_SERIAL_FILTER_PROPERTY, "*");
   
   ```

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.launchers;

Review comment:
       Create a new package `org.apache.geode.serialization.filter` for AcceptanceTests in geode-assembly. Move this class to package `org.apache.geode.serialization.filter` (leave out `internal` only for the AcceptanceTests. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyExistsIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, " ");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThat(OBJECT_INPUT_FILTER_API.getSerialFilter())
+        .as("ObjectInputFilter$Config.getSerialFilter()")
+        .isNotNull();
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenSerialFilterIsNull_onJava9orGreater()

Review comment:
       Rename this test to `doesNotChangeGlobalSerialFilter_whenPropertyExists_onJava9orGreater`

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/management/LocatorManagerConfiguresJmxGlobalSerialFilterAcceptanceTest.java
##########
@@ -14,16 +14,16 @@
  */
 package org.apache.geode.management;

Review comment:
       Move this class to package `org.apache.geode.serialization.filter` (leave out `internal` only for the AcceptanceTests. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyExistsIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()

Review comment:
       Rename this test to `doesNotChangeGlobalSerialFilter_whenPropertyExists_onJava8

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyExistsIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, " ");

Review comment:
       Change the value from `" "` to `"*"` containing an `*`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyBlankIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyBlankIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, "");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThatSerialFilterIsNotNull();
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava9orGreater()

Review comment:
       Rename this test to `doesNotConfigureGlobalSerialFilter_whenPropertyIsSetToBlank_onJava9orGreater`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/Java9LocatorSerializableObjectFilterIntegrationTest.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+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.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.distributed.internal.DistributionConfig.DEFAULT_SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.internal.SystemManagementService;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class Java9LocatorSerializableObjectFilterIntegrationTest {

Review comment:
       Rename class to `Java9LocatorLauncherSerializableObjectFilterIntegrationTest`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/Java9LocatorSerializableObjectFilterIntegrationTest.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+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.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.distributed.internal.DistributionConfig.DEFAULT_SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.internal.SystemManagementService;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class Java9LocatorSerializableObjectFilterIntegrationTest {
+
+  private static final String NAME = "locator";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @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];
+  }
+
+  @Test
+  public void doesNotConfigureValidateSerializableObjects_onJava9orGreater() {
+    assumeThat(isJavaVersionAtLeast(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();
+
+    assertThat(isJmxManagerStarted())
+        .isTrue();
+    assertThat(isValidateSerializableObjectsConfigured())
+        .as(VALIDATE_SERIALIZABLE_OBJECTS)
+        .isFalse();
+  }
+
+  @Test
+  public void doesNotConfigureSerializableObjectFilter_onJava9orGreater() {
+    assumeThat(isJavaVersionAtLeast(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();
+
+    assertThat(isJmxManagerStarted())
+        .isTrue();
+    assertThat(getDistributionConfig().getSerializableObjectFilter())
+        .as(SERIALIZABLE_OBJECT_FILTER)
+        .isEqualTo(DEFAULT_SERIALIZABLE_OBJECT_FILTER);
+  }
+
+  @Test
+  public void doesNotChangeEmptySerializableObjectFilter_onJava9orGreater() {
+    assumeThat(isJavaVersionAtLeast(JAVA_9)).isTrue();
+

Review comment:
       Add this and reference it when setting `VALIDATE_SERIALIZABLE_OBJECTS`:
   ```
   String emptySerializableObjectFilter = "";
   ```

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/io/LocatorLauncherGlobalSerialFilterDistributedTest.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.launchers;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.nio.file.Path;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+
+public class LocatorLauncherConfiguresGlobalSerialFilterAcceptanceTest {

Review comment:
       Rename this class to `LocatorLauncherGlobalSerialFilterAcceptanceTest`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterExistsIntegrationTest.java
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.io;
+
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsSameAs;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+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.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterExistsIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenSerialFilterExists()

Review comment:
       Rename this test to `doesNotConfigureGlobalSerialFilter_whenFilterExists`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterBlankIntegrationTest.java
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.io;
+
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsSameAs;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+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.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterBlankIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenSerialFilterIsSetToBlank()

Review comment:
       Rename this test to `doesNotConfigureGlobalSerialFilter_whenFilterIsSetToBlank`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterBlankIntegrationTest.java
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.io;
+
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsSameAs;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+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.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterBlankIntegrationTest {

Review comment:
       Rename this class to `LocatorLauncherGlobalSerialFilterBlankIntegrationTest`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyBlankIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/Java8LocatorSerializableObjectFilterIntegrationTest.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+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.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.distributed.internal.DistributionConfig.DEFAULT_SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.internal.SystemManagementService;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class Java8LocatorSerializableObjectFilterIntegrationTest {
+
+  private static final String NAME = "locator";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @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];
+  }
+
+  @Test
+  public void configuresValidateSerializableObjects_onJava8() {
+    assumeThat(isJavaVersionAtMost(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();
+
+    assertThat(isJmxManagerStarted())
+        .isTrue();
+    assertThat(isValidateSerializableObjectsConfigured())
+        .as(VALIDATE_SERIALIZABLE_OBJECTS)
+        .isTrue();
+  }
+
+  @Test
+  public void usesDefaultSerializableObjectFilter_onJava8() {
+    assumeThat(isJavaVersionAtMost(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();
+
+    assertThat(isJmxManagerStarted())
+        .isTrue();
+    assertThat(getDistributionConfig().getSerializableObjectFilter())
+        .as(SERIALIZABLE_OBJECT_FILTER)
+        .isEqualTo(DEFAULT_SERIALIZABLE_OBJECT_FILTER);
+  }
+
+  @Test
+  public void doesNotChangeEmptySerializableObjectFilter_onJava8() {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+

Review comment:
       Add this and reference it when setting `VALIDATE_SERIALIZABLE_OBJECTS`:
   ```
   String emptySerializableObjectFilter = "";

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/Java8LocatorSerializableObjectFilterIntegrationTest.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+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.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.distributed.ConfigurationProperties.VALIDATE_SERIALIZABLE_OBJECTS;
+import static org.apache.geode.distributed.internal.DistributionConfig.DEFAULT_SERIALIZABLE_OBJECT_FILTER;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.internal.SystemManagementService;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class Java8LocatorSerializableObjectFilterIntegrationTest {

Review comment:
       Rename this class to `Java8LocatorLauncherSerializableObjectFilterIntegrationTest`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterExistsIntegrationTest.java
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.io;

Review comment:
       Move this class to package `org.apache.geode.internal.serialization.filter`. Keep it in the same module and same src set.

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyBlankIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyBlankIntegrationTest {

Review comment:
       Rename this class to `LocatorLauncherGlobalSerialFilterPropertyBlankIntegrationTest`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyEmptyIntegrationTest.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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_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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyEmptyIntegrationTest {

Review comment:
       Rename this class to `LocatorLauncherGlobalSerialFilterPropertyEmptyIntegrationTest`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyExistsIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, " ");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThat(OBJECT_INPUT_FILTER_API.getSerialFilter())
+        .as("ObjectInputFilter$Config.getSerialFilter()")
+        .isNotNull();

Review comment:
       TODO: we need to find a better way to validate this test

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyExistsIntegrationTest {
+
+  private static final String NAME = "locator";
+  private static final String JDK_SERIAL_FILTER_PROPERTY = "jdk.serialFilter";
+  private static final ObjectInputFilterApi OBJECT_INPUT_FILTER_API =
+      new ReflectionObjectInputFilterApiFactory().createObjectInputFilterApi();
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilterWhenJdkSerialFilterPropertyIsSetToBlank_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    System.setProperty(JDK_SERIAL_FILTER_PROPERTY, " ");
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThat(OBJECT_INPUT_FILTER_API.getSerialFilter())
+        .as("ObjectInputFilter$Config.getSerialFilter()")
+        .isNotNull();
+  }
+
+  @Test
+  public void doesNotSetSerialFilterWhenSerialFilterIsNull_onJava9orGreater()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtLeast(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();
+
+    assertThat(OBJECT_INPUT_FILTER_API.getSerialFilter())
+        .as("ObjectInputFilter$Config.getSerialFilter()")
+        .isNull();

Review comment:
       TODO: we need to find a better way to validate this test

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterPropertyExistsIntegrationTest.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.internal.serialization.filter.ObjectInputFilterApi;
+import org.apache.geode.internal.serialization.filter.ReflectionObjectInputFilterApiFactory;
+import org.apache.geode.test.junit.rules.CloseableReference;
+
+public class LocatorGlobalSerialFilterPropertyExistsIntegrationTest {

Review comment:
       Rename this class to `LocatorLauncherGlobalSerialFilterPropertyExistsIntegrationTest`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterIntegrationTest.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+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 LocatorGlobalSerialFilterIntegrationTest {
+
+  private static final String NAME = "locator";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilter_onJava8()
+      throws InvocationTargetException, IllegalAccessException {
+    assumeThat(isJavaVersionAtMost(JAVA_1_8)).isTrue();
+
+    locator.set(new LocatorLauncher.Builder()
+        .setMemberName(NAME)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .set(LOG_FILE, new File(workingDirectory, NAME + ".log").getAbsolutePath())
+        .build())
+        .get()
+        .start();
+
+    assertThatSerialFilterIsNotNull();
+  }
+
+  @Test
+  public void doesNotSetSerialFilter_onJava9orGreater()

Review comment:
       Rename test to `doesNotConfigureGlobalSerialFilter_onJava9orGreater`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorGlobalSerialFilterIntegrationTest.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.internal.io;
+
+import static org.apache.commons.lang3.JavaVersion.JAVA_1_8;
+import static org.apache.commons.lang3.JavaVersion.JAVA_9;
+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.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNotNull;
+import static org.apache.geode.internal.io.SerialFilterAssertions.assertThatSerialFilterIsNull;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+
+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 LocatorGlobalSerialFilterIntegrationTest {
+
+  private static final String NAME = "locator";
+
+  private File workingDirectory;
+  private int locatorPort;
+  private int jmxPort;
+
+  @Rule
+  public CloseableReference<LocatorLauncher> locator = new CloseableReference<>();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws IOException {
+    workingDirectory = temporaryFolder.newFolder(NAME);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    jmxPort = ports[0];
+    locatorPort = ports[1];
+  }
+
+  @Test
+  public void setsSerialFilter_onJava8()

Review comment:
       Rename test to `configuresGlobalSerialFilter_onJava8`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/internal/io/LocatorJmxSerialFilterIntegrationTest.java
##########
@@ -83,15 +87,18 @@ public void startingLocatorWithJmxManager_configuresSerialFilter_atLeastJava9()
         .get()
         .start();
 
-    String serialFilter = System.getProperty(PROPERTY_NAME);
-    assertThat(serialFilter).isEqualTo(expectedSerialFilter);
+    assertThat(isJmxManagerStarted())
+        .isTrue();
+    assertThat(System.getProperty(JMX_SERIAL_FILTER_PROPERTY))
+        .as(JMX_SERIAL_FILTER_PROPERTY)
+        .isEqualTo(openMBeanFilterPattern);
   }
 
   @Test
-  public void startingLocatorWithJmxManager_changesEmptySerialFilter_atLeastJava9() {
-    assumeThat(isJavaVersionAtLeast(JavaVersion.JAVA_9)).isTrue();
+  public void changesEmptyJmxSerialFilter_onJava9orGreater() {

Review comment:
       Rename this test to `changesJmxSerialFilter_whenPropertyIsEmpty_onJava9orGreater`




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

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

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