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 2016/07/06 12:40:20 UTC

[1/6] cassandra git commit: Range tombstones that are masked by row tombstones should not be written out

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 3c1653f47 -> 98f5f77bb
  refs/heads/cassandra-2.2 d5a15e45b -> 43c741e25
  refs/heads/cassandra-3.0 dd05e46f0 -> 9ed3b42d3


Range tombstones that are masked by row tombstones should not be written out

patch by Nachiket Patil; reviewed by Sylvain Lebresne for CASSANDRA-12030


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

Branch: refs/heads/cassandra-2.1
Commit: 98f5f77bb3c5d50e52cbb6f577a463ca8a5134ad
Parents: 3c1653f
Author: Nachiket Patil <na...@apple.com>
Authored: Wed Jul 6 11:22:56 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:35:10 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/LazilyCompactedRow.java       |  3 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b1dcbe1..7fa995d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Don't write shadowed range tombstone (CASSANDRA-12030)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
  * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
  * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
index f912da2..dab5eeb 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -286,7 +286,8 @@ public class LazilyCompactedRow extends AbstractCompactedRow
                 RangeTombstone t = tombstone;
                 tombstone = null;
 
-                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp())
+                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp() ||
+                    maxRowTombstone.markedForDeleteAt >= t.timestamp())
                 {
                     indexBuilder.tombstoneTracker().update(t, true);
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 3292422..dfd6960 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNames;
@@ -543,6 +544,45 @@ public class RangeTombstoneTest extends SchemaLoader
     }
 
     @Test
