You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bd...@apache.org on 2019/11/06 20:12:57 UTC

[cassandra] branch cassandra-3.0 updated: Validate value sizes in LegacyLayout

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

bdeggleston pushed a commit to branch cassandra-3.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
     new 53f604d  Validate value sizes in LegacyLayout
53f604d is described below

commit 53f604dc1789a800dbcbc3c8aee77f8f36b8b5db
Author: Blake Eggleston <bd...@gmail.com>
AuthorDate: Wed Oct 23 15:40:07 2019 -0700

    Validate value sizes in LegacyLayout
    
    Patch by Blake Eggleston; Reviewed by Benedict Elliot Smith and Sam Tunnicliffe for CASSANDRA-15373
---
 CHANGES.txt                                        |   1 +
 src/java/org/apache/cassandra/db/LegacyLayout.java |  21 ++
 .../apache/cassandra/db/marshal/AbstractType.java  |   8 +
 .../org/apache/cassandra/db/LegacyLayoutTest.java  |   1 -
 .../cassandra/db/LegacyLayoutValidationTest.java   | 221 +++++++++++++++++++++
 5 files changed, 251 insertions(+), 1 deletion(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 89c5610..08e95ed 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.20
+ * validate value sizes in LegacyLayout (CASSANDRA-15373)
  * Ensure that tracing doesn't break connections in 3.x/4.0 mixed mode by default (CASSANDRA-15385)
  * Make sure index summary redistribution does not start when compactions are paused (CASSANDRA-15265)
  * Ensure legacy rows have primary key livenessinfo when they contain illegal cells (CASSANDRA-15365)
diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java
index 6e93d08..c2d715d 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -198,6 +198,10 @@ public abstract class LegacyLayout
         }
 
         ByteBuffer collectionElement = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size() + 1) : null;
+        if (collectionElement != null && def.type instanceof CollectionType)
+        {
+            ((CollectionType)def.type).nameComparator().validateIfFixedSize(collectionElement);
+        }
 
         // Note that because static compact columns are translated to static defs in the new world order, we need to force a static
         // clustering if the definition is static (as it might not be in this case).
@@ -223,6 +227,7 @@ public abstract class LegacyLayout
         {
             // The non compound case is a lot easier, in that there is no EOC nor collection to worry about, so dealing
             // with that first.
+            metadata.comparator.subtype(0).validateIfFixedSize(bound);
             return new LegacyBound(isStart ? Slice.Bound.inclusiveStartOf(bound) : Slice.Bound.inclusiveEndOf(bound), false, null);
         }
 
@@ -231,6 +236,10 @@ public abstract class LegacyLayout
         boolean isStatic = metadata.isCompound() && CompositeType.isStaticName(bound);
         List<ByteBuffer> components = CompositeType.splitName(bound);
         byte eoc = CompositeType.lastEOC(bound);
+        for (int i=0; i<Math.min(clusteringSize, components.size()); i++)
+        {
+            metadata.comparator.subtype(i).validateIfFixedSize(components.get(i));
+        }
 
         // if the bound we have decoded is static, 2.2 format requires there to be N empty clusterings
         assert !isStatic ||
@@ -438,6 +447,11 @@ public abstract class LegacyLayout
                                     ? CompositeType.splitName(value)
                                     : Collections.singletonList(value);
 
+        for (int i=0; i<Math.min(csize, components.size()); i++)
+        {
+            AbstractType<?> type = metadata.comparator.subtype(i);
+            type.validateIfFixedSize(components.get(i));
+        }
         return new Clustering(components.subList(0, Math.min(csize, components.size())).toArray(new ByteBuffer[csize]));
     }
 
@@ -769,6 +783,7 @@ public abstract class LegacyLayout
                     continue;
 
                 foundOne = true;
+                cell.name.column.type.validateIfFixedSize(cell.value);
                 builder.addCell(new BufferCell(cell.name.column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, null));
             }
             else
@@ -1420,6 +1435,7 @@ public abstract class LegacyLayout
                         if (!helper.includes(path))
                             return true;
                     }
+                    column.type.validateIfFixedSize(cell.value);
                     Cell c = new BufferCell(column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, path);
                     if (!helper.isDropped(c, column.isComplex()))
                         builder.addCell(c);
@@ -1592,6 +1608,11 @@ public abstract class LegacyLayout
             this.collectionElement = collectionElement;
         }
 
+        public static LegacyCellName create(Clustering clustering, ColumnDefinition column)
+        {
+            return new LegacyCellName(clustering, column, null);
+        }
+
         public ByteBuffer encode(CFMetaData metadata)
         {
             return encodeCellName(metadata, clustering, column == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : column.name.bytes, collectionElement);
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 20062bd..72bfa66 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -382,6 +382,14 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>
         return -1;
     }
 
