You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dubbo.apache.org by li...@apache.org on 2019/08/20 04:23:06 UTC

[dubbo] 01/01: Fix java serialization vulnerable issue by adding blacklist.

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

liujun pushed a commit to branch deserialization-security
in repository https://gitbox.apache.org/repos/asf/dubbo.git

commit 59d3fc548806c76d167c7bcfba67525cd802d349
Author: ken.lj <ke...@gmail.com>
AuthorDate: Tue Aug 20 12:22:50 2019 +0800

    Fix java serialization vulnerable issue by adding blacklist.
---
 .../apache/dubbo/common/utils/SerialDetector.java  | 187 +++++++++++++++++++++
 .../dubbo/common/utils/SerialDetectorTest.java     |  44 +++++
 dubbo-common/src/test/resources/dubbo.properties   |   4 +-
 .../src/test/resources/security/invalid_data.ser   | Bin 0 -> 4002 bytes
 .../protocol/http/HttpInvokerServiceExporter.java  |  36 ++++
 .../dubbo/rpc/protocol/http/HttpProtocol.java      |   3 +-
 .../rpc/protocol/http/HttpRemoteInvocation.java    |   2 +
 .../serialize/java/CompactedObjectInputStream.java |   3 +-
 .../nativejava/NativeJavaObjectInput.java          |   3 +-
 9 files changed, 277 insertions(+), 5 deletions(-)

diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/utils/SerialDetector.java b/dubbo-common/src/main/java/org/apache/dubbo/common/utils/SerialDetector.java
new file mode 100644
index 0000000..042fda8
--- /dev/null
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/utils/SerialDetector.java
@@ -0,0 +1,187 @@
+/*
+ * 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.dubbo.common.utils;
+
+import org.apache.dubbo.common.config.Configuration;
+import org.apache.dubbo.common.config.Environment;
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.InvalidClassException;
+import java.io.ObjectInputStream;
+import java.io.ObjectStreamClass;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static java.util.Objects.requireNonNull;
+
+public class SerialDetector extends ObjectInputStream {
+
+    private static final Logger logger = LoggerFactory.getLogger(SerialDetector.class);
+
+    private static final BlacklistConfiguration configuration = new BlacklistConfiguration();
+
+    /**
+     * Wrapper Constructor.
+     *
+     * @param inputStream The original InputStream, used by your service to receive serialized objects
+     * @throws java.io.IOException   File I/O exception
+     * @throws IllegalStateException Invalid configuration exception
+     */
+    public SerialDetector(final InputStream inputStream) throws IOException {
+        super(inputStream);
+    }
+
+    @Override
+    protected Class<?> resolveClass(final ObjectStreamClass serialInput) throws IOException, ClassNotFoundException {
+        // Enforce SerialDetector's blacklist
+        Iterable<Pattern> blacklistIterable = configuration.blacklist();
+        if (blacklistIterable == null) {
+            return super.resolveClass(serialInput);
+        }
+
+        for (Pattern blackPattern : blacklistIterable) {
+            Matcher blackMatcher = blackPattern.matcher(serialInput.getName());
+
+            if (blackMatcher.find()) {
+                if (!configuration.shouldCheck()) {
+                    // Reporting mode
+                    logger.info(String.format("Blacklist match: '%s'", serialInput.getName()));
+                } else {
+                    // Blocking mode
+                    logger.error(String.format("Blocked by blacklist '%s'. Match found for '%s'", new Object[]{blackPattern.pattern(), serialInput.getName()}));
+                    throw new InvalidClassException(serialInput.getName(), "Class blocked from deserialization (blacklist)");
+                }
+            }
+        }
+
+        return super.resolveClass(serialInput);
+    }
+
+    static final class BlacklistConfiguration {
+        private static final String DUBBO_SECURITY_SERIALIZATION_CHECK = "dubbo.security.serialization.check";
+        private static final String DUBBO_SECURITY_SERIALIZATION_BLACKLIST = "dubbo.security.serialization.blacklist";
+        private static final String DUBBO_SECURITY_SERIALIZATION_BLACKLIST_FILE = "dubbo.registry.serialization.blacklist.file";
+
+        private boolean check;
+        private PatternList blacklistPattern;
+
+        BlacklistConfiguration() {
+            try {
+                Configuration config = Environment.getInstance().getConfiguration();
+                check = Boolean.valueOf(config.getString(DUBBO_SECURITY_SERIALIZATION_CHECK, "false"));
+                String blacklist = config.getString(DUBBO_SECURITY_SERIALIZATION_BLACKLIST);
+                if (StringUtils.isEmpty(blacklist)) {
+                    String blacklistFile = config.getString(DUBBO_SECURITY_SERIALIZATION_BLACKLIST_FILE);
+                    if (StringUtils.isNotEmpty(blacklistFile)) {
+                        blacklist = loadBlacklistFile(blacklistFile);
+                    }
+                }
+                blacklistPattern = new PatternList(CommonConstants.COMMA_SPLIT_PATTERN.split(blacklist));
+            } catch (Throwable t) {
+                logger.warn("Failed to initialize the Serialization Security Checker component!", t);
+            }
+        }
+
+        Iterable<Pattern> blacklist() {
+            return blacklistPattern;
+        }
+
+        boolean shouldCheck() {
+            return check;
+        }
+
+        private String loadBlacklistFile(String fileName) {
+            StringBuilder blacklist = new StringBuilder();
+            InputStream is = null;
+
+            File file = new File(fileName);
+            if (file.exists()) {
+                try {
+                    is = new FileInputStream(fileName);
+                } catch (Throwable e) {
+                    logger.warn("Failed to load " + fileName + " file " + e.getMessage(), e);
+                }
+            } else {
+                is = this.getClass().getClassLoader().getResourceAsStream(fileName);
+            }
+
+            try {
+                BufferedReader reader = new BufferedReader(new InputStreamReader(is));
+                String line = null;
+                while ((line = reader.readLine()) != null) {
+                    blacklist.append(line);
+                    blacklist.append(",");
+                }
+            } catch (Throwable e) {
+                logger.warn("Failed to read from file " + fileName + e.getMessage(), e);
+            }
+            return blacklist.toString();
+        }
+    }
+
+    static final class PatternList implements Iterable<Pattern> {
+        private final Pattern[] patterns;
+
+        PatternList(final String... regExps) {
+
+            requireNonNull(regExps, "regExps");
+
+            this.patterns = new Pattern[regExps.length];
+            for (int i = 0; i < regExps.length; i++) {
+                patterns[i] = Pattern.compile(regExps[i]);
+            }
+        }
+
+        @Override
+        public Iterator<Pattern> iterator() {
+            return new Iterator<Pattern>() {
+                int index = 0;
+
+                @Override
+                public boolean hasNext() {
+                    return index < patterns.length;
+                }
+
+                @Override
+                public Pattern next() {
+                    return patterns[index++];
+                }
+
+                @Override
+                public void remove() {
+                    throw new UnsupportedOperationException("remove");
+                }
+            };
+        }
+
+        @Override
+        public String toString() {
+            return Arrays.toString(patterns);
+        }
+
+    }
+}
\ No newline at end of file
diff --git a/dubbo-common/src/test/java/org/apache/dubbo/common/utils/SerialDetectorTest.java b/dubbo-common/src/test/java/org/apache/dubbo/common/utils/SerialDetectorTest.java
new file mode 100644
index 0000000..88eeae7
--- /dev/null
+++ b/dubbo-common/src/test/java/org/apache/dubbo/common/utils/SerialDetectorTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.dubbo.common.utils;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.InvalidClassException;
+import java.io.ObjectInputStream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class SerialDetectorTest {
+    @Test
+    public void testBlacklisted() throws Exception {
+        try (ObjectInputStream stream = new SerialDetector(getClass().getResourceAsStream("/security/invalid_data.ser"))) {
+            try {
+                stream.readObject();
+                fail();
+            } catch (InvalidClassException expected) {
+                assertTrue(expected.getMessage().contains("blocked"));
+                assertTrue(expected.getMessage().contains("blacklist"));
+                assertEquals(expected.classname, "org.hibernate.engine.spi.TypedValue");
+            } catch (ClassNotFoundException e) {
+                fail(e.getMessage());
+            }
+        }
+    }
+}
diff --git a/dubbo-common/src/test/resources/dubbo.properties b/dubbo-common/src/test/resources/dubbo.properties
index 15936f9..b505a9e 100644
--- a/dubbo-common/src/test/resources/dubbo.properties
+++ b/dubbo-common/src/test/resources/dubbo.properties
@@ -15,4 +15,6 @@
 # limitations under the License.
 #
 
