You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Marten Kenbeek (Jira)" <ji...@apache.org> on 2022/02/24 13:27:00 UTC

[jira] [Created] (CASSANDRA-17403) UnsupportedOperationException when serializing EmptyCellPath for SliceRange query

Marten Kenbeek created CASSANDRA-17403:
------------------------------------------

             Summary: UnsupportedOperationException when serializing EmptyCellPath for SliceRange query
                 Key: CASSANDRA-17403
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-17403
             Project: Cassandra
          Issue Type: Bug
            Reporter: Marten Kenbeek


When querying a supercolumn table for a slice of sub columns through Thrift, an empty bound can be used as the start and/or finish in the SliceRange to create a (half) unbounded slice. These bounds are translated to CellPath.BOTTOM/TOP when [making the column filter in CassandraServer|https://github.com/apache/cassandra/blob/c3d51a825cd16ca627b24c4190017cf4b4f320c2/src/java/org/apache/cassandra/thrift/CassandraServer.java#L437].
{code:java}
ByteBuffer key = UTF_8.encode("key");
ColumnParent parent = new ColumnParent(tableName).setSuper_column(UTF_8.encode("Default"));
SliceRange range = new SliceRange(UTF_8.encode(""), UTF_8.encode(""), false, 100);
SlicePredicate predicate = new SlicePredicate().setSlice_range(range);

List<ColumnOrSuperColumn> columns = cassandra.get_slice(key, parent, predicate, ConsistencyLevel.QUORUM);{code}
This query works fine on Cassandra 2.2. On 3.11 it works with consistency level ONE, but fails when run on a cluster at a higher consistency level. When attempting to send the (digest) requests to the other nodes, it cannot serialize the EmptyCellPath and fails with the following stacktrace:
{code:java}
ERROR [Thrift:4112] 2022-02-24 12:31:10,989 CustomTThreadPoolServer.java:225 - Error occurred during processing of message.
java.lang.UnsupportedOperationException: null
	at org.apache.cassandra.db.rows.CellPath$EmptyCellPath.get(CellPath.java:143) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.db.marshal.CollectionType$CollectionPathSerializer.serializedSize(CollectionType.java:253) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.db.filter.ColumnSubselection$Serializer.serializedSize(ColumnSubselection.java:228) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.db.filter.ColumnFilter$Serializer.serializedSize(ColumnFilter.java:566) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.db.ReadCommand$Serializer.serializedSize(ReadCommand.java:800) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.db.ReadCommand$Serializer.serializedSize(ReadCommand.java:704) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.io.ForwardingVersionedSerializer.serializedSize(ForwardingVersionedSerializer.java:55) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.net.MessageOut.payloadSize(MessageOut.java:193) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.net.OutboundTcpConnectionPool.getConnection(OutboundTcpConnectionPool.java:76) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:802) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.net.MessagingService.sendOneWay(MessagingService.java:953) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.net.MessagingService.sendRR(MessagingService.java:895) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.net.MessagingService.sendRRWithFailure(MessagingService.java:877) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.service.AbstractReadExecutor.makeRequests(AbstractReadExecutor.java:110) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.service.AbstractReadExecutor.makeDataRequests(AbstractReadExecutor.java:85) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.service.AbstractReadExecutor$NeverSpeculatingReadExecutor.executeAsync(AbstractReadExecutor.java:220) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.doInitialQueries(StorageProxy.java:1907) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1860) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1800) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1705) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.thrift.CassandraServer.read(CassandraServer.java:98) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServer.java:265) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.thrift.CassandraServer.multigetSliceInternal(CassandraServer.java:590) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.thrift.CassandraServer.getSliceInternal(CassandraServer.java:325) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.thrift.CassandraServer.get_slice(CassandraServer.java:302) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at org.apache.cassandra.thrift.Cassandra$Processor$get_slice.getResult(Cassandra.java:3659) ~[apache-cassandra-thrift-3.11.11.jar:3.11.11]
	at org.apache.cassandra.thrift.Cassandra$Processor$get_slice.getResult(Cassandra.java:3643) ~[apache-cassandra-thrift-3.11.11.jar:3.11.11]
	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[libthrift-0.9.2.jar:0.9.2]
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[libthrift-0.9.2.jar:0.9.2]
	at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:206) ~[apache-cassandra-3.11.11.jar:3.11.11]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [na:1.8.0_322]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [na:1.8.0_322]
	at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) [apache-cassandra-3.11.11.jar:3.11.11]
	at java.lang.Thread.run(Thread.java:750) ~[na:1.8.0_322] {code}
Table definition:
{code:java}
CfDef cfd = new CfDef(keyspaceName, tableName)
        .setColumn_type(ColumnType.SUPER.getValue())
        .setKey_validation_class("UTF8Type")
        .setComparator_type("UTF8Type")
        .setSubcomparator_type("UTF8Type")
        .setDefault_validation_class("UTF8Type");
 {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org