You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2017/02/08 10:42:55 UTC

[3/6] cassandra git commit: Duplicate rows after upgrading from 2.1.16 to 3.0.10/3.9

Duplicate rows after upgrading from 2.1.16 to 3.0.10/3.9

patch by Sylvain Lebresne; reviewed by Tyler Hobbs for CASSANDRA-13125


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

Branch: refs/heads/trunk
Commit: 7c2437e26bc289c0a4c40fca303310fb61f362b1
Parents: fb606dd
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Jan 20 15:14:46 2017 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 8 11:36:24 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/db/LegacyLayout.java   | 62 ++++++++++++++---
 .../apache/cassandra/db/LegacyLayoutTest.java   | 71 ++++++++++++++++++++
 3 files changed, 123 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7c2437e2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4387019..7da61e7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.11
+ * Duplicate rows after upgrading from 2.1.16 to 3.0.10/3.9 (CASSANDRA-13125)
  * Fix UPDATE queries with empty IN restrictions (CASSANDRA-13152)
  * Abort or retry on failed hints delivery (CASSANDRA-13124)
  * Fix handling of partition with partition-level deletion plus

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7c2437e2/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 3f69671..3788c3c 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -1803,17 +1803,42 @@ public abstract class LegacyLayout
             if (a.isStatic != b.isStatic)
                 return a.isStatic ? -1 : 1;
 
-            int result = this.clusteringComparator.compare(a.bound, b.bound);
-            if (result != 0)
-                return result;
+            // We have to be careful with bound comparison because of collections. Namely, if the 2 bounds represent the
+            // same prefix, then we should take the collectionName into account before taking the bounds kind
+            // (ClusteringPrefix.Kind). This means we can't really call ClusteringComparator.compare() directly.
+            // For instance, if
+            //    a is (bound=INCL_START_BOUND('x'), collectionName='d')
+            //    b is (bound=INCL_END_BOUND('x'),   collectionName='c')
+            // Ten b < a since the element 'c' of collection 'x' comes before element 'd', but calling
+            // clusteringComparator.compare(a.bound, b.bound) returns -1.
+            // See CASSANDRA-13125 for details.
+            int sa = a.bound.size();
+            int sb = b.bound.size();
+            for (int i = 0; i < Math.min(sa, sb); i++)
+            {
+                int cmp = clusteringComparator.compareComponent(i, a.bound.get(i), b.bound.get(i));
+                if (cmp != 0)
+                    return cmp;
+            }
+
+            if (sa != sb)
+                return sa < sb ? a.bound.kind().comparedToClustering : -b.bound.kind().comparedToClustering;
+
+            // Both bound represent the same prefix, compare the collection names
+            // If one has a collection name and the other doesn't, the other comes before as it points to the beginning of the row.
+            if ((a.collectionName == null) != (b.collectionName == null))
+                return a.collectionName == null ? -1 : 1;
 
-            // If both have equal "bound" but one is a collection tombstone and not the other, then the other comes before as it points to the beginning of the row.
-            if (a.collectionName == null)
-                return b.collectionName == null ? 0 : 1;
-            if (b.collectionName == null)
-                return -1;
+            // If they both have a collection, compare that first
+            if (a.collectionName != null)
+            {
+                int cmp = UTF8Type.instance.compare(a.collectionName.name.bytes, b.collectionName.name.bytes);
+                if (cmp != 0)
+                    return cmp;
+            }
 
-            return UTF8Type.instance.compare(a.collectionName.name.bytes, b.collectionName.name.bytes);
+            // Lastly, if everything so far is equal, compare their clustering kind
+            return ClusteringPrefix.Kind.compare(a.bound.kind(), b.bound.kind());
         }
     }
 
@@ -1830,8 +1855,8 @@ public abstract class LegacyLayout
 
         // Note: we don't want to use a List for the markedAts and delTimes to avoid boxing. We could
         // use a List for starts and ends, but having arrays everywhere is almost simpler.
-        private LegacyBound[] starts;
-        private LegacyBound[] ends;
+        LegacyBound[] starts;
+        LegacyBound[] ends;
         private long[] markedAts;
         private int[] delTimes;
 
@@ -1853,6 +1878,20 @@ public abstract class LegacyLayout
             this(comparator, new LegacyBound[capacity], new LegacyBound[capacity], new long[capacity], new int[capacity], 0);
         }
 
+        @Override
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder();
+            sb.append('[');
+            for (int i = 0; i < size; i++)
+            {
+                if (i > 0)
+                    sb.append(',');
+                sb.append('(').append(starts[i]).append(", ").append(ends[i]).append(')');
+            }
+            return sb.append(']').toString();
+        }
+
         public boolean isEmpty()
         {
             return size == 0;
@@ -2107,6 +2146,7 @@ public abstract class LegacyLayout
             // If we got there, then just insert the remainder at the end
             addInternal(i, start, end, markedAt, delTime);
         }
+
         private int capacity()
         {
             return starts.length;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7c2437e2/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
new file mode 100644
index 0000000..09e9755
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.junit.Assert.*;
+
+public class LegacyLayoutTest
+{
+    @Test
+    public void testFromUnfilteredRowIterator() throws Throwable
+    {
+        CFMetaData table = CFMetaData.Builder.create("ks", "table")
+                                             .addPartitionKey("k", Int32Type.instance)
+                                             .addRegularColumn("a", SetType.getInstance(Int32Type.instance, true))
+                                             .addRegularColumn("b", SetType.getInstance(Int32Type.instance, true))
+                                             .build();
+
+        ColumnDefinition a = table.getColumnDefinition(new ColumnIdentifier("a", false));
+        ColumnDefinition b = table.getColumnDefinition(new ColumnIdentifier("b", false));
+
+        Row.Builder builder = BTreeRow.unsortedBuilder(0);
+        builder.newRow(Clustering.EMPTY);
+        builder.addComplexDeletion(a, new DeletionTime(1L, 1));
+        builder.addComplexDeletion(b, new DeletionTime(1L, 1));
+        Row row = builder.build();
+
+        ByteBuffer key = ByteBufferUtil.bytes(1);
+        PartitionUpdate upd = PartitionUpdate.singleRowUpdate(table, key, row);
+
+        LegacyLayout.LegacyUnfilteredPartition p = LegacyLayout.fromUnfilteredRowIterator(null, upd.unfilteredIterator());
+        assertEquals(DeletionTime.LIVE, p.partitionDeletion);
+        assertEquals(0, p.cells.size());
+
+        LegacyLayout.LegacyRangeTombstoneList l = p.rangeTombstones;
+        assertEquals("a", l.starts[0].collectionName.name.toString());
+        assertEquals("a", l.ends[0].collectionName.name.toString());
+
+        assertEquals("b", l.starts[1].collectionName.name.toString());
+        assertEquals("b", l.ends[1].collectionName.name.toString());
+    }
+}
\ No newline at end of file