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/07 11:03:36 UTC

[1/9] cassandra git commit: NPE when trying to remove purgable tombstones from result

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 30f5d44d8 -> 76e68e9b4
  refs/heads/cassandra-3.9 376dae268 -> a006f577b
  refs/heads/trunk 9fd607778 -> 3016dc7c2


NPE when trying to remove purgable tombstones from result

patch by mck; reviewed by Sylvain Lebresne for CASSANDRA-12143


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

Branch: refs/heads/trunk
Commit: ef18a1768a6589eac212a7f320f9748ca6dc8371
Parents: 00e7ecf
Author: mck <mi...@semb.wever.org>
Authored: Thu Jul 7 11:17:40 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:49:12 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  3 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 50 ++++++++++++++++----
 3 files changed, 44 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef18a176/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7d62f97..e10af6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.8
+ * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 Merged from 2.1:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef18a176/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index d86f941..ff63163 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -2347,7 +2347,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 }
 
                 // remove purgable tombstones from result - see CASSANDRA-11427
-                data.purgeTombstones(gcBefore(filter.timestamp));
+                if (data != null)
+                    data.purgeTombstones(gcBefore(filter.timestamp));
 
                 rows.add(new Row(rawRow.key, data));
                 if (!ignoreTombstonedPartitions || !data.hasOnlyTombstones(filter.timestamp))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef18a176/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 5419ef5..2d67baf 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -66,6 +66,7 @@ import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.composites.Composites;
 import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.filter.ExtendedFilter;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
@@ -94,7 +95,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
-import org.apache.thrift.TException;
 
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
@@ -246,6 +246,38 @@ public class ColumnFamilyStoreTest
     }
 
     @Test
+    public void testFilterWithNullCF() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        final Row row = new Row(Util.dk("key1"), null);
+
+        ColumnFamilyStore.AbstractScanIterator iterator = new ColumnFamilyStore.AbstractScanIterator()
+        {
+            Iterator<Row> it = Collections.singletonList(row).iterator();
+
+            protected Row computeNext()
+            {
+                return it.hasNext() ? it.next() : endOfData();
+            }
+
+            @Override
+            public void close()
+            {
+            }
+        };
+
+        ExtendedFilter filter = ExtendedFilter.create(
+                cfs,
+                DataRange.allData(DatabaseDescriptor.getPartitioner()), null, 1, true, System.currentTimeMillis());
+
+        List<Row> list = cfs.filter(iterator, filter);
+        assert 1 == list.size();
+        assert list.get(0).key == row.key;
+        assert null == list.get(0).cf;
+    }
+
+    @Test
     public void testSkipStartKey()
     {
         ColumnFamilyStore cfs = insertKey1Key2();
@@ -571,7 +603,7 @@ public class ColumnFamilyStoreTest
         cfs.truncateBlocking();
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
-        CellName colName = cellname("birthdate"); 
+        CellName colName = cellname("birthdate");
         ByteBuffer val1 = ByteBufferUtil.bytes(1L);
         ByteBuffer val2 = ByteBufferUtil.bytes(2L);
 
@@ -635,7 +667,7 @@ public class ColumnFamilyStoreTest
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
         ByteBuffer clusterKey = ByteBufferUtil.bytes("ck1");
-        ByteBuffer colName = ByteBufferUtil.bytes("col1"); 
+        ByteBuffer colName = ByteBufferUtil.bytes("col1");
 
         CellNameType baseComparator = cfs.getComparator();
         CellName compositeName = baseComparator.makeCellName(clusterKey, colName);
@@ -2291,7 +2323,7 @@ public class ColumnFamilyStoreTest
         });
         System.err.println("Row key: " + rowKey + " Cols: " + transformed);
     }
-    
+
     @Test
     public void testRebuildSecondaryIndex() throws IOException
     {
@@ -2303,19 +2335,19 @@ public class ColumnFamilyStoreTest
 
         rm.apply();
         assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
-        
+
         Keyspace.open("PerRowSecondaryIndex").getColumnFamilyStore("Indexed1").forceBlockingFlush();
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
-        
+
         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
         assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
         PerRowSecondaryIndexTest.TestIndex.ACTIVE = false;
         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
         assertNull(PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY);
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
     }
 }


