Details
-
Improvement
-
Status: In Progress
-
Low
-
Resolution: Unresolved
-
None
-
C* 3.0.0 | cqlsh | C# driver 3.0.0beta2 | Windows 2012 R2
Description
This is an issue I first noticed through the C# driver, but I was able to repro on cqlsh, leading me to believe this is a Cassandra bug.
Given the following schema:
CREATE TABLE "TestKeySpace_4928dc892922"."coolMovies" ( unique_movie_title text, movie_maker text, director text, list list<text>, "mainGuy" text, "yearMade" int, PRIMARY KEY ((unique_movie_title, movie_maker), director) ) WITH CLUSTERING ORDER BY (director ASC)
Executing a SELECT with FILTERING on a non-PK column, using a NULL as the argument:
SELECT "mainGuy", "movie_maker", "unique_movie_title", "list", "director", "yearMade" FROM "coolMovies" WHERE "mainGuy" = null ALLOW FILTERING
returns a ReadFailure exception:
cqlsh:TestKeySpace_4c8f2cf8d5cc> SELECT "mainGuy", "movie_maker", "unique_movie_title", "list", "director", "yearMade" FROM "coolMovies" WHERE "mainGuy" = null ALLOW FILTERING; ←[0;1;31mTraceback (most recent call last): File "C:\Users\Kishan\.ccm\repository\3.0.0\bin\\cqlsh.py", line 1216, in perform_simple_statement result = future.result() File "C:\Users\Kishan\.ccm\repository\3.0.0\bin\..\lib\cassandra-driver-internal-only-3.0.0a3.post0-3f15725.zip\cassandra-driver-3.0.0a3.post0-3f15725\cassandra\cluster.py", line 3118, in result raise self._final_exception ReadFailure: code=1300 [Replica(s) failed to execute read] message="Operation failed - received 0 responses and 1 failures" info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 'cons istency': 'ONE'} ←[0m
Cassandra log shows:
WARN [SharedPool-Worker-2] 2015-11-16 13:51:00,259 AbstractTracingAwareExecutorService.java:169 - Uncaught exception on thread Thread[SharedPool-Worker-2,10,main]: {} java.lang.AssertionError: null at org.apache.cassandra.db.filter.RowFilter$SimpleExpression.isSatisfiedBy(RowFilter.java:581) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.filter.RowFilter$CQLFilter$1IsSatisfiedFilter.applyToRow(RowFilter.java:243) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.transform.BaseRows.applyOne(BaseRows.java:95) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.transform.BaseRows.add(BaseRows.java:86) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.transform.UnfilteredRows.add(UnfilteredRows.java:21) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.transform.Transformation.add(Transformation.java:136) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.transform.Transformation.apply(Transformation.java:102) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.filter.RowFilter$CQLFilter$1IsSatisfiedFilter.applyToPartition(RowFilter.java:233) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.filter.RowFilter$CQLFilter$1IsSatisfiedFilter.applyToPartition(RowFilter.java:227) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:76) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:293) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:136) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:128) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:123) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:288) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1692) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2346) ~[apache-cassandra-3.0.0.jar:3.0.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_60] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) ~[apache-cassandra-3.0.0.jar:3.0.0] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [apache-cassandra-3.0.0.jar:3.0.0] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60]
In C* < 3.0.0 (such as 2.2.3), this same query correctly returns:
cqlsh:TestKeySpace_3231cd551e49> SELECT "mainGuy", "movie_maker", "unique_movie_title", "list", "director", "yearMade" FROM "coolMovies" WHERE "mainGuy" = null ALLOW FILTERING; ←[0;1;31mInvalidRequest: code=2200 [Invalid query] message="Unsupported null value for indexed column mainGuy"←[0m
Not sure if related, but using a value for the argument instead of null returns 0 rows in 3.0.0, but correctly returns an InvalidRequest exception in C* 2.2.3:
SELECT "mainGuy", "movie_maker", "unique_movie_title", "list", "director", "yearMade" FROM "coolMovies" WHERE "yearMade" = 100 ALLOW FILTERING
In C* 2.2.3:
cqlsh:TestKeySpace_4928dc892922> SELECT "mainGuy", "movie_maker", "unique_movie_title", "list", "director", "yearMade" FROM "coolMovies" WHERE "yearMade" = 100 ALLOW FILTERING; ←[0;1;31mInvalidRequest: code=2200 [Invalid query] message="No secondary indexes on the restricted columns support the provided operators: "←[0m
Attachments
Attachments
Issue Links
- depends upon
-
CASSANDRA-6377 ALLOW FILTERING should allow seq scan filtering
- Resolved
- is related to
-
CASSANDRA-17762 LWT IF col = NULL is inconsistent with SQL NULL
- Open