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

Lerh Chuan Low commented on CASSANDRA-11882:
--------------------------------------------

That is not a good idea - even if I could get it to work in this version by 
serializing and deserializing with more bytes than before, it will break 
backwards compatibility - old SSTables that have been serialized with 
{{writeWithShortLength}} will not be able to be deserialized.

Given that it may just be worthwhile to sanitize the input as before to prevent 
users from inserting a Clustering Key larger than 64k, also because even if the 
underlying storage structure allowed for it, it's still a pretty big size to be 
using for Clustering key. I'm also thinking that if someone did insert a CK at 
the moment larger than 64k, it will keep working until the Memtable is flushed 
(either by it getting full or C* restarting and the CommitLogs replaying) at 
which point AssertionErrors are thrown. If C* restarts this way, it will never 
succeed in starting up (I had to delete the commitlogs folder). Any thoughts? 

> Clustering Key with ByteBuffer size > 64k throws Assertion Error
> ----------------------------------------------------------------
>
>                 Key: CASSANDRA-11882
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11882
>             Project: Cassandra
>          Issue Type: Bug
>          Components: CQL, Streaming and Messaging
>            Reporter: Lerh Chuan Low
>             Fix For: 2.1.x, 2.2.x
>
>         Attachments: 11882-2.1.txt, 11882-2.2.txt
>
>
> Setup:
> {code}
> CREATE KEYSPACE Blues WITH REPLICATION = { 'class' : 'SimpleStrategy', 
> 'replication_factor' : 2};
> CREATE TABLE test (a text, b text, PRIMARY KEY ((a), b))
> {code}
> There currently doesn't seem to be an existing check for selecting clustering 
> keys that are larger than 64k. So if we proceed to do the following select:
> {code}
> CONSISTENCY ALL;
> SELECT * FROM Blues.test WHERE a = 'foo' AND b = 'something larger than 64k';
> {code}
> An AssertionError is thrown in `ByteBufferUtil` with just a number and an 
> error message detailing 'Coordinator node timed out waiting for replica nodes 
> responses' . Additionally, because an error extends Throwable (it's not a 
> subclass of Exception), it's not caught so the connection between the 
> coordinator node and the other nodes which have the replicas seem to be 
> 'stuck' until it's restarted. Any other subsequent queries, even if it's just 
> SELECT where a = 'foo' and b = 'bar', will always return the Coordinator 
> timing out waiting for replica nodes responses'.



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

Reply via email to