+    public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+    {
+        long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+        Keyspace table = Keyspace.open(KSNAME);
+        ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+        ByteBuffer key = ByteBufferUtil.bytes("k4");
+
+        // remove any existing sstables before starting
+        cfs.truncateBlocking();
+        cfs.disableAutoCompaction();
+        cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+
+        Mutation rm = new Mutation(KSNAME, key);
+        for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+            add(rm, i, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        rm = new Mutation(KSNAME, key);
+        ColumnFamily cf = rm.addOrGet(CFNAME);
+
+        // Write the covering row tombstone
+        cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+
+        // Create range tombstones covered by row tombstone above.
+        for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+            delete(cf, 0, 7, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // there should be 2 sstables
+        assertEquals(2, cfs.getSSTables().size());
+
+        // compact down to nothing
+        CompactionManager.instance.performMaximal(cfs);
+        assertEquals(0, cfs.getSSTables().size());
+    }
+
+    @Test
     public void testOverwritesToDeletedColumns() throws Exception
     {
         Keyspace table = Keyspace.open(KSNAME);


[5/6] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by sl...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2

* cassandra-2.1:
  Range tombstones that are masked by row tombstones should not be written out


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

Branch: refs/heads/cassandra-2.2
Commit: 43c741e251102bf5651ff8aa1b5ca078eb0ddc0b
Parents: d5a15e4 98f5f77
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Jul 6 14:39:13 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:39:13 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/LazilyCompactedRow.java       |  3 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 65c7c1f,7fa995d..bfd8aa2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,43 -1,11 +1,44 @@@
 -2.1.16
 +2.2.8
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 +Merged from 2.1:
+  * Don't write shadowed range tombstone (CASSANDRA-12030)
 - * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 - * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
   * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
 -
 -2.1.15
 + * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
   * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +
 +
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 9ce1236,dfd6960..bff0ddf
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@@ -37,10 -33,13 +37,11 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.config.*;
  import org.apache.cassandra.Util;
 -import org.apache.cassandra.config.ColumnDefinition;
 -import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.config.IndexType;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.compaction.CompactionManager;
+ import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
  import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
  import org.apache.cassandra.db.composites.CellName;
  import org.apache.cassandra.db.composites.CellNames;
@@@ -559,6 -543,46 +560,45 @@@ public class RangeTombstoneTes
      }
  
      @Test
+     public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+     {
+         long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+         Keyspace table = Keyspace.open(KSNAME);
+         ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+         ByteBuffer key = ByteBufferUtil.bytes("k4");
+ 
+         // remove any existing sstables before starting
+         cfs.truncateBlocking();
+         cfs.disableAutoCompaction();
+         cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+ 
+         Mutation rm = new Mutation(KSNAME, key);
+         for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+             add(rm, i, testTimeStamp);
+         rm.apply();
+         cfs.forceBlockingFlush();
+ 
+         rm = new Mutation(KSNAME, key);
+         ColumnFamily cf = rm.addOrGet(CFNAME);
+ 
+         // Write the covering row tombstone
+         cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+ 
+         // Create range tombstones covered by row tombstone above.
+         for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+             delete(cf, 0, 7, testTimeStamp);
+         rm.apply();
+         cfs.forceBlockingFlush();
+ 
+         // there should be 2 sstables
+         assertEquals(2, cfs.getSSTables().size());
+ 
+         // compact down to nothing
 -        CompactionManager.instance.performMaximal(cfs);
++        CompactionManager.instance.performMaximal(cfs, false);
+         assertEquals(0, cfs.getSSTables().size());
+     }
+ 
+     @Test
      public void testOverwritesToDeletedColumns() throws Exception
      {
          Keyspace table = Keyspace.open(KSNAME);


[2/6] cassandra git commit: Range tombstones that are masked by row tombstones should not be written out

Posted by sl...@apache.org.
Range tombstones that are masked by row tombstones should not be written out

patch by Nachiket Patil; reviewed by Sylvain Lebresne for CASSANDRA-12030


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

Branch: refs/heads/cassandra-2.2
Commit: 98f5f77bb3c5d50e52cbb6f577a463ca8a5134ad
Parents: 3c1653f
Author: Nachiket Patil <na...@apple.com>
Authored: Wed Jul 6 11:22:56 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:35:10 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/LazilyCompactedRow.java       |  3 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b1dcbe1..7fa995d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Don't write shadowed range tombstone (CASSANDRA-12030)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
  * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
  * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
index f912da2..dab5eeb 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -286,7 +286,8 @@ public class LazilyCompactedRow extends AbstractCompactedRow
                 RangeTombstone t = tombstone;
                 tombstone = null;
 
-                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp())
+                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp() ||
+                    maxRowTombstone.markedForDeleteAt >= t.timestamp())
                 {
                     indexBuilder.tombstoneTracker().update(t, true);
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 3292422..dfd6960 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNames;
@@ -543,6 +544,45 @@ public class RangeTombstoneTest extends SchemaLoader
     }
 
     @Test
+    public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+    {
+        long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+        Keyspace table = Keyspace.open(KSNAME);
+        ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+        ByteBuffer key = ByteBufferUtil.bytes("k4");
+
+        // remove any existing sstables before starting
+        cfs.truncateBlocking();
+        cfs.disableAutoCompaction();
+        cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+
+        Mutation rm = new Mutation(KSNAME, key);
+        for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+            add(rm, i, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        rm = new Mutation(KSNAME, key);
+        ColumnFamily cf = rm.addOrGet(CFNAME);
+
+        // Write the covering row tombstone
+        cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+
+        // Create range tombstones covered by row tombstone above.
+        for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+            delete(cf, 0, 7, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // there should be 2 sstables
+        assertEquals(2, cfs.getSSTables().size());
+
+        // compact down to nothing
+        CompactionManager.instance.performMaximal(cfs);
+        assertEquals(0, cfs.getSSTables().size());
+    }
+
+    @Test
     public void testOverwritesToDeletedColumns() throws Exception
     {
         Keyspace table = Keyspace.open(KSNAME);


[6/6] cassandra git commit: Merge commit '43c741e251102bf5651ff8aa1b5ca078eb0ddc0b' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit '43c741e251102bf5651ff8aa1b5ca078eb0ddc0b' into cassandra-3.0

* commit '43c741e251102bf5651ff8aa1b5ca078eb0ddc0b':
  Range tombstones that are masked by row tombstones should not be written out


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

Branch: refs/heads/cassandra-3.0
Commit: 9ed3b42d3b50237f99485233857a7b34d5238d9a
Parents: dd05e46 43c741e
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Jul 6 14:39:52 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:39:52 2016 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[4/6] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by sl...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2

* cassandra-2.1:
  Range tombstones that are masked by row tombstones should not be written out


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

Branch: refs/heads/cassandra-3.0
Commit: 43c741e251102bf5651ff8aa1b5ca078eb0ddc0b
Parents: d5a15e4 98f5f77
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Jul 6 14:39:13 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:39:13 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/LazilyCompactedRow.java       |  3 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 65c7c1f,7fa995d..bfd8aa2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,43 -1,11 +1,44 @@@
 -2.1.16
 +2.2.8
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 +Merged from 2.1:
+  * Don't write shadowed range tombstone (CASSANDRA-12030)
 - * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 - * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
   * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
 -
 -2.1.15
 + * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
   * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +
 +
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 9ce1236,dfd6960..bff0ddf
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@@ -37,10 -33,13 +37,11 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.config.*;
  import org.apache.cassandra.Util;
 -import org.apache.cassandra.config.ColumnDefinition;
 -import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.config.IndexType;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.compaction.CompactionManager;
+ import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
  import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
  import org.apache.cassandra.db.composites.CellName;
  import org.apache.cassandra.db.composites.CellNames;
@@@ -559,6 -543,46 +560,45 @@@ public class RangeTombstoneTes
      }
  
      @Test
+     public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+     {
+         long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+         Keyspace table = Keyspace.open(KSNAME);
+         ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+         ByteBuffer key = ByteBufferUtil.bytes("k4");
+ 
+         // remove any existing sstables before starting
+         cfs.truncateBlocking();
+         cfs.disableAutoCompaction();
+         cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+ 
+         Mutation rm = new Mutation(KSNAME, key);
+         for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+             add(rm, i, testTimeStamp);
+         rm.apply();
+         cfs.forceBlockingFlush();
+ 
+         rm = new Mutation(KSNAME, key);
+         ColumnFamily cf = rm.addOrGet(CFNAME);
+ 
+         // Write the covering row tombstone
+         cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+ 
+         // Create range tombstones covered by row tombstone above.
+         for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+             delete(cf, 0, 7, testTimeStamp);
+         rm.apply();
+         cfs.forceBlockingFlush();
+ 
+         // there should be 2 sstables
+         assertEquals(2, cfs.getSSTables().size());
+ 
+         // compact down to nothing
 -        CompactionManager.instance.performMaximal(cfs);
++        CompactionManager.instance.performMaximal(cfs, false);
+         assertEquals(0, cfs.getSSTables().size());
+     }
+ 
+     @Test
      public void testOverwritesToDeletedColumns() throws Exception
      {
          Keyspace table = Keyspace.open(KSNAME);


[3/6] cassandra git commit: Range tombstones that are masked by row tombstones should not be written out

Posted by sl...@apache.org.
Range tombstones that are masked by row tombstones should not be written out

patch by Nachiket Patil; reviewed by Sylvain Lebresne for CASSANDRA-12030


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

Branch: refs/heads/cassandra-3.0
Commit: 98f5f77bb3c5d50e52cbb6f577a463ca8a5134ad
Parents: 3c1653f
Author: Nachiket Patil <na...@apple.com>
Authored: Wed Jul 6 11:22:56 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:35:10 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/LazilyCompactedRow.java       |  3 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b1dcbe1..7fa995d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Don't write shadowed range tombstone (CASSANDRA-12030)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
  * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
  * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
index f912da2..dab5eeb 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -286,7 +286,8 @@ public class LazilyCompactedRow extends AbstractCompactedRow
                 RangeTombstone t = tombstone;
                 tombstone = null;
 
-                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp())
+                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp() ||
+                    maxRowTombstone.markedForDeleteAt >= t.timestamp())
                 {
                     indexBuilder.tombstoneTracker().update(t, true);
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 3292422..dfd6960 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNames;
@@ -543,6 +544,45 @@ public class RangeTombstoneTest extends SchemaLoader
     }
 
     @Test
+    public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+    {
+        long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+        Keyspace table = Keyspace.open(KSNAME);
+        ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+        ByteBuffer key = ByteBufferUtil.bytes("k4");
+
+        // remove any existing sstables before starting
+        cfs.truncateBlocking();
+        cfs.disableAutoCompaction();
+        cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+
+        Mutation rm = new Mutation(KSNAME, key);
+        for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+            add(rm, i, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        rm = new Mutation(KSNAME, key);
+        ColumnFamily cf = rm.addOrGet(CFNAME);
+
+        // Write the covering row tombstone
+        cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+
+        // Create range tombstones covered by row tombstone above.
+        for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+            delete(cf, 0, 7, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // there should be 2 sstables
+        assertEquals(2, cfs.getSSTables().size());
+
+        // compact down to nothing
+        CompactionManager.instance.performMaximal(cfs);
+        assertEquals(0, cfs.getSSTables().size());
+    }
+
+    @Test
     public void testOverwritesToDeletedColumns() throws Exception
     {
         Keyspace table = Keyspace.open(KSNAME);