+    public void validateIfFixedSize(ByteBuffer value)
+    {
+        if (valueLengthIfFixed() < 0)
+            return;
+
+        validate(value);
+    }
+
     // This assumes that no empty values are passed
     public void writeValue(ByteBuffer value, DataOutputPlus out) throws IOException
     {
diff --git a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
index 77197d3..7f3b424 100644
--- a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
+++ b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
@@ -370,5 +370,4 @@ public class LegacyLayoutTest
         LegacyLayout.fromUnfilteredRowIterator(null, p.unfilteredIterator());
         LegacyLayout.serializedSizeAsLegacyPartition(null, p.unfilteredIterator(), VERSION_21);
     }
-
 }
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/db/LegacyLayoutValidationTest.java b/test/unit/org/apache/cassandra/db/LegacyLayoutValidationTest.java
new file mode 100644
index 0000000..068d2a2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/LegacyLayoutValidationTest.java
@@ -0,0 +1,221 @@
+/*
+ * 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.cassandra.db;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.LegacyLayout.LegacyCell;
+import org.apache.cassandra.db.LegacyLayout.LegacyCellName;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.serializers.MarshalException;
+
+import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
+
+public class LegacyLayoutValidationTest
+{
+    static final String KEYSPACE = "ks";
+
+    private static final CFMetaData FIXED = CFMetaData.Builder.create("ks", "cf")
+                                                              .addPartitionKey("k", Int32Type.instance)
+                                                              .addClusteringColumn("c1", Int32Type.instance)
+                                                              .addClusteringColumn("c2", Int32Type.instance)
+                                                              .addRegularColumn("v1", Int32Type.instance)
+                                                              .addRegularColumn("v2", Int32Type.instance)
+                                                              .build();
+
+    private static final CFMetaData COMPACT_FIXED = CFMetaData.Builder.create("ks", "cf", true, false, false)
+                                                                      .addPartitionKey("k", Int32Type.instance)
+                                                                      .addClusteringColumn("c", Int32Type.instance)
+                                                                      .addRegularColumn("v", Int32Type.instance)
+                                                                      .build();
+
+    private static final CFMetaData VARIABLE = CFMetaData.Builder.create("ks", "cf")
+                                                                 .addPartitionKey("k", Int32Type.instance)
+                                                                 .addClusteringColumn("c1", UTF8Type.instance)
+                                                                 .addClusteringColumn("c2", UTF8Type.instance)
+                                                                 .addRegularColumn("v1", UTF8Type.instance)
+                                                                 .addRegularColumn("v2", UTF8Type.instance)
+                                                                 .build();
+
+    private static final CFMetaData COMPACT_VARIABLE = CFMetaData.Builder.create("ks", "cf", true, false, false)
+                                                                         .addPartitionKey("k", Int32Type.instance)
+                                                                         .addClusteringColumn("c", UTF8Type.instance)
+                                                                         .addRegularColumn("v", UTF8Type.instance)
+                                                                         .build();
+
+    @Test
+    public void fixedClusteringSuccess()
+    {
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1), Int32Type.instance.decompose(2));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(FIXED, clustering);
+        LegacyLayout.decodeClustering(FIXED, serialized);
+    }
+
+    @Test (expected = MarshalException.class)
+    public void fixedClusteringFailure()
+    {
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1), hexToBytes("07000000000001"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(FIXED, clustering);
+        LegacyLayout.decodeClustering(FIXED, serialized);
+    }
+
+    @Test
+    public void variableClusteringSuccess()
+    {
+        Clustering clustering = new Clustering(UTF8Type.instance.decompose("one"), UTF8Type.instance.decompose("two,three"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(VARIABLE, clustering);
+        LegacyLayout.decodeClustering(VARIABLE, serialized);
+    }
+
+    @Test
+    public void fixedCompactClusteringSuccess()
+    {
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(2));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(COMPACT_FIXED, clustering);
+        LegacyLayout.decodeClustering(COMPACT_FIXED, serialized);
+    }
+
+    @Test (expected = MarshalException.class)
+    public void fixedCompactClusteringFailure()
+    {
+        Clustering clustering = new Clustering(hexToBytes("07000000000001"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(COMPACT_FIXED, clustering);
+        LegacyLayout.decodeClustering(COMPACT_FIXED, serialized);
+    }
+
+    @Test
+    public void variableCompactClusteringSuccess()
+    {
+        Clustering clustering = new Clustering(UTF8Type.instance.decompose("two,three"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(COMPACT_VARIABLE, clustering);
+        LegacyLayout.decodeClustering(COMPACT_VARIABLE, serialized);
+    }
+
+    @Test
+    public void fixedBoundSuccess()
+    {
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1), Int32Type.instance.decompose(2));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(FIXED, clustering);
+        LegacyLayout.decodeSliceBound(FIXED, serialized, true);
+    }
+
+    @Test (expected = MarshalException.class)
+    public void fixedBoundFailure()
+    {
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1), hexToBytes("07000000000001"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(FIXED, clustering);
+        LegacyLayout.decodeSliceBound(FIXED, serialized, true);
+    }
+
+    @Test
+    public void variableBoundSuccess()
+    {
+        Clustering clustering = new Clustering(UTF8Type.instance.decompose("one"), UTF8Type.instance.decompose("two,three"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(VARIABLE, clustering);
+        LegacyLayout.decodeSliceBound(VARIABLE, serialized, true);
+    }
+
+    @Test
+    public void fixedCompactBoundSuccess()
+    {
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(COMPACT_FIXED, clustering);
+        LegacyLayout.decodeSliceBound(COMPACT_FIXED, serialized, true);
+    }
+
+    @Test (expected = MarshalException.class)
+    public void fixedCompactBoundFailure()
+    {
+        Clustering clustering = new Clustering(hexToBytes("07000000000001"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(COMPACT_FIXED, clustering);
+        LegacyLayout.decodeSliceBound(COMPACT_FIXED, serialized, true);
+    }
+
+    @Test
+    public void variableCompactBoundSuccess()
+    {
+        Clustering clustering = new Clustering(UTF8Type.instance.decompose("one"));
+        ByteBuffer serialized = LegacyLayout.encodeClustering(COMPACT_VARIABLE, clustering);
+        LegacyLayout.decodeSliceBound(COMPACT_VARIABLE, serialized, true);
+    }
+
+    private static LegacyCell cell(CFMetaData cfm, Clustering clustering, String name, ByteBuffer value) throws UnknownColumnException
+    {
+        ColumnDefinition definition = cfm.getColumnDefinition(new ColumnIdentifier(name, false));
+
+        ByteBuffer cellName = LegacyCellName.create(clustering, definition).encode(cfm);
+        return LegacyCell.regular(cfm, null, cellName, value, 0);
+
+    }
+
+    @Test
+    public void fixedValueSuccess() throws Throwable
+    {
+        DecoratedKey dk = DatabaseDescriptor.getPartitioner().decorateKey(Int32Type.instance.decompose(1000000));
+        LegacyLayout.LegacyDeletionInfo deletionInfo = LegacyLayout.LegacyDeletionInfo.live();
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1), Int32Type.instance.decompose(2));
+        Iterator<LegacyCell> cells = Iterators.forArray(cell(FIXED, clustering, "v1", Int32Type.instance.decompose(3)),
+                                                        cell(FIXED, clustering, "v2", Int32Type.instance.decompose(4)));
+        try (UnfilteredRowIterator iter = LegacyLayout.toUnfilteredRowIterator(FIXED, dk, deletionInfo, cells))
+        {
+            while (iter.hasNext())
+                iter.next();
+        }
+    }
+
+    @Test (expected = MarshalException.class)
+    public void fixedValueFailure() throws Throwable
+    {
+        DecoratedKey dk = DatabaseDescriptor.getPartitioner().decorateKey(Int32Type.instance.decompose(1000000));
+        LegacyLayout.LegacyDeletionInfo deletionInfo = LegacyLayout.LegacyDeletionInfo.live();
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1), Int32Type.instance.decompose(2));
+        Iterator<LegacyCell> cells = Iterators.forArray(cell(FIXED, clustering, "v1", Int32Type.instance.decompose(3)),
+                                                        cell(FIXED, clustering, "v2", hexToBytes("0000")));
+        try (UnfilteredRowIterator iter = LegacyLayout.toUnfilteredRowIterator(FIXED, dk, deletionInfo, cells))
+        {
+            while (iter.hasNext())
+                iter.next();
+        }
+    }
+
+    @Test
+    public void variableValueSuccess() throws Throwable
+    {
+        DecoratedKey dk = DatabaseDescriptor.getPartitioner().decorateKey(Int32Type.instance.decompose(1000000));
+        LegacyLayout.LegacyDeletionInfo deletionInfo = LegacyLayout.LegacyDeletionInfo.live();
+        Clustering clustering = new Clustering(Int32Type.instance.decompose(1), Int32Type.instance.decompose(2));
+        Iterator<LegacyCell> cells = Iterators.forArray(cell(VARIABLE, clustering, "v1", UTF8Type.instance.decompose("3")),
+                                                        cell(VARIABLE, clustering, "v2", hexToBytes("0000")));
+        try (UnfilteredRowIterator iter = LegacyLayout.toUnfilteredRowIterator(VARIABLE, dk, deletionInfo, cells))
+        {
+            while (iter.hasNext())
+                iter.next();
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org