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 10:50:49 UTC

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

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/cassandra-3.0
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();
     }
 }