Merge branch 'cassandra-3.0' into cassandra-3.11

* cassandra-3.0:
  Potential AssertionError during ReadRepair of range tombstone and partition 
deletions


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

Branch: refs/heads/cassandra-3.11
Commit: ed9b04d6aeffc85a2dbcd8c136df769491f79d85
Parents: 59d4c27 5e57dd1
Author: Sylvain Lebresne <sylv...@datastax.com>
Authored: Thu Aug 24 11:36:14 2017 +0200
Committer: Sylvain Lebresne <sylv...@datastax.com>
Committed: Thu Aug 24 11:36:14 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/db/ReadResponse.java   | 22 +++++
 .../db/partitions/AbstractBTreePartition.java   | 14 +--
 .../db/partitions/PartitionUpdate.java          |  9 ++
 .../apache/cassandra/service/DataResolver.java  | 70 +++++++++++++--
 .../cassandra/service/DataResolverTest.java     | 91 +++++++++++++++++++-
 6 files changed, 192 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ed9b04d6/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 9e42ffb,2b49bc3..b22cb5e
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,11 -1,5 +1,12 @@@
 -3.0.15
 +3.11.1
 + * Fix cassandra-stress hang issues when an error during cluster connection 
happens (CASSANDRA-12938)
 + * Better bootstrap failure message when blocked by (potential) range 
movement (CASSANDRA-13744)
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation 
(CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
+  * Potential AssertionError during ReadRepair of range tombstone and 
partition deletions (CASSANDRA-13719)
   * Don't let stress write warmup data if n=0 (CASSANDRA-13773)
   * Gossip thread slows down when using batch commit log (CASSANDRA-12966)
   * Randomize batchlog endpoint selection with only 1 or 2 racks 
(CASSANDRA-12884)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ed9b04d6/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ed9b04d6/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ed9b04d6/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
index 64d7dd1,7bd5345..435c324
--- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
@@@ -314,9 -296,18 +314,18 @@@ public class PartitionUpdate extends Ab
  
          int nowInSecs = FBUtilities.nowInSeconds();
          List<UnfilteredRowIterator> asIterators = Lists.transform(updates, 
AbstractBTreePartition::unfilteredIterator);
 -        return fromIterator(UnfilteredRowIterators.merge(asIterators, 
nowInSecs));
 +        return fromIterator(UnfilteredRowIterators.merge(asIterators, 
nowInSecs), ColumnFilter.all(updates.get(0).metadata()));
      }
  
+     // We override this, because the version in the super-class calls 
holder(), which build the update preventing
+     // further updates, but that's not necessary here and being able to check 
at least the partition deletion without
+     // "locking" the update is nice (and used in 
DataResolver.RepairMergeListener.MergeListener).
+     @Override
+     public DeletionInfo deletionInfo()
+     {
+         return deletionInfo;
+     }
+ 
      /**
       * Modify this update to set every timestamp for live data to {@code 
newTimestamp} and
       * every deletion timestamp to {@code newTimestamp - 1}.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ed9b04d6/src/java/org/apache/cassandra/service/DataResolver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ed9b04d6/test/unit/org/apache/cassandra/service/DataResolverTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/service/DataResolverTest.java
index 8e4f385,65e18ce..2b1e095
--- a/test/unit/org/apache/cassandra/service/DataResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/DataResolverTest.java
@@@ -576,9 -574,9 +576,9 @@@ public class DataResolverTes
       * same deletion on both side (while is useless but could be created by 
legacy code pre-CASSANDRA-13237 and could
       * thus still be sent).
       */
-     public void testRepairRangeTombstoneBoundary(int timestamp1, int 
timestamp2, int timestamp3) throws UnknownHostException
+     private void testRepairRangeTombstoneBoundary(int timestamp1, int 
timestamp2, int timestamp3) throws UnknownHostException
      {
 -        DataResolver resolver = new DataResolver(ks, command, 
ConsistencyLevel.ALL, 2);
 +        DataResolver resolver = new DataResolver(ks, command, 
ConsistencyLevel.ALL, 2, System.nanoTime());
          InetAddress peer1 = peer();
          InetAddress peer2 = peer();
  
@@@ -623,6 -621,95 +623,95 @@@
          assertRepairContainsDeletions(msg, null, expected);
      }
  
