You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2018/09/14 10:26:03 UTC

[4/9] cassandra git commit: Collection Deletions for Dropped Columns in 2.1/3.0 mixed-mode can delete rows

Collection Deletions for Dropped Columns in 2.1/3.0 mixed-mode can delete rows

patch by Benedict; reviewed by Sylvain Lebresne and Aleksey Yeschenko for CASSANDRA-14749


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

Branch: refs/heads/cassandra-3.0
Commit: 06c55f779ae68de98cce531e0b78be5716849003
Parents: 68dbeb3
Author: Benedict Elliott Smith <be...@apple.com>
Authored: Thu Sep 13 17:09:58 2018 +0100
Committer: Benedict Elliott Smith <be...@apple.com>
Committed: Fri Sep 14 11:17:59 2018 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/db/LegacyLayout.java   |  5 +++
 .../apache/cassandra/db/LegacyLayoutTest.java   | 39 ++++++++++++++++++++
 3 files changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/06c55f77/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 037e2a8..195c97c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.18
+ * Fix corrupted collection deletions for dropped columns in 3.0 <-> 2.{1,2} messages (CASSANDRA-14568)
  * Fix corrupted static collection deletions in 3.0 <-> 2.{1,2} messages (CASSANDRA-14568)
  * Handle failures in parallelAllSSTableOperation (cleanup/upgradesstables/etc) (CASSANDRA-14657)
  * Improve TokenMetaData cache populating performance avoid long locking (CASSANDRA-14660)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/06c55f77/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java
index 2115c7d..ecc507e 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -215,6 +215,11 @@ public abstract class LegacyLayout
             // pop the collection name from the back of the list of clusterings
             ByteBuffer collectionNameBytes = components.remove(clusteringSize);
             collectionName = metadata.getColumnDefinition(collectionNameBytes);
+            if (collectionName == null) {
+                collectionName = metadata.getDroppedColumnDefinition(collectionNameBytes, isStatic);
+                if (collectionName == null)
+                    throw new RuntimeException("Unknown collection column " + UTF8Type.instance.getString(collectionNameBytes) + " during deserialization");
+            }
         }
 
         boolean isInclusive;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/06c55f77/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
index ce818c0..2bed35e 100644
--- a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
+++ b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
@@ -252,4 +252,43 @@ public class LegacyLayoutTest
         }
     }
 
+    @Test
+    public void testStaticRangeTombstoneRoundTripDroppedColumn() throws Throwable
+    {
+        // this variant of the bug deletes a row with the same clustering key value as the name of the static collection
+        QueryProcessor.executeInternal(String.format("CREATE TABLE \"%s\".legacy_static_rt_rt_dc (pk int, ck1 text, v int, s set<text>, primary key (pk, ck1))", KEYSPACE));
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        CFMetaData table = keyspace.getColumnFamilyStore("legacy_static_rt_rt_dc").metadata;
+        ColumnDefinition v = table.getColumnDefinition(new ColumnIdentifier("v", false));
+        ColumnDefinition bug = table.getColumnDefinition(new ColumnIdentifier("s", false));
+
+        Row.Builder builder;
+        builder = BTreeRow.unsortedBuilder(0);
+        builder.newRow(new Clustering(UTF8Serializer.instance.serialize("a")));
+        builder.addCell(BufferCell.live(table, v, 0L, Int32Serializer.instance.serialize(1), null));
+        builder.addComplexDeletion(bug, new DeletionTime(1L, 1));
+        Row row = builder.build();
+
+        DecoratedKey pk = table.decorateKey(ByteBufferUtil.bytes(1));
+        PartitionUpdate upd = PartitionUpdate.singleRowUpdate(table, pk, row);
+
+        // we need to perform the round trip in two parts here, with a column drop inbetween
+        try (RowIterator before = FilteredRows.filter(upd.unfilteredIterator(), FBUtilities.nowInSeconds());
+             DataOutputBuffer serialized21 = new DataOutputBuffer())
+        {
+            LegacyLayout.serializeAsLegacyPartition(null, upd.unfilteredIterator(), serialized21, MessagingService.VERSION_21);
+            QueryProcessor.executeInternal(String.format("ALTER TABLE \"%s\".legacy_static_rt_rt_dc DROP s", KEYSPACE));
+            try (DataInputBuffer in = new DataInputBuffer(serialized21.buffer(), false))
+            {
+                try (UnfilteredRowIterator deser21 = LegacyLayout.deserializeLegacyPartition(in, MessagingService.VERSION_21, SerializationHelper.Flag.LOCAL, upd.partitionKey().getKey());
+                    RowIterator after = FilteredRows.filter(deser21, FBUtilities.nowInSeconds());)
+                {
+                    while (before.hasNext() || after.hasNext())
+                        assertEquals(before.hasNext() ? before.next() : null, after.hasNext() ? after.next() : null);
+                }
+            }
+
+        }
+    }
+
 }
\ No newline at end of file


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