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/12/10 23:18:44 UTC

[GitHub] [geode] kirklund opened a new pull request #7189: GEODE-9758: Add serial filter config and utils

kirklund opened a new pull request #7189:
URL: https://github.com/apache/geode/pull/7189


   1. Add SerializableObjectConfig to geode-serialization and change DistributionConfig to extend it. geode-serialization is upstream from geode-core and cannot reference DistributionConfig.
   2. Add ObjectInputFilterUtils to refactor redundant code that checks if ObjectInputFilter is supported.
   3. Add Jianxia's improvements to LocatorIntegrationTest.
   4. Add ValidateSerializableObjectsDistributedTest to test validate-serializable-objects.


-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768191777



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {

Review comment:
       It's correct as is.
   
   I used `NotPropagated` because the region entry is not propagated to the other server's cache. If you do a region.get(key) operation, you'll find that it's not present in the other server's cache. The message was sent but it failed to update the remote server because the value could not be deserialized.
   
   It would have been propagated to the remote server but the CacheListener.afterCreate attempts to deserialize. This fails and causes the Create message/operation to abort. The test then attempts the region.get(key) as validation that the key/value pair is definitely not present.




-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768192952



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(SerializationException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", new SerializableClass());
+    });
+
+    server2.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.get("key");
+      });
+      assertThat(thrown)
+          .isInstanceOf(SerializationException.class)
+          .hasCauseInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesNotThrow() {

Review comment:
       Looks like this one is wrong. I'll rename it.




-- 
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



[GitHub] [geode] kirklund closed pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund closed pull request #7189:
URL: https://github.com/apache/geode/pull/7189


   


-- 
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



[GitHub] [geode] kirklund commented on pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on pull request #7189:
URL: https://github.com/apache/geode/pull/7189#issuecomment-993038070


   I will resubmit all serialization filtering PRs as one big PR again. This has gotten out of hand.


-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768207881



##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/filter/SerializableObjectConfig.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.serialization.filter;
+
+/**
+ * Defines the configuration properties for serialization filtering in Geode.
+ */
+public interface SerializableObjectConfig {
+
+  default String getSerializableObjectFilterIfEnabled() {
+    return getValidateSerializableObjects() ? getSerializableObjectFilter() : null;
+  }
+
+  boolean getValidateSerializableObjects();
+
+  void setValidateSerializableObjects(boolean value);
+
+  String getSerializableObjectFilter();

Review comment:
       `DistributionConfig` defines all of the methods that geode-core needs to call including `setSerializableObjectFilter` and `setValidateSerializableObjects`. We have lots of tests that are setting `serializable-object-filter`. Take a look at the callers of `DistributionConfig.setSerializableObjectFilter`. 
   
   `SerializableObjectConfig` defines only the methods of `DistributionConfig` that geode-serialization needs to call and it does not currently need to invoke `setSerializableObjectFilter`. The next PR invokes `setValidateSerializableObjects` from geode-serialization but not `setSerializableObjectFilter`.




-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768194386



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {

Review comment:
       Actually, I'll change this test and name to make it more correct and more clear.




-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768194386



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {

Review comment:
       I'll change this test and name to make it more correct and more clear. I'm renaming all of the tests in this test class.




-- 
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



[GitHub] [geode] jchen21 commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
jchen21 commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r767052995



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(SerializationException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", new SerializableClass());
+    });
+
+    server2.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.get("key");
+      });
+      assertThat(thrown)
+          .isInstanceOf(SerializationException.class)
+          .hasCauseInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesNotThrow() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(IOException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.put(new SerializableClass(), new SerializableClass());
+      });
+      assertThat(thrown).isInstanceOf(InternalGemFireException.class);

Review comment:
       Why this is still `InternalGemFireException`? I am expecting an `InvalidClassException`.

##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/filter/SerializableObjectConfig.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.serialization.filter;
+
+/**
+ * Defines the configuration properties for serialization filtering in Geode.
+ */
+public interface SerializableObjectConfig {
+
+  default String getSerializableObjectFilterIfEnabled() {
+    return getValidateSerializableObjects() ? getSerializableObjectFilter() : null;
+  }
+
+  boolean getValidateSerializableObjects();
+
+  void setValidateSerializableObjects(boolean value);
+
+  String getSerializableObjectFilter();

Review comment:
       We also need a setter `setSerializableObjectFilter`.  I have not seen `serializable-object-filter` being set in the product or test.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(SerializationException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", new SerializableClass());
+    });
+
+    server2.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.get("key");
+      });
+      assertThat(thrown)
+          .isInstanceOf(SerializationException.class)
+          .hasCauseInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesNotThrow() {

Review comment:
       Do we need the `Not` in the test name? It does throw exception.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {

Review comment:
       Do we need the `Not` in the test name? It is propagated to the other server.




-- 
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



[GitHub] [geode] jchen21 commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
jchen21 commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r767052995



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(SerializationException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", new SerializableClass());
+    });
+
+    server2.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.get("key");
+      });
+      assertThat(thrown)
+          .isInstanceOf(SerializationException.class)
+          .hasCauseInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesNotThrow() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(IOException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.put(new SerializableClass(), new SerializableClass());
+      });
+      assertThat(thrown).isInstanceOf(InternalGemFireException.class);

