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 2018/11/28 17:32:45 UTC

[1/6] cassandra git commit: RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 0a7fbee43 -> f7630e4c3
  refs/heads/cassandra-3.11 64d828b9d -> 4a531c27c
  refs/heads/trunk 3834a270f -> f2520d7a5


RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases

Patch by Blake Eggleston; Reviewed by Sam Tinnicliffe for CASSANDRA-14894


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

Branch: refs/heads/cassandra-3.0
Commit: f7630e4c3af3bbcf933f0708afaac7e3e7ef6101
Parents: 0a7fbee
Author: Blake Eggleston <bd...@gmail.com>
Authored: Mon Nov 26 13:29:03 2018 -0800
Committer: Blake Eggleston <bd...@gmail.com>
Committed: Wed Nov 28 09:06:18 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/rows/RowAndDeletionMergeIterator.java    | 40 +++++++++++++++-
 .../rows/RowAndDeletionMergeIteratorTest.java   | 49 ++++++++++++++++----
 3 files changed, 81 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0d33e3c..6e18de1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.18
+ * RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases (CASSANDRA-14894)
  * Fix handling of collection tombstones for dropped columns from legacy sstables (CASSANDRA-14912)
  * Fix missing rows when reading 2.1 SSTables with static columns in 3.0 (CASSANDRA-14873)
  * Move TWCS message 'No compaction necessary for bucket size' to Trace level (CASSANDRA-14884)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
index 389fe45..97b13e7 100644
--- a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
@@ -70,7 +70,7 @@ public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
         this.ranges = ranges;
     }
 
-    protected Unfiltered computeNext()
+    private Unfiltered computeNextInternal()
     {
         while (true)
         {
@@ -112,6 +112,44 @@ public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
         }
     }
 
+    /**
+     * RangeTombstoneList doesn't correctly merge multiple superseded rts, or overlapping rts with the
+     * same ts. This causes it to emit noop boundary markers which can cause unneeded read repairs and
+     * repair over streaming. This should technically be fixed in RangeTombstoneList. However, fixing
+     * it isn't trivial and that class is already so complicated that the fix would have a good chance
+     * of adding a worse bug. So we just swallow the noop boundary markers here. See CASSANDRA-14894
+     */
+    private static boolean shouldSkip(Unfiltered unfiltered)
+    {
+        if (unfiltered == null || !unfiltered.isRangeTombstoneMarker())
+            return false;
+
+        RangeTombstoneMarker marker = (RangeTombstoneMarker) unfiltered;
+
+        if (!marker.isBoundary())
+            return false;
+
+        DeletionTime open = marker.openDeletionTime(false);
+        DeletionTime close = marker.closeDeletionTime(false);
+
+        return open.equals(close);
+
+    }
+
+    @Override
+    protected Unfiltered computeNext()
+    {
+        while (true)
+        {
+            Unfiltered next = computeNextInternal();
+
+            if (shouldSkip(next))
+                continue;
+
+            return next;
+        }
+    }
+
     private void updateNextRow()
     {
         if (nextRow == null && rows.hasNext())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index 400d65a..dd88704 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Iterator;
 
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -244,11 +245,13 @@ public class RowAndDeletionMergeIteratorTest
     {
         Iterator<Row> rowIterator = createRowIterator();
 
-        int delTime = nowInSeconds + 1;
-        long timestamp = toMillis(delTime);
+        int delTime1 = nowInSeconds + 1;
+        long timestamp1 = toMillis(delTime1);
+        int delTime2 = delTime1 + 1;
+        long timestamp2 = toMillis(delTime2);
 
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp, delTime),
-                                                                                       greaterThan(2, timestamp, delTime));
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp1, delTime1),
+                                                                                       greaterThan(2, timestamp2, delTime2));
 
         UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
 
@@ -269,11 +272,13 @@ public class RowAndDeletionMergeIteratorTest
     {
         Iterator<Row> rowIterator = createRowIterator();
 
-        int delTime = nowInSeconds + 1;
-        long timestamp = toMillis(delTime);
+        int delTime1 = nowInSeconds + 1;
+        long timestamp1 = toMillis(delTime1);
+        int delTime2 = delTime1 + 1;
+        long timestamp2 = toMillis(delTime2);
 
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp, delTime),
-                                                                                       atLeast(2, timestamp, delTime));
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp1, delTime1),
+                                                                                       atLeast(2, timestamp2, delTime2));
 
         UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
 
