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/02/23 11:48:48 UTC

[2/3] cassandra git commit: Fix ClusteringPrefix not being able to read tombstone range boundaries.

Fix ClusteringPrefix not being able to read tombstone range boundaries.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for CASSANDRA-11158


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

Branch: refs/heads/trunk
Commit: fe37e0644ccb831dda0a7d11a31b02e0ad8232fa
Parents: b3aa334
Author: Branimir Lambov <br...@datastax.com>
Authored: Fri Feb 19 17:02:50 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Feb 23 11:47:57 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ClusteringPrefix.java   |   6 +-
 .../org/apache/cassandra/db/RangeTombstone.java |   6 +
 .../cql3/QueryWithIndexedSSTableTest.java       |   4 +-
 .../cql3/TombstonesWithIndexedSSTableTest.java  | 116 +++++++++++++++++++
 5 files changed, 128 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe37e064/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a6ded6f..da91594 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.4
+ * Fix ClusteringPrefix not being able to read tombstone range boundaries (CASSANDRA-11158)
  * Prevent logging in sandboxed state (CASSANDRA-11033)
  * Disallow drop/alter operations of UDTs used by UDAs (CASSANDRA-10721)
  * Add query time validation method on Index (CASSANDRA-11043)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe37e064/src/java/org/apache/cassandra/db/ClusteringPrefix.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
index 9477651..17befca 100644
--- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
@@ -259,7 +259,7 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
             }
             else
             {
-                Slice.Bound.serializer.serialize((Slice.Bound)clustering, out, version, types);
+                RangeTombstone.Bound.serializer.serialize((RangeTombstone.Bound)clustering, out, version, types);
             }
         }
 
@@ -271,7 +271,7 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
             if (kind == Kind.CLUSTERING)
                 return Clustering.serializer.deserialize(in, version, types);
             else
-                return Slice.Bound.serializer.deserializeValues(in, kind, version, types);
+                return RangeTombstone.Bound.serializer.deserializeValues(in, kind, version, types);
         }
 
         public long serializedSize(ClusteringPrefix clustering, int version, List<AbstractType<?>> types)
@@ -281,7 +281,7 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
             if (clustering.kind() == Kind.CLUSTERING)
                 return 1 + Clustering.serializer.serializedSize((Clustering)clustering, version, types);
             else