[3/9] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by sl...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9

* cassandra-3.0:
  NPE when trying to remove purgable tombstones from result


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

Branch: refs/heads/trunk
Commit: 376dae26833591303cd3140001666f23aa216a11
Parents: 59ee46e 30f5d44
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 12:50:26 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:50:26 2016 +0200

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

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



[5/9] cassandra git commit: Don't ignore deletion info in sstable on reverse queries

Posted by sl...@apache.org.
Don't ignore deletion info in sstable on reverse queries

patch by Sylvain Lebresne; reviewed by Aleksey Yeschenko for CASSANDRA-11733


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

Branch: refs/heads/trunk
Commit: 76e68e9b49b1fbcb601633e6e2b8d8e1f71c7402
Parents: 30f5d44
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jun 30 15:13:24 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:54:52 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8118de1..20ed6e0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 14cec36..3e49a3a 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -356,7 +356,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         {
             deletionInfo = deletionBuilder.build();
             built = new ImmutableBTreePartition(metadata, partitionKey, columns, Rows.EMPTY_STATIC_ROW, rowBuilder.build(),
-                                                DeletionInfo.LIVE, EncodingStats.NO_STATS);
+                                                deletionInfo, EncodingStats.NO_STATS);
             deletionBuilder = null;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 76351ee..814e822 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -1057,4 +1057,30 @@ public class DeleteTest extends CQLTester
         if (forceFlush)
             flush();
     }
+
+    @Test
+    public void testDeleteAndReverseQueries() throws Throwable
+    {
+        // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
+        // validates we correctly get only the non-removed rows when doing reverse queries.
+
+        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
+
+        flush();
+
+        execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+
+        flush();
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+    }
 }


[6/9] cassandra git commit: Don't ignore deletion info in sstable on reverse queries

Posted by sl...@apache.org.
Don't ignore deletion info in sstable on reverse queries

patch by Sylvain Lebresne; reviewed by Aleksey Yeschenko for CASSANDRA-11733


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

Branch: refs/heads/cassandra-3.9
Commit: 76e68e9b49b1fbcb601633e6e2b8d8e1f71c7402
Parents: 30f5d44
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jun 30 15:13:24 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:54:52 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8118de1..20ed6e0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 14cec36..3e49a3a 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -356,7 +356,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         {
             deletionInfo = deletionBuilder.build();
             built = new ImmutableBTreePartition(metadata, partitionKey, columns, Rows.EMPTY_STATIC_ROW, rowBuilder.build(),
-                                                DeletionInfo.LIVE, EncodingStats.NO_STATS);
+                                                deletionInfo, EncodingStats.NO_STATS);
             deletionBuilder = null;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 76351ee..814e822 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -1057,4 +1057,30 @@ public class DeleteTest extends CQLTester
         if (forceFlush)
             flush();
     }
+
+    @Test
+    public void testDeleteAndReverseQueries() throws Throwable
+    {
+        // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
+        // validates we correctly get only the non-removed rows when doing reverse queries.
+
+        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
+
+        flush();
+
+        execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+
+        flush();
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+    }
 }


[9/9] cassandra git commit: Merge branch 'cassandra-3.9' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-3.9' into trunk

* cassandra-3.9:
  Don't ignore deletion info in sstable on reverse queries
  NPE when trying to remove purgable tombstones from result


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

Branch: refs/heads/trunk
Commit: 3016dc7c2f321c072dc11831be92a0331795ae89
Parents: 9fd6077 a006f57
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 13:02:53 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 13:02:53 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3016dc7c/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------


[7/9] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by sl...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9

* cassandra-3.0:
  Don't ignore deletion info in sstable on reverse queries


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