@@ -345,6 +350,29 @@ public class RowAndDeletionMergeIteratorTest
     }
 
 
+    /**
+     * RTL doesn't correctly merge range tombstones in some situations (see CASSANDRA-14894)
+     */
+    @Test
+    public void testWithNoopBoundaryMarkers()
+    {
+        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+        RangeTombstoneList rtl = new RangeTombstoneList(cfm.comparator, 10);
+        rtl.add(rt(1, 2, 5, 5));
+        rtl.add(rt(3, 4, 5, 5));
+        rtl.add(rt(5, 6, 5, 5));
+        rtl.add(rt(0, 8, 6, 6)); // <- supersedes all other tombstones
+
+        Assert.assertEquals(3, rtl.size());
+
+        try (UnfilteredRowIterator partition = createMergeIterator(update.iterator(), rtl.iterator(), false))
+        {
+            assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 0);
+            assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 8);
+            assertFalse(partition.hasNext());
+        }
+    }
+
     private void assertRtMarker(Unfiltered unfiltered, Bound bound)
     {
         assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
@@ -436,6 +464,11 @@ public class RowAndDeletionMergeIteratorTest
         return new RangeTombstone(Slice.make(startBound, endBound), new DeletionTime(tstamp, delTime));
     }
 
+    private static RangeTombstone rt(int start, int end, long tstamp, int delTime)
+    {
+        return rt(start, true, end, true, tstamp, delTime);
+    }
+
     private static ByteBuffer bb(int i)
     {
         return ByteBufferUtil.bytes(i);


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


[4/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by bd...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: 4a531c27ce04e880f72feb49879d1cf6c2ee4b1d
Parents: 64d828b f7630e4
Author: Blake Eggleston <bd...@gmail.com>
Authored: Wed Nov 28 09:13:56 2018 -0800
Committer: Blake Eggleston <bd...@gmail.com>
Committed: Wed Nov 28 09:13:56 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/rows/RowAndDeletionMergeIterator.java    | 40 +++++++++++++++-
 .../rows/RowAndDeletionMergeIteratorTest.java   | 48 ++++++++++++++++----
 3 files changed, 80 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a531c27/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 5fbd53e,6e18de1..ebcf1b2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,6 -1,5 +1,7 @@@
 -3.0.18
 +3.11.4
 + * Correct sstable sorting for garbagecollect and levelled compaction (CASSANDRA-14870)
 +Merged from 3.0:
+  * RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases (CASSANDRA-14894)
   * Fix handling of collection tombstones for dropped columns from legacy sstables (CASSANDRA-14912)
   * Fix missing rows when reading 2.1 SSTables with static columns in 3.0 (CASSANDRA-14873)
   * Move TWCS message 'No compaction necessary for bucket size' to Trace level (CASSANDRA-14884)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a531c27/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a531c27/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index e4c04fb,dd88704..2f48000
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@@ -342,8 -349,31 +347,30 @@@ public class RowAndDeletionMergeIterato
          assertFalse(iterator.hasNext());
      }
  
 -
+     /**
+      * RTL doesn't correctly merge range tombstones in some situations (see CASSANDRA-14894)
+      */
+     @Test
+     public void testWithNoopBoundaryMarkers()
+     {
+         PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+         RangeTombstoneList rtl = new RangeTombstoneList(cfm.comparator, 10);
+         rtl.add(rt(1, 2, 5, 5));
+         rtl.add(rt(3, 4, 5, 5));
+         rtl.add(rt(5, 6, 5, 5));
+         rtl.add(rt(0, 8, 6, 6)); // <- supersedes all other tombstones
+ 
+         Assert.assertEquals(3, rtl.size());
+ 
+         try (UnfilteredRowIterator partition = createMergeIterator(update.iterator(), rtl.iterator(), false))
+         {
+             assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 0);
+             assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 8);
+             assertFalse(partition.hasNext());
+         }
+     }
  
 -    private void assertRtMarker(Unfiltered unfiltered, Bound bound)
 +    private void assertRtMarker(Unfiltered unfiltered, ClusteringBoundOrBoundary bound)
      {
          assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
          assertEquals(bound, unfiltered.clustering());


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


[6/6] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by bd...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: f2520d7a55525d281ff43348b4b913b3ef344208
Parents: 3834a27 4a531c2
Author: Blake Eggleston <bd...@gmail.com>
Authored: Wed Nov 28 09:29:46 2018 -0800
Committer: Blake Eggleston <bd...@gmail.com>
Committed: Wed Nov 28 09:29:46 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  3 ++
 .../db/rows/RowAndDeletionMergeIterator.java    | 40 +++++++++++++++-
 .../rows/RowAndDeletionMergeIteratorTest.java   | 48 ++++++++++++++++----
 3 files changed, 82 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2520d7a/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2520d7a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2520d7a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index 0d790fc,2f48000..f590d36
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@@ -341,6 -347,28 +346,28 @@@ public class RowAndDeletionMergeIterato
          assertFalse(iterator.hasNext());
      }
  
