Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Fixed
-
None
-
None
-
Normal
Description
If you use static columns, as implemented in CASSANDRA-6561, then very simple SELECT...WHERE...IN queries fail with an internal NPE.
create table foo (x text, y text, s text static, primary key (x,y));
insert into foo (x,y,s) values ('a','b','c');
select * from foo where x='a' and y in ('b','c');
Request did not complete within rpc_timeout.
ERROR [ReadStage:190] 2014-02-25 14:19:16,400 CassandraDaemon.java (line 196) Exception in thread Thread[ReadStage:190,5,main]
java.lang.RuntimeException: java.lang.NullPointerException
at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1900)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:722)
Caused by: java.lang.NullPointerException
at org.apache.cassandra.db.filter.ColumnSlice$NavigableMapIterator.computeNext(ColumnSlice.java:141)
at org.apache.cassandra.db.filter.ColumnSlice$NavigableMapIterator.computeNext(ColumnSlice.java:162)
at org.apache.cassandra.db.filter.ColumnSlice$NavigableMapIterator.computeNext(ColumnSlice.java:162)
at org.apache.cassandra.db.filter.ColumnSlice$NavigableMapIterator.computeNext(ColumnSlice.java:117)
at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
at org.apache.cassandra.db.filter.SliceQueryFilter$1.hasNext(SliceQueryFilter.java:148)
at org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:157)
at org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:140)
at org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:200)
at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
at org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:185)
at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:122)
at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:80)
at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:72)
at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:297)
at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53)
at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1550)
at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1379)
at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327)
at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65)
at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1341)
at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1896)