[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-06-15 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-11882:
-

Updated the branches above with the latest version. Tests are now passing (to 
the extent that their respective base version are) and I'm happy with the code.

Marking ready to commit.

> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-06-01 Thread Lerh Chuan Low (JIRA)

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

Lerh Chuan Low commented on CASSANDRA-11882:


Branimir & Sylvian,

Thanks for the reviews so far. I was under the impression that {{Clustering}} 
itself is already a clustering column, good catch. I've retraced through the 
code path and I've updated the patch so that it only refuses larger than 64k 
values for clustering columns (i.e an INSERT with ckey1 = 32k and ckey2 = 32k 
will work, and memtable flushing still works in that case too :)). 

I've created the other issue here: 
https://issues.apache.org/jira/browse/CASSANDRA-11943. I don't think I have 
permissions to assign it to Sylvain, feel free to let me know if there is 
anything unclear with the setup etc. 





> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-06-01 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-11882:
--

bq. From the comments above I understand the fix will need a change in 
StatsMetadata format, which could require major version change or would 
otherwise be involved enough to warrant a separate ticket.

Hum, somehow missed that comment, sorry. I guess that make sense, but this only 
limit each clustering column value to be <= 64k, while the patch currently 
limit the total size of all clustering values to be <= 64k. I guess if we don't 
forget to create a ticket to fix the sstable metadata (which you can indeed 
feel free to assign to me), then I guess I'm fine refusing larger than 64k 
values for clustering columns for now, as long as we don't limit the 
{{Clustering}} as a whole.

> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-06-01 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-11882:
-