+     /**
+      * RTL doesn't correctly merge range tombstones in some situations (see CASSANDRA-14894)
+      */
+     @Test
+     public void testWithNoopBoundaryMarkers()
+     {
 -        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
++        PartitionUpdate update = PartitionUpdate.emptyUpdate(cfm, dk);
+         RangeTombstoneList rtl = new RangeTombstoneList(cfm.comparator, 10);
+         rtl.add(rt(1, 2, 5, 5));
+         rtl.add(rt(3, 4, 5, 5));
+         rtl.add(rt(5, 6, 5, 5));
+         rtl.add(rt(0, 8, 6, 6)); // <- supersedes all other tombstones
+ 
+         Assert.assertEquals(3, rtl.size());
+ 
+         try (UnfilteredRowIterator partition = createMergeIterator(update.iterator(), rtl.iterator(), false))
+         {
+             assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 0);
+             assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 8);
+             assertFalse(partition.hasNext());
+         }
+     }
  
      private void assertRtMarker(Unfiltered unfiltered, ClusteringBoundOrBoundary bound)
      {


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


[3/6] cassandra git commit: RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases

Posted by bd...@apache.org.
RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases

Patch by Blake Eggleston; Reviewed by Sam Tinnicliffe for CASSANDRA-14894


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

Branch: refs/heads/trunk
Commit: f7630e4c3af3bbcf933f0708afaac7e3e7ef6101
Parents: 0a7fbee
Author: Blake Eggleston <bd...@gmail.com>
Authored: Mon Nov 26 13:29:03 2018 -0800
Committer: Blake Eggleston <bd...@gmail.com>
Committed: Wed Nov 28 09:06:18 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/rows/RowAndDeletionMergeIterator.java    | 40 +++++++++++++++-
 .../rows/RowAndDeletionMergeIteratorTest.java   | 49 ++++++++++++++++----
 3 files changed, 81 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0d33e3c..6e18de1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.18
+ * RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases (CASSANDRA-14894)
  * Fix handling of collection tombstones for dropped columns from legacy sstables (CASSANDRA-14912)
  * Fix missing rows when reading 2.1 SSTables with static columns in 3.0 (CASSANDRA-14873)
  * Move TWCS message 'No compaction necessary for bucket size' to Trace level (CASSANDRA-14884)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
index 389fe45..97b13e7 100644
--- a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
@@ -70,7 +70,7 @@ public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
         this.ranges = ranges;
     }
 
-    protected Unfiltered computeNext()
+    private Unfiltered computeNextInternal()
     {
         while (true)
         {
@@ -112,6 +112,44 @@ public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
         }
     }
 
