You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by if...@apache.org on 2017/04/07 10:21:33 UTC

[1/5] cassandra git commit: Make reading of range tombstones more reliable

Repository: cassandra
Updated Branches:
  refs/heads/3.0 [created] 2d6fd7824
  refs/heads/cassandra-3.11 5efaaf91c -> 863ad11c8
  refs/heads/trunk 94aa57e27 -> a1cb8e5ab


Make reading of range tombstones more reliable

Patch by Alex Petrov; reviewed by Benjamin Lerer for CASSANDRA-12811

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

Branch: refs/heads/cassandra-3.11
Commit: 2d6fd782465395d54d8958e2da8a5c8744a81942
Parents: 833c993
Author: Alex Petrov <ol...@gmail.com>
Authored: Fri Apr 7 12:09:32 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Fri Apr 7 12:10:46 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/SinglePartitionReadCommand.java          | 11 +--
 .../db/filter/ClusteringIndexNamesFilter.java   |  6 +-
 .../db/partitions/AbstractBTreePartition.java   |  5 --
 .../cassandra/utils/IndexedSearchIterator.java  |  5 ++
 .../apache/cassandra/utils/SearchIterator.java  |  2 -
 .../cql3/validation/operations/DeleteTest.java  | 82 +++++++++++++++++++-
 .../partition/PartitionImplementationTest.java  |  2 +-
 8 files changed, 92 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 33d5028..440ccd8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.13
+ * Make reading of range tombstones more reliable (CASSANDRA-12811)
  * Fix startup problems due to schema tables not completely flushed (CASSANDRA-12213)
  * Fix view builder bug that can filter out data on restart (CASSANDRA-13405)
  * Fix 2i page size calculation when there are no regular columns (CASSANDRA-13400)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 5f8df1b..99abd10 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -736,13 +736,13 @@ public class SinglePartitionReadCommand extends ReadCommand
 
                 // We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable.
                 sstable.incrementReadCount();
-                try (UnfilteredRowIterator iter = sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift()))
+                try (UnfilteredRowIterator iter = filter.filter(sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift())))
                 {
+                    sstablesIterated++;
                     if (!iter.partitionLevelDeletion().isLive())
-                    {
-                        sstablesIterated++;
                         result = add(UnfilteredRowIterators.noRowsIterator(iter.metadata(), iter.partitionKey(), Rows.EMPTY_STATIC_ROW, iter.partitionLevelDeletion(), filter.isReversed()), result, filter, sstable.isRepaired());
-                    }
+                    else
+                        result = add(iter, result, filter, sstable.isRepaired());
                 }
                 continue;
             }
