[ 
https://issues.apache.org/jira/browse/CASSANDRA-8062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14173432#comment-14173432
 ] 

Sylvain Lebresne commented on CASSANDRA-8062:
---------------------------------------------

+1

> IllegalArgumentException passing blob as tuple value element in list
> --------------------------------------------------------------------
>
>                 Key: CASSANDRA-8062
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8062
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Windows 7, DataStax 2.1.0 Cassandra server, Java 
> cassandra-driver-2.1.1 
>            Reporter: Bill Mitchell
>            Assignee: Tyler Hobbs
>             Fix For: 2.1.1
>
>         Attachments: 8062.txt
>
>
> I am using the same table schema as described in earlier reports, e.g., 
> CASSANDRA-7105:
> {code}
> CREATE TABLE sr (siteid uuid, listid bigint, partition int, createdate 
> timestamp, emailcrypt blob, emailaddr text, properties text, removedate 
> timestamp. removeimportid bigint,
>     PRIMARY KEY ((siteid, listid, partition), createdate, emailcrypt)
> ) WITH CLUSTERING ORDER BY (createdate DESC, emailcrypt DESC);
> {code}
> I am trying to take advantage of the new Tuple support to issue a query to 
> request multiple rows in a single wide row by (createdate,emailcrypt) pair.  
> I declare a new TupleType that covers the clustering columns and then issue 
> an IN predicate against a list of these values:
> {code}
>         private static final TupleType dateEmailTupleType = 
> TupleType.of(DataType.timestamp(), DataType.blob());
> ...
>         List<TupleValue> partitionKeys = new ArrayList<>(recipKeys.size());
> ...
>         BoundStatement boundStatement = new BoundStatement(preparedStatement);
>         boundStatement = boundStatement.bind(siteID, partition, listID);
>         boundStatement.setList(3, partitionKeys);
> {code}
> When I issue a SELECT against this table, the server fails apparently trying 
> to break apart the list values:
> {code}
> DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,312 Message.java:420 - 
> Received: PREPARE SELECT emailCrypt, emailAddr, removeDate, removeImportID, 
> properties FROM sr WHERE siteID = ? AND partition = ? AND listID = ? AND ( 
> createDate, emailCrypt ) IN ? ;, v=2
> DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,323 Tracing.java:157 - 
> request complete
> DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,323 Message.java:433 - 
> Responding: RESULT PREPARED a18ff9151e8bd3b13b48a0ba56ecb784 
> [siteid(testdb_1412536748414, sr), 
> org.apache.cassandra.db.marshal.UUIDType][partition(testdb_1412536748414, 
> sr), org.apache.cassandra.db.marshal.Int32Type][listid(testdb_1412536748414, 
> sr), 
> org.apache.cassandra.db.marshal.LongType][in(createdate,emailcrypt)(testdb_1412536748414,
>  sr), 
> org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.TupleType(org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType),org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.BytesType)))]
>  (resultMetadata=[emailcrypt(testdb_1412536748414, sr), 
> org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.BytesType)][emailaddr(testdb_1412536748414,
>  sr), 
> org.apache.cassandra.db.marshal.UTF8Type][removedate(testdb_1412536748414, 
> sr), 
> org.apache.cassandra.db.marshal.TimestampType][removeimportid(testdb_1412536748414,
>  sr), 
> org.apache.cassandra.db.marshal.LongType][properties(testdb_1412536748414, 
> sr), org.apache.cassandra.db.marshal.UTF8Type]), v=2
> DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,363 Message.java:420 - 
> Received: EXECUTE a18ff9151e8bd3b13b48a0ba56ecb784 with 4 values at 
> consistency QUORUM, v=2
> DEBUG [SharedPool-Worker-2] 2014-10-05 14:20:15,380 Message.java:420 - 
> Received: EXECUTE a18ff9151e8bd3b13b48a0ba56ecb784 with 4 values at 
> consistency QUORUM, v=2
> DEBUG [SharedPool-Worker-5] 2014-10-05 14:20:15,402 Message.java:420 - 
> Received: EXECUTE a18ff9151e8bd3b13b48a0ba56ecb784 with 4 values at 
> consistency QUORUM, v=2
> ERROR [SharedPool-Worker-5] 2014-10-05 14:20:16,125 ErrorMessage.java:218 - 
> Unexpected exception during request
> java.lang.IllegalArgumentException: null
>       at java.nio.Buffer.limit(Unknown Source) ~[na:1.7.0_25]
>       at 
> org.apache.cassandra.utils.ByteBufferUtil.readBytes(ByteBufferUtil.java:539) 
> ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.serializers.CollectionSerializer.readValue(CollectionSerializer.java:122)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.serializers.ListSerializer.deserializeForNativeProtocol(ListSerializer.java:87)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.serializers.ListSerializer.deserializeForNativeProtocol(ListSerializer.java:27)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.serializers.CollectionSerializer.deserialize(CollectionSerializer.java:48)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.db.marshal.AbstractType.compose(AbstractType.java:66) 
> ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.Tuples$InValue.fromSerialized(Tuples.java:249) 
> ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at org.apache.cassandra.cql3.Tuples$InMarker.bind(Tuples.java:394) 
> ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.MultiColumnRestriction$InWithMarker.splitValues(MultiColumnRestriction.java:103)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.buildMultiColumnInBound(SelectStatement.java:951)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.buildBound(SelectStatement.java:806)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.getRequestedBound(SelectStatement.java:1012)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.makeFilter(SelectStatement.java:448)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.getSliceCommands(SelectStatement.java:336)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.getPageableCommand(SelectStatement.java:237)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:200)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:60)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:187)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:413)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:133)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:422)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:318)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:103)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:332)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>       at 
> io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:31)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>       at 
> io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:323)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>       at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) 
> ~[na:1.7.0_25]
>       at 
> org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:163)
>  ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) 
> ~[apache-cassandra-2.1.0.jar:2.1.0]
>       at java.lang.Thread.run(Unknown Source) ~[na:1.7.0_25]
> ERROR [SharedPool-Worker-1] 2014-10-05 14:20:16,125 ErrorMessage.java:218 - 
> Unexpected exception during request
> java.lang.IllegalArgumentException: null
> ...
> ERROR [SharedPool-Worker-2] 2014-10-05 14:20:16,125 ErrorMessage.java:218 - 
> Unexpected exception during request
> java.lang.IllegalArgumentException: null
> ...
> DEBUG [SharedPool-Worker-5] 2014-10-05 14:20:16,126 Tracing.java:157 - 
> request complete
> DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:16,126 Tracing.java:157 - 
> request complete
> DEBUG [SharedPool-Worker-2] 2014-10-05 14:20:16,126 Tracing.java:157 - 
> request complete
> DEBUG [SharedPool-Worker-5] 2014-10-05 14:20:16,126 Message.java:433 - 
> Responding: ERROR SERVER_ERROR: java.lang.IllegalArgumentException, v=2
> DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:16,126 Message.java:433 - 
> Responding: ERROR SERVER_ERROR: java.lang.IllegalArgumentException, v=2
> DEBUG [SharedPool-Worker-2] 2014-10-05 14:20:16,127 Message.java:433 - 
> Responding: ERROR SERVER_ERROR: java.lang.IllegalArgumentException, v=2
> {code}
> (There are three exceptions as the client code issues three concurrent 
> asynchronous requests.)
> There is a fair chance this is a Java driver bug serializing the data into 
> the message, but I'm reporting this against the server as that is where the 
> exception occurs.  If I had a simple DAO layer above the driver, I would 
> attach a simple testcase, but I don't.  



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to