You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2021/10/21 21:22:37 UTC

[geode] branch support/1.14 updated: GEODE-9714: Add Shiro packages to sanctioned serializables (#7017)

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

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


The following commit(s) were added to refs/heads/support/1.14 by this push:
     new ff81dec  GEODE-9714: Add Shiro packages to sanctioned serializables (#7017)
ff81dec is described below

commit ff81dec73c6ed5e0a2de1f14868d008d189d066f
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Thu Oct 21 14:21:05 2021 -0700

    GEODE-9714: Add Shiro packages to sanctioned serializables (#7017)
    
    Adds all Shiro subpackages that contain exceptions to the list of
    sanctioned serializables.
    
    PROBLEM
    
    QueryConfigurationServiceConstraintsDistributedTest failed in
    CI when it tried to deserialize
    org.apache.shiro.session.StoppedSessionException.
    
    SOLUTION
    
    I did some research and I believe we should add all Shiro
    subpackages containing exceptions to the accept-list.
    
    (cherry picked from commit a2a80edf5f4d6ab6405edc05f82bb468c6161267)
---
 .../geode/internal/InternalDataSerializer.java     |  17 ++-
 ...lDataSerializerSerializationAcceptlistTest.java |  11 +-
 .../InternalDataSerializerShiroAcceptListTest.java | 161 +++++++++++++++++++++
 3 files changed, 178 insertions(+), 11 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
index 997b87f..dbab162 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
@@ -156,6 +156,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
    */
   @MakeNotStatic
   private static final Map<String, DataSerializer> classesToSerializers = new ConcurrentHashMap<>();
+
   /**
    * This list contains classes that Geode's classes subclass, such as antlr AST classes which are
    * used by our Object Query Language. It also contains certain classes that are DataSerializable
@@ -176,6 +177,11 @@ public abstract class InternalDataSerializer extends DataSerializer {
    * Do not add to this list unless absolutely necessary. Instead, put your classes either in the
    * sanctionedSerializables file for your module or in its excludedClasses file. Run
    * AnalyzeSerializables to generate the content for the file.
+   *
+   * <p>
+   * Syntax is documented by the javadocs of {@code ObjectInputFilter.Config.createFilter}. In
+   * Java 8, {@code ObjectInputFilter} is in package {@code sun.misc}. In Java 9 and above, it's
+   * in package {@code java.io}.
    */
   private static final String SANCTIONED_SERIALIZABLES_DEPENDENCIES_PATTERN =
       // Java
@@ -207,9 +213,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
           + ";org.apache.geode.internal.cache.tier.sockets.VersionedObjectList"
 
           // security services
-          + ";org.apache.shiro.*"
-          + ";org.apache.shiro.authz.*"
-          + ";org.apache.shiro.authc.*"
+          + ";org.apache.shiro.**"
 
           // export logs
           + ";org.apache.logging.log4j.Level"
@@ -220,12 +224,14 @@ public abstract class InternalDataSerializer extends DataSerializer {
           + ";com.healthmarketscience.rmiio.RemoteInputStream"
           + ";javax.rmi.ssl.SslRMIClientSocketFactory"
           + ";javax.net.ssl.SSLHandshakeException"
-          + ";javax.net.ssl.SSLException;sun.security.validator.ValidatorException"
+          + ";javax.net.ssl.SSLException"
+          + ";sun.security.validator.ValidatorException"
           + ";sun.security.provider.certpath.SunCertPathBuilderException"
 
           // geode-modules
           + ";org.apache.geode.modules.util.SessionCustomExpiry"
           + ";";
+
   private static final String serializationVersionTxt =
       System.getProperty(GeodeGlossary.GEMFIRE_PREFIX + "serializationVersion");
   /**
@@ -422,7 +428,8 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  private static void clearSerializationFilter() {
+  @VisibleForTesting
+  static void clearSerializationFilter() {
     serializationFilter = defaultSerializationFilter;
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/InternalDataSerializerSerializationAcceptlistTest.java b/geode-core/src/test/java/org/apache/geode/internal/InternalDataSerializerSerializationAcceptlistTest.java
index 61de4ba..81246da 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/InternalDataSerializerSerializationAcceptlistTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/InternalDataSerializerSerializationAcceptlistTest.java
@@ -29,7 +29,7 @@ import java.io.InvalidClassException;
 import java.io.Serializable;
 import java.util.Properties;
 
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -50,7 +50,7 @@ public class InternalDataSerializerSerializationAcceptlistTest {
 
   @BeforeClass
   public static void hasObjectInputFilter() {
-    assumeTrue("ObjectInputFilter is present in this JVM (post- 8.111)",
+    assumeTrue("ObjectInputFilter is present in this JVM",
         isClassAvailable("sun.misc.ObjectInputFilter") ||
             isClassAvailable("java.io.ObjectInputFilter"));
   }
@@ -62,10 +62,9 @@ public class InternalDataSerializerSerializationAcceptlistTest {
     properties = new Properties();
   }
 
-  @AfterClass
-  public static void clearDataSerializerFilter() {
-    InternalDataSerializer
-        .initializeSerializationFilter(new DistributionConfigImpl(new Properties()), emptySet());
+  @After
+  public void clearSerializationFilter() {
+    InternalDataSerializer.clearSerializationFilter();
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/internal/InternalDataSerializerShiroAcceptListTest.java b/geode-core/src/test/java/org/apache/geode/internal/InternalDataSerializerShiroAcceptListTest.java
new file mode 100644
index 0000000..60d8e9c
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/InternalDataSerializerShiroAcceptListTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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;
+
+import static java.util.Collections.emptySet;
+import static org.apache.geode.distributed.internal.DistributionConfig.VALIDATE_SERIALIZABLE_OBJECTS_NAME;
+import static org.apache.geode.internal.lang.ClassUtils.isClassAvailable;
+import static org.apache.geode.internal.serialization.KnownVersion.CURRENT;
+import static org.junit.Assume.assumeTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+import org.apache.shiro.ShiroException;
+import org.apache.shiro.authc.AuthenticationException;
+import org.apache.shiro.authz.AuthorizationException;
+import org.apache.shiro.codec.CodecException;
+import org.apache.shiro.config.ConfigurationException;
+import org.apache.shiro.crypto.UnknownAlgorithmException;
+import org.apache.shiro.dao.InvalidResourceUsageException;
+import org.apache.shiro.env.RequiredTypeException;
+import org.apache.shiro.io.SerializationException;
+import org.apache.shiro.ldap.UnsupportedAuthenticationMechanismException;
+import org.apache.shiro.session.SessionException;
+import org.apache.shiro.session.StoppedSessionException;
+import org.apache.shiro.subject.ExecutionException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.categories.SerializationTest;
+
+@Category({SecurityTest.class, SerializationTest.class})
+public class InternalDataSerializerShiroAcceptListTest {
+
+  @BeforeClass
+  public static void hasObjectInputFilter() {
+    assumeTrue("ObjectInputFilter is present in this JVM",
+        isClassAvailable("sun.misc.ObjectInputFilter") ||
+            isClassAvailable("java.io.ObjectInputFilter"));
+  }
+
+  @After
+  public void clearSerializationFilter() {
+    InternalDataSerializer.clearSerializationFilter();
+  }
+
+  @Test
+  public void acceptsAuthenticationException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new AuthenticationException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsAuthorizationException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new AuthorizationException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsCodecException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new CodecException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsConfigurationException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new ConfigurationException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsExecutionException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new ExecutionException("testing", new Exception("testing")),
+        propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsInstantiationException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new org.apache.shiro.util.InstantiationException("testing"),
+        propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsInvalidResourceUsageException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new InvalidResourceUsageException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsRequiredTypeException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new RequiredTypeException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsSerializationException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new SerializationException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsSessionException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new SessionException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsShiroException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new ShiroException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsStoppedSessionException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new StoppedSessionException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsUnknownAlgorithmException() throws IOException, ClassNotFoundException {
+    trySerializingObject(new UnknownAlgorithmException("testing"), propertiesWithoutFilter());
+  }
+
+  @Test
+  public void acceptsUnsupportedAuthenticationMechanismException()
+      throws IOException, ClassNotFoundException {
+    trySerializingObject(new UnsupportedAuthenticationMechanismException("testing"),
+        propertiesWithoutFilter());
+  }
+
+  private static Properties propertiesWithoutFilter() {
+    Properties properties = new Properties();
+    properties.setProperty(VALIDATE_SERIALIZABLE_OBJECTS_NAME, "true");
+
+    return properties;
+  }
+
+  private static void trySerializingObject(Object object, Properties properties)
+      throws IOException, ClassNotFoundException {
+    DistributionConfig distributionConfig = new DistributionConfigImpl(properties);
+    InternalDataSerializer.initializeSerializationFilter(distributionConfig, emptySet());
+    HeapDataOutputStream outputStream = new HeapDataOutputStream(CURRENT);
+
+    DataSerializer.writeObject(object, outputStream);
+
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(outputStream.toByteArray());
+        DataInputStream dis = new DataInputStream(bais)) {
+      DataSerializer.readObject(dis);
+    }
+  }
+}