>From [the comments 
>above|https://issues.apache.org/jira/browse/CASSANDRA-11882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15306355#comment-15306355]
> I understand the fix will need a change in {{StatsMetadata}} format, which 
>could require major version change or would otherwise be involved enough to 
>warrant a separate ticket.

[~Lerh Low], could you create a new issue explaining what goes wrong with 64k+ 
keys in 3.0+ and assign it to Sylvain?

I restarted the tests as something went wrong with half of them yesterday.

> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-06-01 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-11882:
--

I'd love more details on why it doesn't work on 3.0, and in particular what 
"not always" means. But if it doesn't, it's a bug (by opposition to 2.1/2.2 
where it's a genuine limitation of the underlying format) and I'd rather we 
start by checking if it's not an easy fix before rejecting queries, because I 
suspect we'll just end up forgetting about it and let it exist longer that it 
has to be. I'm happy to create a separate ticket for that, to be assigned on 
that ticket and to look at it ASAP. But I'm not happy shoving it under the rug 
without at least having looked at it.

> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-31 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-11882:
-

Sylvain, the 64k+ keys currently _do not_ work in 3.x. They are not always 
serialized properly. We should find a way to fix the problem and make sure they 
are properly supported, but not as part of this ticket.

While they are not working, it's best to validate and error out early rather 
than accept the writes and let the node fail badly on flush and commit log 
recovery, thus I _would_ include the validation in the 3.x patch.

> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-31 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-11882:
--

We don't want to add such limitation in 3.0. The fact that don't have a 64K 
limit for the clustering columns value is a feature in 3.0, not a bug, and we 
don't want to artificially add it back. The only thing that 3.0 might need is 
the change to {{OutboundTcpConnection}}, but as far as I can tell, that's about 
it.

> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-31 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-11882:
-

Uploaded for testing:
|[2.1 
branch|https://github.com/blambov/cassandra/tree/11882-2.1]|[utest|http://cassci.datastax.com/job/blambov-11882-2.1-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11882-2.1-dtest/]|
|[2.2 
branch|https://github.com/blambov/cassandra/tree/11882-2.2]|[utest|http://cassci.datastax.com/job/blambov-11882-2.2-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11882-2.2-dtest/]|
|[3.0 
branch|https://github.com/blambov/cassandra/tree/11882-3.0]|[utest|http://cassci.datastax.com/job/blambov-11882-3.0-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11882-3.0-dtest/]|
|[trunk|https://github.com/blambov/cassandra/tree/11882]|[utest|http://cassci.datastax.com/job/blambov-11882-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11882-dtest/]|


> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-31 Thread Lerh Chuan Low (JIRA)

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

Lerh Chuan Low commented on CASSANDRA-11882:


I've just included the patch with the input sanitization for now for 3.X (I 
based it off trunk). 

> 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, 11882-3.X.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-31 Thread Lerh Chuan Low (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-11882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-30 Thread Lerh Chuan Low (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-11882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-27 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-11882:
--

Sorry for being slightly annoying, but catching {{Throwable}} in 
{{writeConnected}} is imo right, but it means we can catch stuffs like 
{{OutOfMemoryException}} for which we kind of want to let the JVM die. So 
basically, we should call {{JVMStabilityInspector.inspectThrowable()}} at the 
beginning of the {{catch}} clause. We probably always should have done that in 
fact really, so 3.0 should probably get that part (and yes, the storage engine 
revamp is why you can know have larger than 64k values in clustering). 

> 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
>
> Attachments: 11882-2.1.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-27 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-11882:
-

Code LGTM. Uploaded for testing:
|[2.1 
branch|https://github.com/blambov/cassandra/tree/11882-2.1]|[utest|http://cassci.datastax.com/job/blambov-11882-2.1-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11882-2.1-dtest/]|

Could you make a patch for 2.2 as well ({{SelectStatement.java}} has some 
incompatible changes)? And possibly a test that makes sure this is fine in 3.0+?

> 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
>
> Attachments: 11882-2.1.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-27 Thread Lerh Chuan Low (JIRA)

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

Lerh Chuan Low commented on CASSANDRA-11882:


That makes sense. There is indeed a place in {{writeConnected}} catching 
Exception. I've updated the patch to make that  catch Throwable since an 
AssertionError extends Error, and the asserts will just have a nicer message. 



> 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
>
> Attachments: 11882-2.1.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-27 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-11882:
--

bq. but I guess the idea was that if it did, at least the node will not see the 
other node as down forever after that until restart

But my point is, are you sure that is the case? I mean, is there some place 
where we catch {{IllegalArgumentException}} and not {{AssertionError}} so that 
which exception is thrown makes a difference? If there is, then maybe the right 
fix is to properly catch all exceptions at that particular place.

> 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
>
> Attachments: 11882-2.1.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-26 Thread Lerh Chuan Low (JIRA)

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

Lerh Chuan Low commented on CASSANDRA-11882:


Hey Branimir and Sylvain, thanks for reviewing. I've updated the patch and 
re-ran the tests. 

{{overloadBuffer}}: I re-read the Code style documentation, I've made changes 
to the braces and moved the comment away so the line isn't overly long, let me 
know if there are any more format changes required.

{{assertInvalidThrow}}: Didn't realise this method existed, updated the tests. 

{{ByteBufferUtil}}: I've updated it to throw an IllegalArgumentException. I 
wasn't certain if it could be reached via another way (I'm not familiar with 
the code base, naively I assume no) but I guess the idea was that if it did, at 
least the node will not see the other node as down forever after that until 
restart. Let me know if you feel strongly to leave it as assert. 

I also did try this on C* 3.5, it doesn't suffer from the same problem as 2.1, 
which I guess is due to the big revamp of the storage engine :)

> 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
>
> Attachments: 11882-2.1.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-26 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-11882:
--

I think the proper fix is the added call to {{validateComposite}}. With that, 
we shouldn't ever reach the assertion in {{ByteBufferUtil}}, and so I'd 
personally just left it as an assertion. 

> 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
>
> Attachments: 11882-2.1.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-26 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-11882:
-

The code looks good overall (with small style issue in {{overloadBuffer}})), 
but I'd prefer to throw an {{IllegalArgumentException}} instead of 
{{IOException}} and would use {{assertInvalidThrow}} in the test to make sure 
the right class of exception is thrown.

> 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
>
> Attachments: 11882-2.1.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)


[jira] [Commented] (CASSANDRA-11882) Clustering Key with ByteBuffer size > 64k throws Assertion Error

2016-05-23 Thread Lerh Chuan Low (JIRA)

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

Lerh Chuan Low commented on CASSANDRA-11882:


I can't find any ways of assigning it to myself, is it a permissions issue?

I've basically made a really simple patch - it just throws an IOException 
instead of an AssertionError (but it looks like there are many asserts 
scattered around the code base, any reason why?) with a more helpful message so 
that the coordinator doesn't see the other nodes as down forever. I've also 
written a couple of tests and attempted to use the existing logic in 
QueryProcessor (That looks like the right place to put it) to sanitize the 
SELECT to begin with. Feel free to leave behind any comments! 

> 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
>Priority: Minor
> Fix For: 2.1.x
>
> Attachments: 11882-2.1.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)