@@ -835,9 +835,6 @@ public class SinglePartitionReadCommand extends ReadCommand
         NavigableSet<Clustering> toRemove = null;
         for (Clustering clustering : clusterings)
         {
-            if (!searchIter.hasNext())
-                break;
-
             Row row = searchIter.next(clustering);
             if (row == null || !canRemoveRow(row, columns.regulars, sstableTimestamp))
                 continue;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index a81a7a6..7769f2e 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@ -176,7 +176,9 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     public UnfilteredRowIterator getUnfilteredRowIterator(final ColumnFilter columnFilter, final Partition partition)
     {
+        final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
         final SearchIterator<Clustering, Row> searcher = partition.searchIterator(columnFilter, reversed);
+
         return new AbstractUnfilteredRowIterator(partition.metadata(),
                                         partition.partitionKey(),
                                         partition.partitionLevelDeletion(),
@@ -185,11 +187,9 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
                                         reversed,
                                         partition.stats())
         {
-            private final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
-
             protected Unfiltered computeNext()
             {
-                while (clusteringIter.hasNext() && searcher.hasNext())
+                while (clusteringIter.hasNext())
                 {
                     Row row = searcher.next(clusteringIter.next());
                     if (row != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index c63acc2..2aa622e 100644
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@ -139,11 +139,6 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
             private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata.comparator, desc(reversed));
             private final DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
 
-            public boolean hasNext()
-            {
-                return rawIter.hasNext();
-            }
-
             public Row next(Clustering clustering)
             {
                 if (clustering == Clustering.STATIC_CLUSTERING)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java b/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
index a156629..597e5bb 100644
--- a/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
+++ b/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
@@ -20,6 +20,11 @@ package org.apache.cassandra.utils;
 public interface IndexedSearchIterator<K, V> extends SearchIterator<K, V>
 {
     /**
+     * @return true if iterator has any elements left, false otherwise
+     */
+    public boolean hasNext();
+
+    /**
      * @return the value just recently returned by next()
      * @throws java.util.NoSuchElementException if next() returned null
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/utils/SearchIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/SearchIterator.java b/src/java/org/apache/cassandra/utils/SearchIterator.java
index 5309f4a..908053b 100644
--- a/src/java/org/apache/cassandra/utils/SearchIterator.java
+++ b/src/java/org/apache/cassandra/utils/SearchIterator.java
@@ -19,8 +19,6 @@ package org.apache.cassandra.utils;
 
 public interface SearchIterator<K, V>
 {
-    public boolean hasNext();
-
     /**
      * Searches "forwards" (in direction of travel) in the iterator for the required key;
      * if this or any key greater has already been returned by the iterator, the method may

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/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 9f770a5..4c9f4d6 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -43,15 +43,89 @@ public class DeleteTest extends CQLTester
     @Test
     public void testRangeDeletion() throws Throwable
     {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+        testRangeDeletion(true, true);
+        testRangeDeletion(false, true);
+        testRangeDeletion(true, false);
+        testRangeDeletion(false, false);
+    }
 
+    private void testRangeDeletion(boolean flushData, boolean flushTombstone) throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
-        flush();
+        flush(flushData);
         execute("DELETE FROM %s WHERE a=? AND b=?", 1, 1);
-        flush();
+        flush(flushTombstone);
         assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 1, 1, 1));
     }
 
+    @Test
+    public void testDeleteRange() throws Throwable
+    {
+        testDeleteRange(true, true);
+        testDeleteRange(false, true);
+        testDeleteRange(true, false);
+        testDeleteRange(false, false);
+    }
+
+    private void testDeleteRange(boolean flushData, boolean flushTombstone) throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 1, 2);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 2, 3);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 3, 4);
+        flush(flushData);
+
+        execute("DELETE FROM %s WHERE a = ? AND b >= ?", 2, 2);
+        flush(flushTombstone);
+
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s"),
+                                row(1, 1, 1),
+                                row(2, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 1),
+                   row(2, 1, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 3));
+    }
+
+    @Test
+    public void testCrossMemSSTableMultiColumn() throws Throwable
+    {
+        testCrossMemSSTableMultiColumn(true, true);
+        testCrossMemSSTableMultiColumn(false, true);
+        testCrossMemSSTableMultiColumn(true, false);
+        testCrossMemSSTableMultiColumn(false, false);
+    }
+
+    private void testCrossMemSSTableMultiColumn(boolean flushData, boolean flushTombstone) throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 1, 2);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 2, 2);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 3, 3);
+        flush(flushData);
+
+        execute("DELETE FROM %s WHERE a = ? AND (b) = (?)", 2, 2);
+        execute("DELETE FROM %s WHERE a = ? AND (b) = (?)", 2, 3);
+
+        flush(flushTombstone);
+
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s"),
+                                row(1, 1, 1),
+                                row(2, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 1),
+                   row(2, 1, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 3));
+    }
+
+
     /**
      * Test simple deletion and in particular check for #4193 bug
      * migrated from cql_tests.py:TestCQL.deletion_test()
@@ -440,7 +514,7 @@ public class DeleteTest extends CQLTester
                 assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1));
             }
 
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) = (?)", 0, 1);
+            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
             flush(forceFlush);
             assertEmpty(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
index f215331..f4c93d6 100644
--- a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
+++ b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
@@ -326,7 +326,7 @@ public class PartitionImplementationTest
         int pos = reversed ? KEY_RANGE : 0;
         int mul = reversed ? -1 : 1;
         boolean started = false;
-        while (searchIter.hasNext())
+        while (pos < KEY_RANGE)
         {
             int skip = rand.nextInt(KEY_RANGE / 10);
             pos += skip * mul;


[2/5] cassandra git commit: Make reading of range tombstones more reliable

Posted by if...@apache.org.
Make reading of range tombstones more reliable

Patch by Alex Petrov; reviewed by Benjamin Lerer for CASSANDRA-12811

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

Branch: refs/heads/trunk
Commit: 2d6fd782465395d54d8958e2da8a5c8744a81942
Parents: 833c993
Author: Alex Petrov <ol...@gmail.com>
Authored: Fri Apr 7 12:09:32 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Fri Apr 7 12:10:46 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/SinglePartitionReadCommand.java          | 11 +--
 .../db/filter/ClusteringIndexNamesFilter.java   |  6 +-
 .../db/partitions/AbstractBTreePartition.java   |  5 --
 .../cassandra/utils/IndexedSearchIterator.java  |  5 ++
 .../apache/cassandra/utils/SearchIterator.java  |  2 -
 .../cql3/validation/operations/DeleteTest.java  | 82 +++++++++++++++++++-
 .../partition/PartitionImplementationTest.java  |  2 +-
 8 files changed, 92 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 33d5028..440ccd8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.13
+ * Make reading of range tombstones more reliable (CASSANDRA-12811)
  * Fix startup problems due to schema tables not completely flushed (CASSANDRA-12213)
  * Fix view builder bug that can filter out data on restart (CASSANDRA-13405)
  * Fix 2i page size calculation when there are no regular columns (CASSANDRA-13400)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 5f8df1b..99abd10 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -736,13 +736,13 @@ public class SinglePartitionReadCommand extends ReadCommand
 
                 // We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable.
                 sstable.incrementReadCount();
-                try (UnfilteredRowIterator iter = sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift()))
+                try (UnfilteredRowIterator iter = filter.filter(sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift())))
                 {
+                    sstablesIterated++;
                     if (!iter.partitionLevelDeletion().isLive())
-                    {
-                        sstablesIterated++;
                         result = add(UnfilteredRowIterators.noRowsIterator(iter.metadata(), iter.partitionKey(), Rows.EMPTY_STATIC_ROW, iter.partitionLevelDeletion(), filter.isReversed()), result, filter, sstable.isRepaired());
-                    }
+                    else
+                        result = add(iter, result, filter, sstable.isRepaired());
                 }
                 continue;
             }
@@ -835,9 +835,6 @@ public class SinglePartitionReadCommand extends ReadCommand
         NavigableSet<Clustering> toRemove = null;
         for (Clustering clustering : clusterings)
         {
-            if (!searchIter.hasNext())
-                break;
-
             Row row = searchIter.next(clustering);
             if (row == null || !canRemoveRow(row, columns.regulars, sstableTimestamp))
                 continue;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index a81a7a6..7769f2e 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@ -176,7 +176,9 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     public UnfilteredRowIterator getUnfilteredRowIterator(final ColumnFilter columnFilter, final Partition partition)
     {
+        final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
         final SearchIterator<Clustering, Row> searcher = partition.searchIterator(columnFilter, reversed);
+
         return new AbstractUnfilteredRowIterator(partition.metadata(),
                                         partition.partitionKey(),
                                         partition.partitionLevelDeletion(),
@@ -185,11 +187,9 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
                                         reversed,
                                         partition.stats())
         {
-            private final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
-
             protected Unfiltered computeNext()
             {
-                while (clusteringIter.hasNext() && searcher.hasNext())
+                while (clusteringIter.hasNext())
                 {
                     Row row = searcher.next(clusteringIter.next());
                     if (row != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index c63acc2..2aa622e 100644
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@ -139,11 +139,6 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
             private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata.comparator, desc(reversed));
             private final DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
 
-            public boolean hasNext()
-            {
-                return rawIter.hasNext();
-            }
-
             public Row next(Clustering clustering)
             {
                 if (clustering == Clustering.STATIC_CLUSTERING)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java b/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
index a156629..597e5bb 100644
--- a/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
+++ b/src/java/org/apache/cassandra/utils/IndexedSearchIterator.java
@@ -20,6 +20,11 @@ package org.apache.cassandra.utils;
 public interface IndexedSearchIterator<K, V> extends SearchIterator<K, V>
 {
     /**
+     * @return true if iterator has any elements left, false otherwise
+     */
+    public boolean hasNext();
+
+    /**
      * @return the value just recently returned by next()
      * @throws java.util.NoSuchElementException if next() returned null
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/src/java/org/apache/cassandra/utils/SearchIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/SearchIterator.java b/src/java/org/apache/cassandra/utils/SearchIterator.java
index 5309f4a..908053b 100644
--- a/src/java/org/apache/cassandra/utils/SearchIterator.java
+++ b/src/java/org/apache/cassandra/utils/SearchIterator.java
@@ -19,8 +19,6 @@ package org.apache.cassandra.utils;
 
 public interface SearchIterator<K, V>
 {
-    public boolean hasNext();
-
     /**
      * Searches "forwards" (in direction of travel) in the iterator for the required key;
      * if this or any key greater has already been returned by the iterator, the method may

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/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 9f770a5..4c9f4d6 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -43,15 +43,89 @@ public class DeleteTest extends CQLTester
     @Test
     public void testRangeDeletion() throws Throwable
     {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+        testRangeDeletion(true, true);
+        testRangeDeletion(false, true);
+        testRangeDeletion(true, false);
+        testRangeDeletion(false, false);
+    }
 
+    private void testRangeDeletion(boolean flushData, boolean flushTombstone) throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
-        flush();
+        flush(flushData);
         execute("DELETE FROM %s WHERE a=? AND b=?", 1, 1);
-        flush();
+        flush(flushTombstone);
         assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 1, 1, 1));
     }
 
+    @Test
+    public void testDeleteRange() throws Throwable
+    {
+        testDeleteRange(true, true);
+        testDeleteRange(false, true);
+        testDeleteRange(true, false);
+        testDeleteRange(false, false);
+    }
+
+    private void testDeleteRange(boolean flushData, boolean flushTombstone) throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 1, 2);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 2, 3);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 3, 4);
+        flush(flushData);
+
+        execute("DELETE FROM %s WHERE a = ? AND b >= ?", 2, 2);
+        flush(flushTombstone);
+
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s"),
+                                row(1, 1, 1),
+                                row(2, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 1),
+                   row(2, 1, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 3));
+    }
+
+    @Test
+    public void testCrossMemSSTableMultiColumn() throws Throwable
+    {
+        testCrossMemSSTableMultiColumn(true, true);
+        testCrossMemSSTableMultiColumn(false, true);
+        testCrossMemSSTableMultiColumn(true, false);
+        testCrossMemSSTableMultiColumn(false, false);
+    }
+
+    private void testCrossMemSSTableMultiColumn(boolean flushData, boolean flushTombstone) throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 1, 2);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 2, 2);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, 3, 3);
+        flush(flushData);
+
+        execute("DELETE FROM %s WHERE a = ? AND (b) = (?)", 2, 2);
+        execute("DELETE FROM %s WHERE a = ? AND (b) = (?)", 2, 3);
+
+        flush(flushTombstone);
+
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s"),
+                                row(1, 1, 1),
+                                row(2, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 1),
+                   row(2, 1, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 2, 3));
+    }
+
+
     /**
      * Test simple deletion and in particular check for #4193 bug
      * migrated from cql_tests.py:TestCQL.deletion_test()
@@ -440,7 +514,7 @@ public class DeleteTest extends CQLTester
                 assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1));
             }
 
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) = (?)", 0, 1);
+            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
             flush(forceFlush);
             assertEmpty(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2d6fd782/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
index f215331..f4c93d6 100644
--- a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
+++ b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
@@ -326,7 +326,7 @@ public class PartitionImplementationTest
         int pos = reversed ? KEY_RANGE : 0;
         int mul = reversed ? -1 : 1;
         boolean started = false;
-        while (searchIter.hasNext())
+        while (pos < KEY_RANGE)
         {
             int skip = rand.nextInt(KEY_RANGE / 10);
             pos += skip * mul;


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

Posted by if...@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/a1cb8e5a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a1cb8e5a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a1cb8e5a

Branch: refs/heads/trunk
Commit: a1cb8e5abc224623c35fe84969dc08894c91278a
Parents: 94aa57e 863ad11
Author: Alex Petrov <ol...@gmail.com>
Authored: Fri Apr 7 12:20:02 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Fri Apr 7 12:20:02 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/SinglePartitionReadCommand.java          | 11 +--
 .../db/filter/ClusteringIndexNamesFilter.java   |  6 +-
 .../db/partitions/AbstractBTreePartition.java   |  5 --
 .../cassandra/utils/IndexedSearchIterator.java  |  5 ++
 .../apache/cassandra/utils/SearchIterator.java  |  2 -
 .../cassandra/utils/memory/EnsureOnHeap.java    |  5 --
 .../cql3/validation/operations/DeleteTest.java  | 82 +++++++++++++++++++-
 .../partition/PartitionImplementationTest.java  |  2 +-
 9 files changed, 92 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a1cb8e5a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 1de02a9,1358f29..d5d2901
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@@ -744,13 -778,13 +744,13 @@@ public class SinglePartitionReadComman
  
                  // We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable.
                  sstable.incrementReadCount();
-                 try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), Slices.ALL, columnFilter(), filter.isReversed()))
 -                try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), filter.getSlices(metadata()), columnFilter(), filter.isReversed(), isForThrift()))
++                try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), filter.getSlices(metadata()), columnFilter(), filter.isReversed()))
                  {
+                     sstablesIterated++;
                      if (!iter.partitionLevelDeletion().isLive())
-                     {
-                         sstablesIterated++;
                          result = add(UnfilteredRowIterators.noRowsIterator(iter.metadata(), iter.partitionKey(), Rows.EMPTY_STATIC_ROW, iter.partitionLevelDeletion(), filter.isReversed()), result, filter, sstable.isRepaired());
-                     }
+                     else
+                         result = add(iter, result, filter, sstable.isRepaired());
                  }
                  continue;
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a1cb8e5a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index 2f7c13a,31535fc..264e327
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@@ -136,20 -136,20 +136,20 @@@ public class ClusteringIndexNamesFilte
  
      public UnfilteredRowIterator getUnfilteredRowIterator(final ColumnFilter columnFilter, final Partition partition)
      {
+         final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
          final SearchIterator<Clustering, Row> searcher = partition.searchIterator(columnFilter, reversed);
+ 
          return new AbstractUnfilteredRowIterator(partition.metadata(),
 -                                        partition.partitionKey(),
 -                                        partition.partitionLevelDeletion(),
 -                                        columnFilter.fetchedColumns(),
 -                                        searcher.next(Clustering.STATIC_CLUSTERING),
 -                                        reversed,
 -                                        partition.stats())
 +                                                 partition.partitionKey(),
 +                                                 partition.partitionLevelDeletion(),
 +                                                 columnFilter.fetchedColumns(),
 +                                                 searcher.next(Clustering.STATIC_CLUSTERING),
 +                                                 reversed,
 +                                                 partition.stats())
          {
-             private final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
- 
              protected Unfiltered computeNext()
              {
-                 while (clusteringIter.hasNext() && searcher.hasNext())
+                 while (clusteringIter.hasNext())
                  {
                      Row row = searcher.next(clusteringIter.next());
                      if (row != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a1cb8e5a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index 465dec3,bb9201b..d8f2856
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@@ -131,14 -136,9 +131,9 @@@ public abstract class AbstractBTreePart
          final Holder current = holder();
          return new SearchIterator<Clustering, Row>()
          {
 -            private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata.comparator, desc(reversed));
 +            private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata().comparator, desc(reversed));
              private final DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
  
-             public boolean hasNext()
-             {
-                 return rawIter.hasNext();
-             }
- 
              public Row next(Clustering clustering)
              {
                  if (clustering == Clustering.STATIC_CLUSTERING)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a1cb8e5a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
----------------------------------------------------------------------


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

Posted by if...@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/863ad11c
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/863ad11c
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/863ad11c

Branch: refs/heads/trunk
Commit: 863ad11c8acb6c5f811d2cb2668eb353863f29e6
Parents: 5efaaf9 2d6fd78
Author: Alex Petrov <ol...@gmail.com>
Authored: Fri Apr 7 12:16:22 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Fri Apr 7 12:16:22 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/SinglePartitionReadCommand.java          | 11 +--
 .../db/filter/ClusteringIndexNamesFilter.java   |  6 +-
 .../db/partitions/AbstractBTreePartition.java   |  5 --
 .../cassandra/utils/IndexedSearchIterator.java  |  5 ++
 .../apache/cassandra/utils/SearchIterator.java  |  2 -
 .../cassandra/utils/memory/EnsureOnHeap.java    |  5 --
 .../cql3/validation/operations/DeleteTest.java  | 82 +++++++++++++++++++-
 .../partition/PartitionImplementationTest.java  |  2 +-
 9 files changed, 92 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 26e64d2,440ccd8..aa8c3ff
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,27 -1,5 +1,28 @@@
 -3.0.13
 +3.11.0
 + * Avoid rebuilding SASI indexes containing no values (CASSANDRA-12962)
 + * Add charset to Analyser input stream (CASSANDRA-13151)
 + * Fix testLimitSSTables flake caused by concurrent flush (CASSANDRA-12820)
 + * cdc column addition strikes again (CASSANDRA-13382)
 + * Fix static column indexes (CASSANDRA-13277)
 + * DataOutputBuffer.asNewBuffer broken (CASSANDRA-13298)
 + * unittest CipherFactoryTest failed on MacOS (CASSANDRA-13370)
 + * Forbid SELECT restrictions and CREATE INDEX over non-frozen UDT columns (CASSANDRA-13247)
 + * Default logging we ship will incorrectly print "?:?" for "%F:%L" pattern (CASSANDRA-13317)
 + * Possible AssertionError in UnfilteredRowIteratorWithLowerBound (CASSANDRA-13366)
 + * Support unaligned memory access for AArch64 (CASSANDRA-13326)
 + * Improve SASI range iterator efficiency on intersection with an empty range (CASSANDRA-12915).
 + * Fix equality comparisons of columns using the duration type (CASSANDRA-13174)
 + * Obfuscate password in stress-graphs (CASSANDRA-12233)
 + * Move to FastThreadLocalThread and FastThreadLocal (CASSANDRA-13034)
 + * nodetool stopdaemon errors out (CASSANDRA-13030)
 + * Tables in system_distributed should not use gcgs of 0 (CASSANDRA-12954)
 + * Fix primary index calculation for SASI (CASSANDRA-12910)
 + * More fixes to the TokenAllocator (CASSANDRA-12990)
 + * NoReplicationTokenAllocator should work with zero replication factor (CASSANDRA-12983)
 + * Address message coalescing regression (CASSANDRA-12676)
 + * Delete illegal character from StandardTokenizerImpl.jflex (CASSANDRA-13417)
 +Merged from 3.0:
+  * Make reading of range tombstones more reliable (CASSANDRA-12811)
   * Fix startup problems due to schema tables not completely flushed (CASSANDRA-12213)
   * Fix view builder bug that can filter out data on restart (CASSANDRA-13405)
   * Fix 2i page size calculation when there are no regular columns (CASSANDRA-13400)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 724f59e,99abd10..1358f29
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@@ -778,13 -736,13 +778,13 @@@ public class SinglePartitionReadComman
  
                  // We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable.
                  sstable.incrementReadCount();
-                 try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), Slices.ALL, columnFilter(), filter.isReversed(), isForThrift()))
 -                try (UnfilteredRowIterator iter = filter.filter(sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift())))
++                try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), filter.getSlices(metadata()), columnFilter(), filter.isReversed(), isForThrift()))
                  {
+                     sstablesIterated++;
                      if (!iter.partitionLevelDeletion().isLive())
-                     {
-                         sstablesIterated++;
                          result = add(UnfilteredRowIterators.noRowsIterator(iter.metadata(), iter.partitionKey(), Rows.EMPTY_STATIC_ROW, iter.partitionLevelDeletion(), filter.isReversed()), result, filter, sstable.isRepaired());
-                     }
+                     else
+                         result = add(iter, result, filter, sstable.isRepaired());
                  }
                  continue;
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/utils/SearchIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
index 8345118,0000000..54ace5e
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
+++ b/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
@@@ -1,170 -1,0 +1,165 @@@
 +/*
 + *
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + *
 + */
 +package org.apache.cassandra.utils.memory;
 +
 +import java.util.Iterator;
 +
 +import org.apache.cassandra.db.BufferDecoratedKey;
 +import org.apache.cassandra.db.Clustering;
 +import org.apache.cassandra.db.DecoratedKey;
 +import org.apache.cassandra.db.DeletionInfo;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.db.transform.Transformation;
 +import org.apache.cassandra.utils.SearchIterator;
 +
 +public abstract class EnsureOnHeap extends Transformation
 +{
 +    public abstract DecoratedKey applyToPartitionKey(DecoratedKey key);
 +    public abstract UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition);
 +    public abstract SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition);
 +    public abstract Iterator<Row> applyToPartition(Iterator<Row> partition);
 +    public abstract DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo);
 +    public abstract Row applyToRow(Row row);
 +    public abstract Row applyToStatic(Row row);
 +    public abstract RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker);
 +
 +    static class CloneToHeap extends EnsureOnHeap
 +    {
 +        protected BaseRowIterator<?> applyToPartition(BaseRowIterator partition)
 +        {
 +            return partition instanceof UnfilteredRowIterator
 +                   ? Transformation.apply((UnfilteredRowIterator) partition, this)
 +                   : Transformation.apply((RowIterator) partition, this);
 +        }
 +
 +        public DecoratedKey applyToPartitionKey(DecoratedKey key)
 +        {
 +            return new BufferDecoratedKey(key.getToken(), HeapAllocator.instance.clone(key.getKey()));
 +        }
 +
 +        public Row applyToRow(Row row)
 +        {
 +            if (row == null)
 +                return null;
 +            return Rows.copy(row, HeapAllocator.instance.cloningBTreeRowBuilder()).build();
 +        }
 +
 +        public Row applyToStatic(Row row)
 +        {
 +            if (row == Rows.EMPTY_STATIC_ROW)
 +                return row;
 +            return applyToRow(row);
 +        }
 +
 +        public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +        {
 +            return marker.copy(HeapAllocator.instance);
 +        }
 +
 +        public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition)
 +        {
 +            return Transformation.apply(partition, this);
 +        }
 +
 +        public SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition)
 +        {
 +            return new SearchIterator<Clustering, Row>()
 +            {
-                 public boolean hasNext()
-                 {
-                     return partition.hasNext();
-                 }
- 
 +                public Row next(Clustering key)
 +                {
 +                    return applyToRow(partition.next(key));
 +                }
 +            };
 +        }
 +
 +        public Iterator<Row> applyToPartition(Iterator<Row> partition)
 +        {
 +            return new Iterator<Row>()
 +            {
 +                public boolean hasNext()
 +                {
 +                    return partition.hasNext();
 +                }
 +                public Row next()
 +                {
 +                    return applyToRow(partition.next());
 +                }
 +                public void remove()
 +                {
 +                    partition.remove();
 +                }
 +            };
 +        }
 +
 +        public DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo)
 +        {
 +            return deletionInfo.copy(HeapAllocator.instance);
 +        }
 +    }
 +
 +    static class NoOp extends EnsureOnHeap
 +    {
 +        protected BaseRowIterator<?> applyToPartition(BaseRowIterator partition)
 +        {
 +            return partition;
 +        }
 +
 +        public DecoratedKey applyToPartitionKey(DecoratedKey key)
 +        {
 +            return key;
 +        }
 +
 +        public Row applyToRow(Row row)
 +        {
 +            return row;
 +        }
 +
 +        public Row applyToStatic(Row row)
 +        {
 +            return row;
 +        }
 +
 +        public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +        {
 +            return marker;
 +        }
 +
 +        public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition)
 +        {
 +            return partition;
 +        }
 +
 +        public SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition)
 +        {
 +            return partition;
 +        }
 +
 +        public Iterator<Row> applyToPartition(Iterator<Row> partition)
 +        {
 +            return partition;
 +        }
 +
 +        public DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo)
 +        {
 +            return deletionInfo;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
----------------------------------------------------------------------


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

Posted by if...@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/863ad11c
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/863ad11c
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/863ad11c

Branch: refs/heads/cassandra-3.11
Commit: 863ad11c8acb6c5f811d2cb2668eb353863f29e6
Parents: 5efaaf9 2d6fd78
Author: Alex Petrov <ol...@gmail.com>
Authored: Fri Apr 7 12:16:22 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Fri Apr 7 12:16:22 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/SinglePartitionReadCommand.java          | 11 +--
 .../db/filter/ClusteringIndexNamesFilter.java   |  6 +-
 .../db/partitions/AbstractBTreePartition.java   |  5 --
 .../cassandra/utils/IndexedSearchIterator.java  |  5 ++
 .../apache/cassandra/utils/SearchIterator.java  |  2 -
 .../cassandra/utils/memory/EnsureOnHeap.java    |  5 --
 .../cql3/validation/operations/DeleteTest.java  | 82 +++++++++++++++++++-
 .../partition/PartitionImplementationTest.java  |  2 +-
 9 files changed, 92 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 26e64d2,440ccd8..aa8c3ff
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,27 -1,5 +1,28 @@@
 -3.0.13
 +3.11.0
 + * Avoid rebuilding SASI indexes containing no values (CASSANDRA-12962)
 + * Add charset to Analyser input stream (CASSANDRA-13151)
 + * Fix testLimitSSTables flake caused by concurrent flush (CASSANDRA-12820)
 + * cdc column addition strikes again (CASSANDRA-13382)
 + * Fix static column indexes (CASSANDRA-13277)
 + * DataOutputBuffer.asNewBuffer broken (CASSANDRA-13298)
 + * unittest CipherFactoryTest failed on MacOS (CASSANDRA-13370)
 + * Forbid SELECT restrictions and CREATE INDEX over non-frozen UDT columns (CASSANDRA-13247)
 + * Default logging we ship will incorrectly print "?:?" for "%F:%L" pattern (CASSANDRA-13317)
 + * Possible AssertionError in UnfilteredRowIteratorWithLowerBound (CASSANDRA-13366)
 + * Support unaligned memory access for AArch64 (CASSANDRA-13326)
 + * Improve SASI range iterator efficiency on intersection with an empty range (CASSANDRA-12915).
 + * Fix equality comparisons of columns using the duration type (CASSANDRA-13174)
 + * Obfuscate password in stress-graphs (CASSANDRA-12233)
 + * Move to FastThreadLocalThread and FastThreadLocal (CASSANDRA-13034)
 + * nodetool stopdaemon errors out (CASSANDRA-13030)
 + * Tables in system_distributed should not use gcgs of 0 (CASSANDRA-12954)
 + * Fix primary index calculation for SASI (CASSANDRA-12910)
 + * More fixes to the TokenAllocator (CASSANDRA-12990)
 + * NoReplicationTokenAllocator should work with zero replication factor (CASSANDRA-12983)
 + * Address message coalescing regression (CASSANDRA-12676)
 + * Delete illegal character from StandardTokenizerImpl.jflex (CASSANDRA-13417)
 +Merged from 3.0:
+  * Make reading of range tombstones more reliable (CASSANDRA-12811)
   * Fix startup problems due to schema tables not completely flushed (CASSANDRA-12213)
   * Fix view builder bug that can filter out data on restart (CASSANDRA-13405)
   * Fix 2i page size calculation when there are no regular columns (CASSANDRA-13400)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 724f59e,99abd10..1358f29
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@@ -778,13 -736,13 +778,13 @@@ public class SinglePartitionReadComman
  
                  // We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable.
                  sstable.incrementReadCount();
-                 try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), Slices.ALL, columnFilter(), filter.isReversed(), isForThrift()))
 -                try (UnfilteredRowIterator iter = filter.filter(sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift())))