-dubbo=properties
\ No newline at end of file
+dubbo=properties
+dubbo.security.serialization.blacklist=org.hibernate.engine.spi.TypedValue
+dubbo.security.serialization.check=true
\ No newline at end of file
diff --git a/dubbo-common/src/test/resources/security/invalid_data.ser b/dubbo-common/src/test/resources/security/invalid_data.ser
new file mode 100644
index 0000000..2a3159d
Binary files /dev/null and b/dubbo-common/src/test/resources/security/invalid_data.ser differ
diff --git a/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpInvokerServiceExporter.java b/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpInvokerServiceExporter.java
new file mode 100644
index 0000000..7a910f6
--- /dev/null
+++ b/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpInvokerServiceExporter.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dubbo.rpc.protocol.http;
+
+import org.apache.dubbo.common.utils.SerialDetector;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+
+/**
+ * Wrap InputStream to avoid well-known java serialization security issue.
+ */
+public class HttpInvokerServiceExporter extends org.springframework.remoting.httpinvoker.HttpInvokerServiceExporter {
+
+    @Override
+    protected ObjectInputStream createObjectInputStream(InputStream is) throws IOException {
+        ObjectInputStream inputStream = super.createObjectInputStream(is);
+        return new SerialDetector(inputStream);
+    }
+
+}
diff --git a/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpProtocol.java b/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpProtocol.java
index c8f6e58..0cf68ed 100644
--- a/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpProtocol.java
+++ b/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpProtocol.java
@@ -17,8 +17,8 @@
 package org.apache.dubbo.rpc.protocol.http;
 
 import org.apache.dubbo.common.URL;
-import org.apache.dubbo.common.utils.StringUtils;
 import org.apache.dubbo.common.Version;
+import org.apache.dubbo.common.utils.StringUtils;
 import org.apache.dubbo.remoting.Constants;
 import org.apache.dubbo.remoting.http.HttpBinder;
 import org.apache.dubbo.remoting.http.HttpHandler;
@@ -33,7 +33,6 @@ import org.aopalliance.intercept.MethodInvocation;
 import org.springframework.remoting.RemoteAccessException;
 import org.springframework.remoting.httpinvoker.HttpComponentsHttpInvokerRequestExecutor;
 import org.springframework.remoting.httpinvoker.HttpInvokerProxyFactoryBean;
-import org.springframework.remoting.httpinvoker.HttpInvokerServiceExporter;
 import org.springframework.remoting.httpinvoker.SimpleHttpInvokerRequestExecutor;
 import org.springframework.remoting.support.RemoteInvocation;
 import org.springframework.remoting.support.RemoteInvocationFactory;
diff --git a/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpRemoteInvocation.java b/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpRemoteInvocation.java
index 2656172..56cbd84 100644
--- a/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpRemoteInvocation.java
+++ b/dubbo-rpc/dubbo-rpc-http/src/main/java/org/apache/dubbo/rpc/protocol/http/HttpRemoteInvocation.java
@@ -56,4 +56,6 @@ public class HttpRemoteInvocation extends RemoteInvocation {
 
         }
     }
+
+
 }
diff --git a/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/java/CompactedObjectInputStream.java b/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/java/CompactedObjectInputStream.java
index e881774..e977f5c 100644
--- a/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/java/CompactedObjectInputStream.java
+++ b/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/java/CompactedObjectInputStream.java
@@ -18,6 +18,7 @@ package org.apache.dubbo.common.serialize.java;
 
 
 import org.apache.dubbo.common.utils.ClassUtils;
+import org.apache.dubbo.common.utils.SerialDetector;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -33,7 +34,7 @@ public class CompactedObjectInputStream extends ObjectInputStream {
     private ClassLoader mClassLoader;
 
     public CompactedObjectInputStream(InputStream in) throws IOException {
-        this(in, Thread.currentThread().getContextClassLoader());
+        this(new SerialDetector(in), Thread.currentThread().getContextClassLoader());
     }
 
     public CompactedObjectInputStream(InputStream in, ClassLoader cl) throws IOException {
diff --git a/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/nativejava/NativeJavaObjectInput.java b/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/nativejava/NativeJavaObjectInput.java
index c791250..6bd42f1 100644
--- a/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/nativejava/NativeJavaObjectInput.java
+++ b/dubbo-serialization/dubbo-serialization-jdk/src/main/java/org/apache/dubbo/common/serialize/nativejava/NativeJavaObjectInput.java
@@ -19,6 +19,7 @@ package org.apache.dubbo.common.serialize.nativejava;
 
 import org.apache.dubbo.common.serialize.ObjectInput;
 import org.apache.dubbo.common.utils.Assert;
+import org.apache.dubbo.common.utils.SerialDetector;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -33,7 +34,7 @@ public class NativeJavaObjectInput implements ObjectInput {
     private final ObjectInputStream inputStream;
 
     public NativeJavaObjectInput(InputStream is) throws IOException {
-        this(new ObjectInputStream(is));
+        this(new ObjectInputStream(new SerialDetector(is)));
     }
 
     protected NativeJavaObjectInput(ObjectInputStream is) {