+    /**
+     * RangeTombstoneList doesn't correctly merge multiple superseded rts, or overlapping rts with the
+     * same ts. This causes it to emit noop boundary markers which can cause unneeded read repairs and
+     * repair over streaming. This should technically be fixed in RangeTombstoneList. However, fixing
+     * it isn't trivial and that class is already so complicated that the fix would have a good chance
+     * of adding a worse bug. So we just swallow the noop boundary markers here. See CASSANDRA-14894
+     */
+    private static boolean shouldSkip(Unfiltered unfiltered)
+    {
+        if (unfiltered == null || !unfiltered.isRangeTombstoneMarker())
+            return false;
+
+        RangeTombstoneMarker marker = (RangeTombstoneMarker) unfiltered;
+
+        if (!marker.isBoundary())
+            return false;
+
+        DeletionTime open = marker.openDeletionTime(false);
+        DeletionTime close = marker.closeDeletionTime(false);
+
+        return open.equals(close);
+
+    }
+
+    @Override
+    protected Unfiltered computeNext()
+    {
+        while (true)
+        {
+            Unfiltered next = computeNextInternal();
+
+            if (shouldSkip(next))
+                continue;
+
+            return next;
+        }
+    }
+
     private void updateNextRow()
     {
         if (nextRow == null && rows.hasNext())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index 400d65a..dd88704 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Iterator;
 
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -244,11 +245,13 @@ public class RowAndDeletionMergeIteratorTest
     {
         Iterator<Row> rowIterator = createRowIterator();
 
-        int delTime = nowInSeconds + 1;
-        long timestamp = toMillis(delTime);
+        int delTime1 = nowInSeconds + 1;
+        long timestamp1 = toMillis(delTime1);
+        int delTime2 = delTime1 + 1;
+        long timestamp2 = toMillis(delTime2);
 
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp, delTime),
-                                                                                       greaterThan(2, timestamp, delTime));
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp1, delTime1),
+                                                                                       greaterThan(2, timestamp2, delTime2));
 
         UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
 
@@ -269,11 +272,13 @@ public class RowAndDeletionMergeIteratorTest
     {
         Iterator<Row> rowIterator = createRowIterator();
 
-        int delTime = nowInSeconds + 1;
-        long timestamp = toMillis(delTime);
+        int delTime1 = nowInSeconds + 1;
+        long timestamp1 = toMillis(delTime1);
+        int delTime2 = delTime1 + 1;
+        long timestamp2 = toMillis(delTime2);
 
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp, delTime),
-                                                                                       atLeast(2, timestamp, delTime));
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp1, delTime1),
+                                                                                       atLeast(2, timestamp2, delTime2));
 
         UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
 
@@ -345,6 +350,29 @@ public class RowAndDeletionMergeIteratorTest
     }
 
 
+    /**
+     * RTL doesn't correctly merge range tombstones in some situations (see CASSANDRA-14894)
+     */
+    @Test
+    public void testWithNoopBoundaryMarkers()
+    {
+        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+        RangeTombstoneList rtl = new RangeTombstoneList(cfm.comparator, 10);
+        rtl.add(rt(1, 2, 5, 5));
+        rtl.add(rt(3, 4, 5, 5));
+        rtl.add(rt(5, 6, 5, 5));
+        rtl.add(rt(0, 8, 6, 6)); // <- supersedes all other tombstones
+
+        Assert.assertEquals(3, rtl.size());
+
+        try (UnfilteredRowIterator partition = createMergeIterator(update.iterator(), rtl.iterator(), false))
+        {
+            assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 0);
+            assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 8);
+            assertFalse(partition.hasNext());
+        }
+    }
+
     private void assertRtMarker(Unfiltered unfiltered, Bound bound)
     {
         assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
@@ -436,6 +464,11 @@ public class RowAndDeletionMergeIteratorTest
         return new RangeTombstone(Slice.make(startBound, endBound), new DeletionTime(tstamp, delTime));
     }
 
