You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2016/09/27 12:03:55 UTC

cassandra git commit: Make Collections deserialization more robust

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 e9fe96f40 -> 6dc595dd2


Make Collections deserialization more robust

patch by Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-12618


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6dc595dd
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6dc595dd
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6dc595dd

Branch: refs/heads/cassandra-2.2
Commit: 6dc595dd2ebf8477b4775473d1d2417fa3a0fcfd
Parents: e9fe96f
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Sep 27 14:02:10 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Sep 27 14:02:10 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 ++
 .../apache/cassandra/db/marshal/UserType.java   |  8 ++--
 .../cassandra/serializers/ListSerializer.java   | 10 ++++-
 .../cassandra/serializers/MapSerializer.java    | 10 ++++-
 .../cassandra/serializers/SetSerializer.java    | 13 ++++++-
 .../cassandra/serializers/UTF8Serializer.java   |  3 ++
 .../validation/entities/CollectionsTest.java    | 41 ++++++++++++++++++++
 .../cql3/validation/entities/TupleTypeTest.java | 10 +++++
 .../cql3/validation/entities/UserTypesTest.java | 11 ++++++
 9 files changed, 101 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bc9fc5b..998849e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,7 @@
+2.2.9
+ * Make Collections deserialization more robust (CASSANDRA-12618)
+ 
+ 
 2.2.8
  * Fix exceptions when enabling gossip on nodes that haven't joined the ring (CASSANDRA-12253)
  * Fix authentication problem when invoking clqsh copy from a SOURCE command (CASSANDRA-12642)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/src/java/org/apache/cassandra/db/marshal/UserType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index 6102d26..187deeb 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
-import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.*;
 
@@ -28,8 +27,7 @@ import com.google.common.base.Objects;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.serializers.*;
-import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
@@ -123,7 +121,7 @@ public class UserType extends TupleType
                 return;
 
             if (input.remaining() < 4)
-                throw new MarshalException(String.format("Not enough bytes to read size of %dth field %s", i, fieldName(i)));
+                throw new MarshalException(String.format("Not enough bytes to read size of %dth field %s", i, fieldNameAsString(i)));
 
             int size = input.getInt();
 
@@ -132,7 +130,7 @@ public class UserType extends TupleType
                 continue;
 
             if (input.remaining() < size)
-                throw new MarshalException(String.format("Not enough bytes to read %dth field %s", i, fieldName(i)));
+                throw new MarshalException(String.format("Not enough bytes to read %dth field %s", i, fieldNameAsString(i)));
 
             ByteBuffer field = ByteBufferUtil.readBytes(input, size);
             types.get(i).validate(field);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/src/java/org/apache/cassandra/serializers/ListSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/ListSerializer.java b/src/java/org/apache/cassandra/serializers/ListSerializer.java