++                try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), filter.getSlices(metadata()), columnFilter(), filter.isReversed(), isForThrift()))
                  {
+                     sstablesIterated++;
                      if (!iter.partitionLevelDeletion().isLive())
-                     {
-                         sstablesIterated++;
                          result = add(UnfilteredRowIterators.noRowsIterator(iter.metadata(), iter.partitionKey(), Rows.EMPTY_STATIC_ROW, iter.partitionLevelDeletion(), filter.isReversed()), result, filter, sstable.isRepaired());
-                     }
+                     else
+                         result = add(iter, result, filter, sstable.isRepaired());
                  }
                  continue;
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/utils/SearchIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
index 8345118,0000000..54ace5e
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
+++ b/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
@@@ -1,170 -1,0 +1,165 @@@
 +/*
 + *
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + *
 + */
 +package org.apache.cassandra.utils.memory;
 +
 +import java.util.Iterator;
 +
 +import org.apache.cassandra.db.BufferDecoratedKey;
 +import org.apache.cassandra.db.Clustering;
 +import org.apache.cassandra.db.DecoratedKey;
 +import org.apache.cassandra.db.DeletionInfo;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.db.transform.Transformation;
 +import org.apache.cassandra.utils.SearchIterator;
 +
 +public abstract class EnsureOnHeap extends Transformation
 +{
 +    public abstract DecoratedKey applyToPartitionKey(DecoratedKey key);
 +    public abstract UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition);
 +    public abstract SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition);
 +    public abstract Iterator<Row> applyToPartition(Iterator<Row> partition);
 +    public abstract DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo);
 +    public abstract Row applyToRow(Row row);
 +    public abstract Row applyToStatic(Row row);
 +    public abstract RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker);
 +
 +    static class CloneToHeap extends EnsureOnHeap
 +    {
 +        protected BaseRowIterator<?> applyToPartition(BaseRowIterator partition)
 +        {
 +            return partition instanceof UnfilteredRowIterator
 +                   ? Transformation.apply((UnfilteredRowIterator) partition, this)
 +                   : Transformation.apply((RowIterator) partition, this);
 +        }
 +
 +        public DecoratedKey applyToPartitionKey(DecoratedKey key)
 +        {
 +            return new BufferDecoratedKey(key.getToken(), HeapAllocator.instance.clone(key.getKey()));
 +        }
 +
 +        public Row applyToRow(Row row)
 +        {
 +            if (row == null)
 +                return null;
 +            return Rows.copy(row, HeapAllocator.instance.cloningBTreeRowBuilder()).build();
 +        }
 +
 +        public Row applyToStatic(Row row)
 +        {
 +            if (row == Rows.EMPTY_STATIC_ROW)
 +                return row;
 +            return applyToRow(row);
 +        }
 +
 +        public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +        {
 +            return marker.copy(HeapAllocator.instance);
 +        }
 +
 +        public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition)
 +        {
 +            return Transformation.apply(partition, this);
 +        }
 +
 +        public SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition)
 +        {
 +            return new SearchIterator<Clustering, Row>()
 +            {
-                 public boolean hasNext()
-                 {
-                     return partition.hasNext();
-                 }
- 
 +                public Row next(Clustering key)
 +                {
 +                    return applyToRow(partition.next(key));
 +                }
 +            };
 +        }
 +
 +        public Iterator<Row> applyToPartition(Iterator<Row> partition)
 +        {
 +            return new Iterator<Row>()
 +            {
 +                public boolean hasNext()
 +                {
 +                    return partition.hasNext();
 +                }
 +                public Row next()
 +                {
 +                    return applyToRow(partition.next());
 +                }
 +                public void remove()
 +                {
 +                    partition.remove();
 +                }
 +            };
 +        }
 +
 +        public DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo)
 +        {
 +            return deletionInfo.copy(HeapAllocator.instance);
 +        }
 +    }
 +
 +    static class NoOp extends EnsureOnHeap
 +    {
 +        protected BaseRowIterator<?> applyToPartition(BaseRowIterator partition)
 +        {
 +            return partition;
 +        }
 +
 +        public DecoratedKey applyToPartitionKey(DecoratedKey key)
 +        {
 +            return key;
 +        }
 +
 +        public Row applyToRow(Row row)
 +        {
 +            return row;
 +        }
 +
 +        public Row applyToStatic(Row row)
 +        {
 +            return row;
 +        }
 +
 +        public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +        {
 +            return marker;
 +        }
 +
 +        public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition)
 +        {
 +            return partition;
 +        }
 +
 +        public SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition)
 +        {
 +            return partition;
 +        }
 +
 +        public Iterator<Row> applyToPartition(Iterator<Row> partition)
 +        {
 +            return partition;
 +        }
 +
 +        public DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo)
 +        {
 +            return deletionInfo;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/863ad11c/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
----------------------------------------------------------------------