-                return Slice.Bound.serializer.serializedSize((Slice.Bound)clustering, version, types);
+                return RangeTombstone.Bound.serializer.serializedSize((RangeTombstone.Bound)clustering, version, types);
         }
 
         void serializeValuesWithoutSize(ClusteringPrefix clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe37e064/src/java/org/apache/cassandra/db/RangeTombstone.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeTombstone.java b/src/java/org/apache/cassandra/db/RangeTombstone.java
index 54199ab..8af3b97 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstone.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstone.java
@@ -199,6 +199,12 @@ public class RangeTombstone
             public RangeTombstone.Bound deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
             {
                 Kind kind = Kind.values()[in.readByte()];
+                return deserializeValues(in, kind, version, types);
+            }
+
+            public RangeTombstone.Bound deserializeValues(DataInputPlus in, Kind kind, int version,
+                    List<AbstractType<?>> types) throws IOException
+            {
                 int size = in.readUnsignedShort();
                 if (size == 0)
                     return kind.isStart() ? BOTTOM : TOP;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe37e064/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java b/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java
index 4838392..01a2afd 100644
--- a/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java
+++ b/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java
@@ -43,7 +43,7 @@ public class QueryWithIndexedSSTableTest extends CQLTester
 
         // We create a partition that is big enough that the underlying sstable will be indexed
         // For that, we use a large-ish number of row, and a value that isn't too small.
-        String text = makeRandomTest(VALUE_LENGTH);
+        String text = TombstonesWithIndexedSSTableTest.makeRandomString(VALUE_LENGTH);
         for (int i = 0; i < ROWS; i++)
             execute("INSERT INTO %s(k, t, v) VALUES (?, ?, ?)", 0, i, text + i);
 
@@ -72,7 +72,7 @@ public class QueryWithIndexedSSTableTest extends CQLTester
     }
 
     // Creates a random string 
-    public static String makeRandomTest(int length)
+    public static String makeRandomSt(int length)
     {
         Random random = new Random();
         char[] chars = new char[26];

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe37e064/test/unit/org/apache/cassandra/cql3/TombstonesWithIndexedSSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TombstonesWithIndexedSSTableTest.java b/test/unit/org/apache/cassandra/cql3/TombstonesWithIndexedSSTableTest.java
new file mode 100644
index 0000000..3042acd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/TombstonesWithIndexedSSTableTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.cql3;
+
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.io.sstable.IndexHelper;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TombstonesWithIndexedSSTableTest extends CQLTester
+{
+    @Test
+    public void testTombstoneBoundariesInIndexCached() throws Throwable
+    {
+        testTombstoneBoundariesInIndex("ALL");
+    }
+
+    @Test
+    public void testTombstoneBoundariesInIndexNotCached() throws Throwable
+    {
+        testTombstoneBoundariesInIndex("NONE");
+    }
+
+    public void testTombstoneBoundariesInIndex(String cacheKeys) throws Throwable
+    {
+        // That test reproduces the bug from CASSANDRA-11158 where a range tombstone boundary in the column index would
+        // cause an assertion failure.
+
+        int ROWS = 1000;
+        int VALUE_LENGTH = 100;
+
+        createTable("CREATE TABLE %s (k int, t int, s text static, v text, PRIMARY KEY (k, t)) WITH caching = { 'keys' : '" + cacheKeys + "' }");
+
+        // We create a partition that is big enough that the underlying sstable will be indexed
+        // For that, we use a large-ish number of row, and a value that isn't too small.
+        String text = makeRandomString(VALUE_LENGTH);
+        for (int i = 0; i < ROWS; i++)
+            execute("INSERT INTO %s(k, t, v) VALUES (?, ?, ?)", 0, i, text + i);
+
+        DecoratedKey dk = Util.dk(ByteBufferUtil.bytes(0));
+        int minDeleted = ROWS;
+        int maxDeleted = 0;
+
+        // Place some range deletions around an indexed location to get a tombstone boundary as the index's firstName.
+        // Because we insert a tombstone before it, the index position may move, so repeat procedure until the index
+        // boundary hits a tombstone boundary.
+        deletionLoop:
+        while (true)
+        {
+            flush();
+            compact();
+
+            int indexedRow = -1;
+            for (SSTableReader sstable : getCurrentColumnFamilyStore().getLiveSSTables())
+            {
+                // The line below failed with key caching off (CASSANDRA-11158)
+                @SuppressWarnings("unchecked")
+                RowIndexEntry<IndexHelper.IndexInfo> indexEntry = sstable.getPosition(dk, SSTableReader.Operator.EQ);
+                if (indexEntry != null && indexEntry.isIndexed())
+                {
+                    ClusteringPrefix firstName = indexEntry.columnsIndex().get(1).firstName;
+                    if (firstName.kind().isBoundary())
+                        break deletionLoop;
+                    indexedRow = Int32Type.instance.compose(firstName.get(0));
+                }
+            }
+            assert indexedRow >= 0;
+            minDeleted = Math.min(minDeleted, indexedRow - 2);
+            maxDeleted = Math.max(maxDeleted, indexedRow + 5);
+
+            execute("DELETE FROM %s WHERE k = 0 AND t >= ? AND t < ?", indexedRow - 2, indexedRow + 3);
+            execute("DELETE FROM %s WHERE k = 0 AND t >= ? AND t < ?", indexedRow, indexedRow + 5);
+        }
+
+        flush();
+        // The line below failed with key caching on (CASSANDRA-11158)
+        compact();
+
+        assertRowCount(execute("SELECT s FROM %s WHERE k = ?", 0), ROWS - (maxDeleted - minDeleted));
+        assertRowCount(execute("SELECT s FROM %s WHERE k = ? ORDER BY t DESC", 0), ROWS - (maxDeleted - minDeleted));
+
+        assertRowCount(execute("SELECT DISTINCT s FROM %s WHERE k = ?", 0), 1);
+        assertRowCount(execute("SELECT DISTINCT s FROM %s WHERE k = ? ORDER BY t DESC", 0), 1);
+    }
+
+    // Creates a random string
+    public static String makeRandomString(int length)
+    {
+        Random random = new Random();
+        char[] chars = new char[length];
+        for (int i = 0; i < length; ++i)
+            chars[i++] = (char) ('a' + random.nextInt('z' - 'a' + 1));
+        return new String(chars);
+    }
+}