+    private static RangeTombstone rt(int start, int end, long tstamp, int delTime)
+    {
+        return rt(start, true, end, true, tstamp, delTime);
+    }
+
     private static ByteBuffer bb(int i)
     {
         return ByteBufferUtil.bytes(i);


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


[2/6] cassandra git commit: RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases

Posted by bd...@apache.org.
RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases

Patch by Blake Eggleston; Reviewed by Sam Tinnicliffe for CASSANDRA-14894


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

Branch: refs/heads/cassandra-3.11
Commit: f7630e4c3af3bbcf933f0708afaac7e3e7ef6101
Parents: 0a7fbee
Author: Blake Eggleston <bd...@gmail.com>
Authored: Mon Nov 26 13:29:03 2018 -0800
Committer: Blake Eggleston <bd...@gmail.com>
Committed: Wed Nov 28 09:06:18 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/rows/RowAndDeletionMergeIterator.java    | 40 +++++++++++++++-
 .../rows/RowAndDeletionMergeIteratorTest.java   | 49 ++++++++++++++++----
 3 files changed, 81 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0d33e3c..6e18de1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.18
+ * RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases (CASSANDRA-14894)
  * Fix handling of collection tombstones for dropped columns from legacy sstables (CASSANDRA-14912)
  * Fix missing rows when reading 2.1 SSTables with static columns in 3.0 (CASSANDRA-14873)
  * Move TWCS message 'No compaction necessary for bucket size' to Trace level (CASSANDRA-14884)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
index 389fe45..97b13e7 100644
--- a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
@@ -70,7 +70,7 @@ public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
         this.ranges = ranges;
     }
 
-    protected Unfiltered computeNext()
+    private Unfiltered computeNextInternal()
     {
         while (true)
         {
@@ -112,6 +112,44 @@ public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
         }
     }
 
+    /**
+     * RangeTombstoneList doesn't correctly merge multiple superseded rts, or overlapping rts with the
+     * same ts. This causes it to emit noop boundary markers which can cause unneeded read repairs and
+     * repair over streaming. This should technically be fixed in RangeTombstoneList. However, fixing
+     * it isn't trivial and that class is already so complicated that the fix would have a good chance
+     * of adding a worse bug. So we just swallow the noop boundary markers here. See CASSANDRA-14894
+     */
+    private static boolean shouldSkip(Unfiltered unfiltered)
+    {
+        if (unfiltered == null || !unfiltered.isRangeTombstoneMarker())
+            return false;
+
+        RangeTombstoneMarker marker = (RangeTombstoneMarker) unfiltered;
+
+        if (!marker.isBoundary())
+            return false;
+
+        DeletionTime open = marker.openDeletionTime(false);
+        DeletionTime close = marker.closeDeletionTime(false);
+
+        return open.equals(close);
+
+    }
+
+    @Override
+    protected Unfiltered computeNext()
+    {
+        while (true)
+        {
+            Unfiltered next = computeNextInternal();
+
+            if (shouldSkip(next))
+                continue;
+
+            return next;
+        }
+    }
+
     private void updateNextRow()
     {
         if (nextRow == null && rows.hasNext())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7630e4c/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index 400d65a..dd88704 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Iterator;
 
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -244,11 +245,13 @@ public class RowAndDeletionMergeIteratorTest
     {
         Iterator<Row> rowIterator = createRowIterator();
 
-        int delTime = nowInSeconds + 1;
-        long timestamp = toMillis(delTime);
+        int delTime1 = nowInSeconds + 1;
+        long timestamp1 = toMillis(delTime1);
+        int delTime2 = delTime1 + 1;
+        long timestamp2 = toMillis(delTime2);
 
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp, delTime),
-                                                                                       greaterThan(2, timestamp, delTime));
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp1, delTime1),
+                                                                                       greaterThan(2, timestamp2, delTime2));
 
         UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
 
@@ -269,11 +272,13 @@ public class RowAndDeletionMergeIteratorTest
     {
         Iterator<Row> rowIterator = createRowIterator();
 
-        int delTime = nowInSeconds + 1;
-        long timestamp = toMillis(delTime);
+        int delTime1 = nowInSeconds + 1;
+        long timestamp1 = toMillis(delTime1);
+        int delTime2 = delTime1 + 1;
+        long timestamp2 = toMillis(delTime2);
 
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp, delTime),
-                                                                                       atLeast(2, timestamp, delTime));
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp1, delTime1),
+                                                                                       atLeast(2, timestamp2, delTime2));
 
         UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
 
@@ -345,6 +350,29 @@ public class RowAndDeletionMergeIteratorTest
     }
 
 