Review comment:
       Why this is still `InternalGemFireException`? I am expecting an `InvalidClassException`.

##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/filter/SerializableObjectConfig.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.serialization.filter;
+
+/**
+ * Defines the configuration properties for serialization filtering in Geode.
+ */
+public interface SerializableObjectConfig {
+
+  default String getSerializableObjectFilterIfEnabled() {
+    return getValidateSerializableObjects() ? getSerializableObjectFilter() : null;
+  }
+
+  boolean getValidateSerializableObjects();
+
+  void setValidateSerializableObjects(boolean value);
+
+  String getSerializableObjectFilter();

Review comment:
       We also need a setter `setSerializableObjectFilter`.  I have not seen `serializable-object-filter` being set in the product or test.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(SerializationException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", new SerializableClass());
+    });
+
+    server2.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.get("key");
+      });
+      assertThat(thrown)
+          .isInstanceOf(SerializationException.class)
+          .hasCauseInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesNotThrow() {

Review comment:
       Do we need the `Not` in the test name? It does throw exception.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {

Review comment:
       Do we need the `Not` in the test name? It is propagated to the other server.




-- 
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



[GitHub] [geode] jchen21 commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
jchen21 commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768220800



##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/filter/SerializableObjectConfig.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.serialization.filter;
+
+/**
+ * Defines the configuration properties for serialization filtering in Geode.
+ */
+public interface SerializableObjectConfig {
+
+  default String getSerializableObjectFilterIfEnabled() {
+    return getValidateSerializableObjects() ? getSerializableObjectFilter() : null;
+  }
+
+  boolean getValidateSerializableObjects();
+
+  void setValidateSerializableObjects(boolean value);
+
+  String getSerializableObjectFilter();

Review comment:
       Maybe I am missing something. No usage is found for `DistributionConfig.setSerializableObjectFilter` or `DistributionConfigImpl.setSerializableObjectFilter`. It is true that there are lots of test that are setting `serializable-object-filter`. But somehow those tests bypass the API `DistributionConfig.setSerializableObjectFilter` by doing a `Properties.put`. I am OK that we address the concern later though.




-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768191777



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {

Review comment:
       It's correct as is.
   
   I used `NotPropagated` because the region entry is not propagated to the other server's cache. If you do a region.get(key) operation, you'll find that it's not present in the other server's cache. The message was sent but it failed to update the remote server because the value could not be deserialized.
   
   It would have been propagated to the remote server but the CacheListener.afterCreate attempts to deserialize. This fails and causes the Create message/operation to abort. The test then attempts the region.get(key) as validation that the key/value pair is definitely not present.




-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768188273



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(SerializationException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", new SerializableClass());
+    });
+
+    server2.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.get("key");
+      });
+      assertThat(thrown)
+          .isInstanceOf(SerializationException.class)
+          .hasCauseInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesNotThrow() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(IOException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.put(new SerializableClass(), new SerializableClass());
+      });
+      assertThat(thrown).isInstanceOf(InternalGemFireException.class);