index aeee2b9..d2d0610 100644
--- a/src/java/org/apache/cassandra/serializers/ListSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/ListSerializer.java
@@ -84,7 +84,15 @@ public class ListSerializer<T> extends CollectionSerializer<List<T>>
         {
             ByteBuffer input = bytes.duplicate();
             int n = readCollectionSize(input, version);
-            List<T> l = new ArrayList<T>(n);
+
+            if (n < 0)
+                throw new MarshalException("The data cannot be deserialized as a list");
+
+            // If the received bytes are not corresponding to a list, n might be a huge number.
+            // In such a case we do not want to initialize the list with that size as it can result
+            // in an OOM (see CASSANDRA-12618). On the other hand we do not want to have to resize the list
+            // if we can avoid it, so we put a reasonable limit on the initialCapacity.
+            List<T> l = new ArrayList<T>(Math.min(n, 256));
             for (int i = 0; i < n; i++)
             {
                 // We can have nulls in lists that are used for IN values

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/src/java/org/apache/cassandra/serializers/MapSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/MapSerializer.java b/src/java/org/apache/cassandra/serializers/MapSerializer.java
index 8350f66..66831a3 100644
--- a/src/java/org/apache/cassandra/serializers/MapSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/MapSerializer.java
@@ -94,7 +94,15 @@ public class MapSerializer<K, V> extends CollectionSerializer<Map<K, V>>
         {
             ByteBuffer input = bytes.duplicate();
             int n = readCollectionSize(input, version);
-            Map<K, V> m = new LinkedHashMap<K, V>(n);
+
+            if (n < 0)
+                throw new MarshalException("The data cannot be deserialized as a map");
+
+            // If the received bytes are not corresponding to a map, n might be a huge number.
+            // In such a case we do not want to initialize the map with that initialCapacity as it can result
+            // in an OOM when put is called (see CASSANDRA-12618). On the other hand we do not want to have to resize
+            // the map if we can avoid it, so we put a reasonable limit on the initialCapacity.
+            Map<K, V> m = new LinkedHashMap<K, V>(Math.min(n, 256));
             for (int i = 0; i < n; i++)
             {
                 ByteBuffer kbb = readValue(input, version);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/src/java/org/apache/cassandra/serializers/SetSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/SetSerializer.java b/src/java/org/apache/cassandra/serializers/SetSerializer.java
index 21f5075..4aaf36a 100644
--- a/src/java/org/apache/cassandra/serializers/SetSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/SetSerializer.java
@@ -81,7 +81,16 @@ public class SetSerializer<T> extends CollectionSerializer<Set<T>>
         {
             ByteBuffer input = bytes.duplicate();
             int n = readCollectionSize(input, version);
-            Set<T> l = new LinkedHashSet<T>(n);
+
+            if (n < 0)
+                throw new MarshalException("The data cannot be deserialized as a set");
+
+            // If the received bytes are not corresponding to a set, n might be a huge number.
+            // In such a case we do not want to initialize the set with that initialCapacity as it can result
+            // in an OOM when add is called (see CASSANDRA-12618). On the other hand we do not want to have to resize
+            // the set if we can avoid it, so we put a reasonable limit on the initialCapacity.
+            Set<T> l = new LinkedHashSet<T>(Math.min(n, 256));
+
             for (int i = 0; i < n; i++)
             {
                 ByteBuffer databb = readValue(input, version);
@@ -94,7 +103,7 @@ public class SetSerializer<T> extends CollectionSerializer<Set<T>>
         }
         catch (BufferUnderflowException e)
         {
-            throw new MarshalException("Not enough bytes to read a list");
+            throw new MarshalException("Not enough bytes to read a set");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/src/java/org/apache/cassandra/serializers/UTF8Serializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/UTF8Serializer.java b/src/java/org/apache/cassandra/serializers/UTF8Serializer.java
index e3ea2d5..7c41b94 100644
--- a/src/java/org/apache/cassandra/serializers/UTF8Serializer.java
+++ b/src/java/org/apache/cassandra/serializers/UTF8Serializer.java
@@ -52,6 +52,9 @@ public class UTF8Serializer extends AbstractTextSerializer
         // buf has already been sliced/duplicated.
         static boolean validate(ByteBuffer buf)
         {
+            if (buf == null)
+                return false;
+
             buf = buf.slice();
             int b = 0;
             State state = State.START;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
index 6266fe7..115b755 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
@@ -607,4 +607,45 @@ public class CollectionsTest extends CQLTester
         assertRows(execute("select s_list from %s where k1='a'"), row(list(0)));
     }
 
+    @Test
+    public void testInvalidInputForList() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, l list<text>)");
+        assertInvalidMessage("Not enough bytes to read a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("Not enough bytes to read a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, Long.MAX_VALUE);
+        assertInvalidMessage("Not enough bytes to read a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, "");
+        assertInvalidMessage("The data cannot be deserialized as a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, -1);
+    }
+
+    @Test
+    public void testInvalidInputForSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, s set<text>)");
+        assertInvalidMessage("Not enough bytes to read a set",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("String didn't validate.",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, Long.MAX_VALUE);
+        assertInvalidMessage("Not enough bytes to read a set",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, "");
+        assertInvalidMessage("The data cannot be deserialized as a set",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, -1);
+    }
+
+    @Test
+    public void testInvalidInputForMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, m map<text, text>)");
+        assertInvalidMessage("Not enough bytes to read a map",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("String didn't validate.",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, Long.MAX_VALUE);
+        assertInvalidMessage("Not enough bytes to read a map",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, "");
+        assertInvalidMessage("The data cannot be deserialized as a map",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, -1);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
index 0e7084f..0783dd1 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -193,4 +193,14 @@ public class TupleTypeTest extends CQLTester
                    row(0, 0, "b"),
                    row(0, 0, "c"));
     }
+
+    @Test
+    public void testInvalidInputForTuple() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, t tuple<text, text>)");
+        assertInvalidMessage("Not enough bytes to read 0th component",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("Not enough bytes to read 0th component",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, Long.MAX_VALUE);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc595dd/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index cd87eda..3803e5c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -45,6 +45,17 @@ public class UserTypesTest extends CQLTester
     }
 
     @Test
+    public void testInvalidInputForUserType() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, t frozen<" + myType + ">)");
+        assertInvalidMessage("Not enough bytes to read 0th field f",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("Not enough bytes to read 0th field f",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, Long.MAX_VALUE);
+    }
+
+    @Test
     public void testCassandra8105() throws Throwable
     {
         String ut1 = createType("CREATE TYPE %s (a int, b int)");