+    /**
+     * RTL doesn't correctly merge range tombstones in some situations (see CASSANDRA-14894)
+     */
+    @Test
+    public void testWithNoopBoundaryMarkers()
+    {
+        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+        RangeTombstoneList rtl = new RangeTombstoneList(cfm.comparator, 10);
+        rtl.add(rt(1, 2, 5, 5));
+        rtl.add(rt(3, 4, 5, 5));
+        rtl.add(rt(5, 6, 5, 5));
+        rtl.add(rt(0, 8, 6, 6)); // <- supersedes all other tombstones
+
+        Assert.assertEquals(3, rtl.size());
+
+        try (UnfilteredRowIterator partition = createMergeIterator(update.iterator(), rtl.iterator(), false))
+        {
+            assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 0);
+            assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 8);
+            assertFalse(partition.hasNext());
+        }
+    }
+
     private void assertRtMarker(Unfiltered unfiltered, Bound bound)
     {
         assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
@@ -436,6 +464,11 @@ public class RowAndDeletionMergeIteratorTest
         return new RangeTombstone(Slice.make(startBound, endBound), new DeletionTime(tstamp, delTime));
     }
 
+    private static RangeTombstone rt(int start, int end, long tstamp, int delTime)
+    {
+        return rt(start, true, end, true, tstamp, delTime);
+    }
+
     private static ByteBuffer bb(int i)
     {
         return ByteBufferUtil.bytes(i);


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


[5/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by bd...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/cassandra-3.11
Commit: 4a531c27ce04e880f72feb49879d1cf6c2ee4b1d
Parents: 64d828b f7630e4
Author: Blake Eggleston <bd...@gmail.com>
Authored: Wed Nov 28 09:13:56 2018 -0800
Committer: Blake Eggleston <bd...@gmail.com>
Committed: Wed Nov 28 09:13:56 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/rows/RowAndDeletionMergeIterator.java    | 40 +++++++++++++++-
 .../rows/RowAndDeletionMergeIteratorTest.java   | 48 ++++++++++++++++----
 3 files changed, 80 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a531c27/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 5fbd53e,6e18de1..ebcf1b2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,6 -1,5 +1,7 @@@
 -3.0.18
 +3.11.4
 + * Correct sstable sorting for garbagecollect and levelled compaction (CASSANDRA-14870)
 +Merged from 3.0:
+  * RangeTombstoneList doesn't properly clean up mergeable or superseded rts in some cases (CASSANDRA-14894)
   * Fix handling of collection tombstones for dropped columns from legacy sstables (CASSANDRA-14912)
   * Fix missing rows when reading 2.1 SSTables with static columns in 3.0 (CASSANDRA-14873)
   * Move TWCS message 'No compaction necessary for bucket size' to Trace level (CASSANDRA-14884)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a531c27/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a531c27/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index e4c04fb,dd88704..2f48000
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@@ -342,8 -349,31 +347,30 @@@ public class RowAndDeletionMergeIterato
          assertFalse(iterator.hasNext());
      }
  
 -
+     /**
+      * RTL doesn't correctly merge range tombstones in some situations (see CASSANDRA-14894)
+      */
+     @Test
+     public void testWithNoopBoundaryMarkers()
+     {
+         PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+         RangeTombstoneList rtl = new RangeTombstoneList(cfm.comparator, 10);
+         rtl.add(rt(1, 2, 5, 5));
+         rtl.add(rt(3, 4, 5, 5));
+         rtl.add(rt(5, 6, 5, 5));
+         rtl.add(rt(0, 8, 6, 6)); // <- supersedes all other tombstones
+ 
+         Assert.assertEquals(3, rtl.size());
+ 
+         try (UnfilteredRowIterator partition = createMergeIterator(update.iterator(), rtl.iterator(), false))
+         {
+             assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 0);
+             assertRtMarker(partition.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 8);
+             assertFalse(partition.hasNext());
+         }
+     }
  
 -    private void assertRtMarker(Unfiltered unfiltered, Bound bound)
 +    private void assertRtMarker(Unfiltered unfiltered, ClusteringBoundOrBoundary bound)
      {
          assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
          assertEquals(bound, unfiltered.clustering());


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