Review comment:
       In wrapped in a couple onion layers. Here's the full stack if I comment out the catchThrowable in this test:
   ```
   While invoking org.apache.geode.cache.ValidateSerializableObjectsDistributedTest$$Lambda$283/429359848.run in VM 0 running on Host 192.168.254.12 with 4 VMs
   org.apache.geode.test.dunit.RMIException: While invoking org.apache.geode.cache.ValidateSerializableObjectsDistributedTest$$Lambda$283/429359848.run in VM 0 running on Host 192.168.254.12 with 4 VMs
   	at org.apache.geode.test.dunit.VM.executeMethodOnObject(VM.java:631)
   	at org.apache.geode.test.dunit.VM.invoke(VM.java:448)
   	at org.apache.geode.cache.ValidateSerializableObjectsDistributedTest.nonAllowedDoesNotThrow(ValidateSerializableObjectsDistributedTest.java:152)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.apache.geode.test.dunit.rules.AbstractDistributedRule$1.evaluate(AbstractDistributedRule.java:59)
   	at org.apache.geode.test.dunit.rules.AbstractDistributedRule$1.evaluate(AbstractDistributedRule.java:59)
   	at org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder$1.evaluate(SerializableTemporaryFolder.java:130)
   	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
   	at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
   	at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
   	at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
   	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
   	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
   	at org.junit.runner.JUnitCore.run(JUnitCore.java:115)
   	at org.junit.vintage.engine.execution.RunnerExecutor.execute(RunnerExecutor.java:43)
   	at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
   	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
   	at java.util.Iterator.forEachRemaining(Iterator.java:116)
   	at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
   	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
   	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
   	at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
   	at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
   	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
   	at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
   	at org.junit.vintage.engine.VintageTestEngine.executeAllChildren(VintageTestEngine.java:82)
   	at org.junit.vintage.engine.VintageTestEngine.execute(VintageTestEngine.java:73)
   	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:108)
   	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:88)
   	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.lambda$execute$0(EngineExecutionOrchestrator.java:54)
   	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.withInterceptedStreams(EngineExecutionOrchestrator.java:67)
   	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:52)
   	at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:96)
   	at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:75)
   	at org.gradle.api.internal.tasks.testing.junitplatform.JUnitPlatformTestClassProcessor$CollectAllTestClassesExecutor.processAllTestClasses(JUnitPlatformTestClassProcessor.java:99)
   	at org.gradle.api.internal.tasks.testing.junitplatform.JUnitPlatformTestClassProcessor$CollectAllTestClassesExecutor.access$000(JUnitPlatformTestClassProcessor.java:79)
   	at org.gradle.api.internal.tasks.testing.junitplatform.JUnitPlatformTestClassProcessor.stop(JUnitPlatformTestClassProcessor.java:75)
   	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.stop(SuiteTestClassProcessor.java:61)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
   	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
   	at com.sun.proxy.$Proxy2.stop(Unknown Source)
   	at org.gradle.api.internal.tasks.testing.worker.TestWorker.stop(TestWorker.java:133)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:182)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:164)
   	at org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:414)
   	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:64)
   	at org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:48)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:56)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: org.apache.geode.InternalGemFireException: unexpected exception on member 192.168.254.12(server2:31286)<v2>:41003
   	at org.apache.geode.distributed.internal.ReplyException.handleCause(ReplyException.java:98)
   	at org.apache.geode.internal.cache.DistributedCacheOperation.waitForAckIfNeeded(DistributedCacheOperation.java:777)
   	at org.apache.geode.internal.cache.DistributedCacheOperation._distribute(DistributedCacheOperation.java:661)
   	at org.apache.geode.internal.cache.DistributedCacheOperation.startOperation(DistributedCacheOperation.java:267)
   	at org.apache.geode.internal.cache.DistributedCacheOperation.distribute(DistributedCacheOperation.java:308)
   	at org.apache.geode.internal.cache.DistributedRegion.distributeUpdate(DistributedRegion.java:520)
   	at org.apache.geode.internal.cache.DistributedRegion.basicPutPart3(DistributedRegion.java:498)
   	at org.apache.geode.internal.cache.map.RegionMapPut.doAfterCompletionActions(RegionMapPut.java:307)
   	at org.apache.geode.internal.cache.map.AbstractRegionMapPut.doPut(AbstractRegionMapPut.java:185)
   	at org.apache.geode.internal.cache.map.AbstractRegionMapPut.runWhileLockedForCacheModification(AbstractRegionMapPut.java:119)
   	at org.apache.geode.internal.cache.map.RegionMapPut.runWhileLockedForCacheModification(RegionMapPut.java:161)
   	at org.apache.geode.internal.cache.map.AbstractRegionMapPut.put(AbstractRegionMapPut.java:169)
   	at org.apache.geode.internal.cache.AbstractRegionMap.basicPut(AbstractRegionMap.java:2045)
   	at org.apache.geode.internal.cache.LocalRegion.virtualPut(LocalRegion.java:5615)
   	at org.apache.geode.internal.cache.DistributedRegion.virtualPut(DistributedRegion.java:393)
   	at org.apache.geode.internal.cache.LocalRegion.virtualPut(LocalRegion.java:5593)
   	at org.apache.geode.internal.cache.LocalRegionDataView.putEntry(LocalRegionDataView.java:157)
   	at org.apache.geode.internal.cache.LocalRegion.basicPut(LocalRegion.java:5050)
   	at org.apache.geode.internal.cache.LocalRegion.validatedPut(LocalRegion.java:1646)
   	at org.apache.geode.internal.cache.LocalRegion.put(LocalRegion.java:1633)
   	at org.apache.geode.internal.cache.AbstractRegion.put(AbstractRegion.java:445)
   	at org.apache.geode.cache.ValidateSerializableObjectsDistributedTest.lambda$nonAllowedDoesNotThrow$bb17a952$1(ValidateSerializableObjectsDistributedTest.java:155)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.apache.geode.test.dunit.internal.MethodInvoker.executeObject(MethodInvoker.java:123)
   	at org.apache.geode.test.dunit.internal.RemoteDUnitVM.executeMethodOnObject(RemoteDUnitVM.java:78)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:357)
   	at sun.rmi.transport.Transport$1.run(Transport.java:200)
   	at sun.rmi.transport.Transport$1.run(Transport.java:197)
   	at java.security.AccessController.doPrivileged(Native Method)
   	at sun.rmi.transport.Transport.serviceCall(Transport.java:196)
   	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:573)
   	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:834)
   	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.lambda$run$0(TCPTransport.java:688)
   	at java.security.AccessController.doPrivileged(Native Method)
   	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:687)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	... 1 more
   Caused by: java.io.IOException: Could not create an instance of org.apache.geode.internal.cache.UpdateOperation$UpdateMessage .
   	at org.apache.geode.internal.serialization.internal.DSFIDSerializerImpl.invokeFromData(DSFIDSerializerImpl.java:330)
   	at org.apache.geode.internal.serialization.internal.DSFIDSerializerImpl.create(DSFIDSerializerImpl.java:377)
   	at org.apache.geode.internal.DSFIDFactory.create(DSFIDFactory.java:1029)
   	at org.apache.geode.internal.InternalDataSerializer.readDSFID(InternalDataSerializer.java:2416)
   	at org.apache.geode.internal.InternalDataSerializer.readDSFID(InternalDataSerializer.java:2430)
   	at org.apache.geode.internal.tcp.Connection.readMessage(Connection.java:3062)
   	at org.apache.geode.internal.tcp.Connection.processInputBuffer(Connection.java:2849)
   	at org.apache.geode.internal.tcp.Connection.readMessages(Connection.java:1660)
   	at org.apache.geode.internal.tcp.Connection.run(Connection.java:1500)
   	... 3 more
   Caused by: java.io.InvalidClassException: filter status: REJECTED
   	at java.io.ObjectInputStream.filterCheck(ObjectInputStream.java:1331)
   	at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1996)
   	at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1850)
   	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2160)
   	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1667)
   	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:503)
   	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:461)
   	at org.apache.geode.internal.InternalDataSerializer.readSerializable(InternalDataSerializer.java:2715)
   	at org.apache.geode.internal.InternalDataSerializer.basicReadObject(InternalDataSerializer.java:2659)
   	at org.apache.geode.DataSerializer.readObject(DataSerializer.java:2864)
   	at org.apache.geode.internal.cache.UpdateOperation$UpdateMessage.fromData(UpdateOperation.java:366)
   	at org.apache.geode.internal.serialization.internal.DSFIDSerializerImpl.invokeFromData(DSFIDSerializerImpl.java:317)
   	... 11 more
   ```




