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

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

Sylvain, it's not annoying, I agree with you on that. I've changed it.

Branimir, I've attached the 2 patches again (updated). I've put in the check 
for the JVM and moved the Insert tests in the 2.1 patch to a new {{InsertTest}} 
class because they really aren't Selects. I've also removed the 
{{overloadBuffer}} method and followed the idea of using just a static 
{{TOO_BIG}} variable that were present in 2.2 and 3, that way we don't need an 
extra method.

I did retry this on 3 because the tests I ported over doesn't work on Inserts 
into CK with values larger than 64k. Using trunk, if I attempt (cqlsh) to run 
an insert query that with a CK that is larger than 64k, it actually works. I 
can even select it afterwards and it returns the inserted record well, so I 
mistakenly thought it worked fine on 3. But if I restart Cassandra, it will 
never succeed in starting up:

{code}
Caused by: java.lang.AssertionError: 131082
        at 
org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:308)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.serialize(StatsMetadata.java:286)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.serialize(StatsMetadata.java:235)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.metadata.MetadataSerializer.serialize(MetadataSerializer.java:75)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter.writeMetadata(BigTableWriter.java:378)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter.access$300(BigTableWriter.java:51)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.format.big.BigTableWriter$TransactionalProxy.doPrepare(BigTableWriter.java:342)
 ~[main/:na]
        at 
org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.format.SSTableWriter.prepareToCommit(SSTableWriter.java:280)
 ~[main/:na]
        at 
org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.prepareToCommit(SimpleSSTableMultiWriter.java:101)
 ~[main/:na]
        at 
org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1145)
 ~[main/:na]
        at 
org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1095)
 ~[main/:na]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) 
~[na:1.8.0_40]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
~[na:1.8.0_40]
        at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_40]
{code}

It can also be reproduced by doing the INSERT with a CK having a value larger 
than 64k, and then calling {{nodetool flush}}. It looks like because writing 
{{StatsMetadata}} while flushing Memtables it still calls 
{{writeWithShortLength}}. I'll test with changing it to {{writeWithLength}}. 

> 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