+     /**
+      * Test for CASSANDRA-13719: tests that having a partition deletion 
shadow a range tombstone on another source
+      * doesn't trigger an assertion error.
+      */
+     @Test
+     public void testRepairRangeTombstoneWithPartitionDeletion()
+     {
 -        DataResolver resolver = new DataResolver(ks, command, 
ConsistencyLevel.ALL, 2);
++        DataResolver resolver = new DataResolver(ks, command, 
ConsistencyLevel.ALL, 2, System.nanoTime());
+         InetAddress peer1 = peer();
+         InetAddress peer2 = peer();
+ 
+         // 1st "stream": just a partition deletion
+         UnfilteredPartitionIterator iter1 = 
iter(PartitionUpdate.fullPartitionDelete(cfm, dk, 10, nowInSec));
+ 
+         // 2nd "stream": a range tombstone that is covered by the 1st stream
+         RangeTombstone rt = tombstone("0", true , "10", true, 5, nowInSec);
+         UnfilteredPartitionIterator iter2 = iter(new RowUpdateBuilder(cfm, 
nowInSec, 1L, dk)
+                                                  .addRangeTombstone(rt)
+                                                  .buildUpdate());
+ 
+         resolver.preprocess(readResponseMessage(peer1, iter1));
+         resolver.preprocess(readResponseMessage(peer2, iter2));
+ 
+         // No results, we've only reconciled tombstones.
+         try (PartitionIterator data = resolver.resolve())
+         {
+             assertFalse(data.hasNext());
+             // 2nd stream should get repaired
+             assertRepairFuture(resolver, 1);
+         }
+ 
+         assertEquals(1, messageRecorder.sent.size());
+ 
+         MessageOut msg = getSentMessage(peer2);
+         assertRepairMetadata(msg);
+         assertRepairContainsNoColumns(msg);
+ 
+         assertRepairContainsDeletions(msg, new DeletionTime(10, nowInSec));
+     }
+ 
+     /**
+      * Additional test for CASSANDRA-13719: tests the case where a partition 
deletion doesn't shadow a range tombstone.
+      */
+     @Test
+     public void testRepairRangeTombstoneWithPartitionDeletion2()
+     {
 -        DataResolver resolver = new DataResolver(ks, command, 
ConsistencyLevel.ALL, 2);
++        DataResolver resolver = new DataResolver(ks, command, 
ConsistencyLevel.ALL, 2, System.nanoTime());
+         InetAddress peer1 = peer();
+         InetAddress peer2 = peer();
+ 
+         // 1st "stream": a partition deletion and a range tombstone
+         RangeTombstone rt1 = tombstone("0", true , "9", true, 11, nowInSec);
+         PartitionUpdate upd1 = new RowUpdateBuilder(cfm, nowInSec, 1L, dk)
+                                                  .addRangeTombstone(rt1)
+                                                  .buildUpdate();
+         ((MutableDeletionInfo)upd1.deletionInfo()).add(new DeletionTime(10, 
nowInSec));
+         UnfilteredPartitionIterator iter1 = iter(upd1);
+ 
+         // 2nd "stream": a range tombstone that is covered by the other 
stream rt
+         RangeTombstone rt2 = tombstone("2", true , "3", true, 11, nowInSec);
+         RangeTombstone rt3 = tombstone("4", true , "5", true, 10, nowInSec);
+         UnfilteredPartitionIterator iter2 = iter(new RowUpdateBuilder(cfm, 
nowInSec, 1L, dk)
+                                                  .addRangeTombstone(rt2)
+                                                  .addRangeTombstone(rt3)
+                                                  .buildUpdate());
+ 
+         resolver.preprocess(readResponseMessage(peer1, iter1));
+         resolver.preprocess(readResponseMessage(peer2, iter2));
+ 
+         // No results, we've only reconciled tombstones.
+         try (PartitionIterator data = resolver.resolve())
+         {
+             assertFalse(data.hasNext());
+             // 2nd stream should get repaired
+             assertRepairFuture(resolver, 1);
+         }
+ 
+         assertEquals(1, messageRecorder.sent.size());
+ 
+         MessageOut msg = getSentMessage(peer2);
+         assertRepairMetadata(msg);
+         assertRepairContainsNoColumns(msg);
+ 
+         // 2nd stream should get both the partition deletion, as well as the 
part of the 1st stream RT that it misses
+         assertRepairContainsDeletions(msg, new DeletionTime(10, nowInSec),
+                                       tombstone("0", true, "2", false, 11, 
nowInSec),
+                                       tombstone("3", false, "9", true, 11, 
nowInSec));
+     }
+ 
      // Forces the start to be exclusive if the condition holds
      private static RangeTombstone withExclusiveStartIf(RangeTombstone rt, 
boolean condition)
      {


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

Reply via email to