Branch: refs/heads/trunk
Commit: a006f577bdba7c4b248ef9f4cbd02a6c35a03162
Parents: 376dae2 76e68e9
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 12:59:34 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:59:34 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d459e34,20ed6e0..1d11149
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
   * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
   * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 9ead942,814e822..9b92ebb
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@@ -1105,4 -1051,36 +1105,30 @@@ public class DeleteTest extends CQLTest
          compact();
          assertRows(execute("SELECT * FROM %s"), row(0, null));
      }
+ 
 -    private void flush(boolean forceFlush)
 -    {
 -        if (forceFlush)
 -            flush();
 -    }
 -
+     @Test
+     public void testDeleteAndReverseQueries() throws Throwable
+     {
+         // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
+         // validates we correctly get only the non-removed rows when doing reverse queries.
+ 
+         createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+ 
+         for (int i = 0; i < 10; i++)
+             execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
+ 
+         flush();
+ 
+         execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
+ 
+         assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+             row(9), row(8), row(1), row(0)
+         );
+ 
+         flush();
+ 
+         assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+             row(9), row(8), row(1), row(0)
+         );
+     }
  }


[2/9] cassandra git commit: Merge commit 'ef18a17' into cassandra-3.0

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

* commit 'ef18a17':
  NPE when trying to remove purgable tombstones from result


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

Branch: refs/heads/trunk
Commit: 30f5d44d8cc53726fc9a17b6df4928ccd23af977
Parents: 778f2a4 ef18a17
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 12:50:03 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:50:03 2016 +0200

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

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



[4/9] cassandra git commit: Don't ignore deletion info in sstable on reverse queries

Posted by sl...@apache.org.
Don't ignore deletion info in sstable on reverse queries

patch by Sylvain Lebresne; reviewed by Aleksey Yeschenko for CASSANDRA-11733


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

Branch: refs/heads/cassandra-3.0
Commit: 76e68e9b49b1fbcb601633e6e2b8d8e1f71c7402
Parents: 30f5d44
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jun 30 15:13:24 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:54:52 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8118de1..20ed6e0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 14cec36..3e49a3a 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -356,7 +356,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         {
             deletionInfo = deletionBuilder.build();
             built = new ImmutableBTreePartition(metadata, partitionKey, columns, Rows.EMPTY_STATIC_ROW, rowBuilder.build(),
-                                                DeletionInfo.LIVE, EncodingStats.NO_STATS);
+                                                deletionInfo, EncodingStats.NO_STATS);
             deletionBuilder = null;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 76351ee..814e822 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -1057,4 +1057,30 @@ public class DeleteTest extends CQLTester
         if (forceFlush)
             flush();
     }
+
+    @Test
+    public void testDeleteAndReverseQueries() throws Throwable
+    {
+        // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
+        // validates we correctly get only the non-removed rows when doing reverse queries.
+
+        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
+
+        flush();
+
+        execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+
+        flush();
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+    }
 }


[8/9] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by sl...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9

* cassandra-3.0:
  Don't ignore deletion info in sstable on reverse queries


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

Branch: refs/heads/cassandra-3.9
Commit: a006f577bdba7c4b248ef9f4cbd02a6c35a03162
Parents: 376dae2 76e68e9
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 12:59:34 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:59:34 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d459e34,20ed6e0..1d11149
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
   * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
   * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 9ead942,814e822..9b92ebb
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@@ -1105,4 -1051,36 +1105,30 @@@ public class DeleteTest extends CQLTest
          compact();
          assertRows(execute("SELECT * FROM %s"), row(0, null));
      }
+ 
 -    private void flush(boolean forceFlush)
 -    {
 -        if (forceFlush)
 -            flush();
 -    }
 -
+     @Test
+     public void testDeleteAndReverseQueries() throws Throwable
+     {
+         // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
+         // validates we correctly get only the non-removed rows when doing reverse queries.
+ 
+         createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+ 
+         for (int i = 0; i < 10; i++)
+             execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
+ 
+         flush();
+ 
+         execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
+ 
+         assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+             row(9), row(8), row(1), row(0)
+         );
+ 
+         flush();
+ 
+         assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+             row(9), row(8), row(1), row(0)
+         );
+     }
  }