-- 
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



[GitHub] [geode] kirklund commented on a change in pull request #7189: GEODE-9758: Add serial filter config and utils

Posted by GitBox <gi...@apache.org>.
kirklund commented on a change in pull request #7189:
URL: https://github.com/apache/geode/pull/7189#discussion_r768189267



##########
File path: geode-core/src/distributedTest/java/org/apache/geode/cache/ValidateSerializableObjectsDistributedTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.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) {
+              // cache listener afterCreate causes all creates to deserialize the value which causes
+              // the tests to pass if serialization filter is configured
+              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 primitiveIsAllowed() {
+    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 nonAllowedIsNotPropagatedToOtherServer() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(SerializationException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      region.put("key", new SerializableClass());
+    });
+
+    server2.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.get("key");
+      });
+      assertThat(thrown)
+          .isInstanceOf(SerializationException.class)
+          .hasCauseInstanceOf(InvalidClassException.class);
+    });
+  }
+
+  @Test
+  public void nonAllowedDoesNotThrow() {
+    addIgnoredException(InvalidClassException.class);
+    addIgnoredException(IOException.class);
+
+    server1.invoke(() -> {
+      Region<Object, Object> region = server.get().getCache().getRegion("region");
+      Throwable thrown = catchThrowable(() -> {
+        region.put(new SerializableClass(), new SerializableClass());
+      });
+      assertThat(thrown).isInstanceOf(InternalGemFireException.class);

Review comment:
       That assertion can be expanded to cover the full stack trace:
   ```
         assertThat(thrown)
             .isInstanceOf(InternalGemFireException.class)
             .hasCauseInstanceOf(IOException.class)
             .hasRootCauseInstanceOf(InvalidClassException.class);
   ```




-- 
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