[jira] [Comment Edited] (CASSANDRA-12454) Unable to start on IPv6-only node with local JMX

2016-08-13 Thread Vadim TSes'ko (JIRA)

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

Vadim TSes'ko edited comment on CASSANDRA-12454 at 8/13/16 3:54 PM:


The patch:
{code:java}
diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java 
b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
index ad87efd..e420600 100644
--- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java
+++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
+import java.net.Inet4Address;
+import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.rmi.NoSuchObjectException;
 import java.rmi.Remote;
@@ -76,7 +78,18 @@ public class JMXServerUtils
 // Configure the RMI client & server socket factories, including SSL 
config.
 env.putAll(configureJmxSocketFactories(serverAddress));

-String url = String.format(urlTemplate, (serverAddress != null ? 
serverAddress.getHostAddress() : "0.0.0.0"), port);
+final String hostAddress;
+if (serverAddress == null) {
+hostAddress = "0.0.0.0";
+} else if (serverAddress instanceof Inet4Address) {
+hostAddress = serverAddress.getHostAddress();
+} else if (serverAddress instanceof Inet6Address) {
+hostAddress = '[' + serverAddress.getHostAddress() + ']';
+} else {
+throw new UnsupportedOperationException("Unsupported InetAddress");
+}
+
+String url = String.format(urlTemplate, hostAddress, port);
 LocateRegistry.createRegistry(port,
  (RMIClientSocketFactory) 
env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE),
  (RMIServerSocketFactory) 
env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE));
{code}
And the result in {{LOCAL_JMX}} mode:
{code}
INFO  [main] 2016-08-13 18:51:36,511 JMXServerUtils.java:121 - Configured JMX 
server at: 
service:jmx:rmi://[0:0:0:0:0:0:0:1]/jndi/rmi://[0:0:0:0:0:0:0:1]:7199/jmxrmi
{code}


was (Author: incubos):
I can prepare a pull request, if it helps.

> Unable to start on IPv6-only node with local JMX
> 
>
> Key: CASSANDRA-12454
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12454
> Project: Cassandra
>  Issue Type: Bug
> Environment: Ubuntu Trusty, Oracle JDK 1.8.0_102-b14, IPv6-only host
>Reporter: Vadim TSes'ko
>
> A Cassandra node using *default* configuration is unable to start on 
> *IPv6-only* machine with the following error message:
> {code}
> ERROR [main] 2016-08-13 14:38:07,309 CassandraDaemon.java:731 - Bad URL path: 
> :0:0:0:0:0:1/jndi/rmi://0:0:0:0:0:0:0:1:7199/jmxrmi
> {code}
> The problem might be located in {{JMXServerUtils.createJMXServer()}} (I am 
> not sure, because there is no stack trace in {{system.log}}):
> {code:java}
> String urlTemplate = "service:jmx:rmi://%1$s/jndi/rmi://%1$s:%2$d/jmxrmi";
> ...
> String url = String.format(urlTemplate, (serverAddress != null ? 
> serverAddress.getHostAddress() : "0.0.0.0"), port);
> {code}
> IPv6 addresses must be surrounded by square brackets when passed to 
> {{JMXServiceURL}}.
> Disabling {{LOCAL_JMX}} mode in {{cassandra-env.sh}} (and enabling JMX 
> authentication) helps.



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


[jira] [Created] (CASSANDRA-12454) Unable to start on IPv6-only node with local JMX

2016-08-13 Thread Vadim TSes'ko (JIRA)
Vadim TSes'ko created CASSANDRA-12454:
-

 Summary: Unable to start on IPv6-only node with local JMX
 Key: CASSANDRA-12454
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12454
 Project: Cassandra
  Issue Type: Bug
 Environment: Ubuntu Trusty, Oracle JDK 1.8.0_102-b14, IPv6-only host
Reporter: Vadim TSes'ko


A Cassandra node using *default* configuration is unable to start on 
*IPv6-only* machine with the following error message:
{code}
ERROR [main] 2016-08-13 14:38:07,309 CassandraDaemon.java:731 - Bad URL path: 
:0:0:0:0:0:1/jndi/rmi://0:0:0:0:0:0:0:1:7199/jmxrmi
{code}
The problem might be located in {{JMXServerUtils.createJMXServer()}} (I am not 
sure, because there is no stack trace in {{system.log}}):
{code:java}
String urlTemplate = "service:jmx:rmi://%1$s/jndi/rmi://%1$s:%2$d/jmxrmi";
...
String url = String.format(urlTemplate, (serverAddress != null ? 
serverAddress.getHostAddress() : "0.0.0.0"), port);
{code}
IPv6 addresses must be surrounded by square brackets when passed to 
{{JMXServiceURL}}.
Disabling {{LOCAL_JMX}} mode in {{cassandra-env.sh}} (and enabling JMX 
authentication) helps.



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


[jira] [Commented] (CASSANDRA-12454) Unable to start on IPv6-only node with local JMX

2016-08-13 Thread Vadim TSes'ko (JIRA)

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

Vadim TSes'ko commented on CASSANDRA-12454:
---

I can prepare a pull request, if it helps.

> Unable to start on IPv6-only node with local JMX
> 
>
> Key: CASSANDRA-12454
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12454
> Project: Cassandra
>  Issue Type: Bug
> Environment: Ubuntu Trusty, Oracle JDK 1.8.0_102-b14, IPv6-only host
>Reporter: Vadim TSes'ko
>
> A Cassandra node using *default* configuration is unable to start on 
> *IPv6-only* machine with the following error message:
> {code}
> ERROR [main] 2016-08-13 14:38:07,309 CassandraDaemon.java:731 - Bad URL path: 
> :0:0:0:0:0:1/jndi/rmi://0:0:0:0:0:0:0:1:7199/jmxrmi
> {code}
> The problem might be located in {{JMXServerUtils.createJMXServer()}} (I am 
> not sure, because there is no stack trace in {{system.log}}):
> {code:java}
> String urlTemplate = "service:jmx:rmi://%1$s/jndi/rmi://%1$s:%2$d/jmxrmi";
> ...
> String url = String.format(urlTemplate, (serverAddress != null ? 
> serverAddress.getHostAddress() : "0.0.0.0"), port);
> {code}
> IPv6 addresses must be surrounded by square brackets when passed to 
> {{JMXServiceURL}}.
> Disabling {{LOCAL_JMX}} mode in {{cassandra-env.sh}} (and enabling JMX 
> authentication) helps.



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


[jira] [Comment Edited] (CASSANDRA-8216) Select Count with Limit returns wrong value

2016-04-11 Thread Vadim TSes'ko (JIRA)

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

Vadim TSes'ko edited comment on CASSANDRA-8216 at 4/11/16 5:39 PM:
---

I'm sorry. Then [the 
docs|http://docs.datastax.com/en/cql/3.3/cql/cql_reference/select_r.html] 
should be fixed, because they say:
{code}
Specifying rows returned using LIMIT 
Using the LIMIT option, you can specify that the query return a limited number 
of rows.

SELECT COUNT(*) FROM big_table LIMIT 5;
SELECT COUNT(*) FROM big_table LIMIT 20;
The output of these statements if you had 105,291 rows in the database would 
be: 5, and 105,291. The cqlsh shell has a default row limit of 10,000. The 
Cassandra server and native protocol do not limit the number of rows that can 
be returned, although a timeout stops running queries to protect against 
running malformed queries that would cause system instability.
{code}


was (Author: incubos):
I'm sorry.

> Select Count with Limit returns wrong value
> ---
>
> Key: CASSANDRA-8216
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8216
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Philip Thompson
>Assignee: Benjamin Lerer
>  Labels: qa-resolved
> Fix For: 2.2.0 beta 1
>
>
> The dtest cql_tests.py:TestCQL.select_count_paging_test is failing on trunk 
> HEAD but not 2.1-HEAD.
> The query {code} select count(*) from test where field3 = false limit 1; 
> {code} is returning 2, where obviously it should only return 1 because of the 
> limit. This may end up having the same root cause of #8214, I will be 
> bisecting them both soon.



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


[jira] [Commented] (CASSANDRA-8216) Select Count with Limit returns wrong value

2016-04-11 Thread Vadim TSes'ko (JIRA)

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

Vadim TSes'ko commented on CASSANDRA-8216:
--

I'm sorry.

> Select Count with Limit returns wrong value
> ---
>
> Key: CASSANDRA-8216
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8216
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Philip Thompson
>Assignee: Benjamin Lerer
>  Labels: qa-resolved
> Fix For: 2.2.0 beta 1
>
>
> The dtest cql_tests.py:TestCQL.select_count_paging_test is failing on trunk 
> HEAD but not 2.1-HEAD.
> The query {code} select count(*) from test where field3 = false limit 1; 
> {code} is returning 2, where obviously it should only return 1 because of the 
> limit. This may end up having the same root cause of #8214, I will be 
> bisecting them both soon.



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


[jira] [Commented] (CASSANDRA-10380) SELECT count within a partition does not respect LIMIT

2016-04-11 Thread Vadim TSes'ko (JIRA)

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

Vadim TSes'ko commented on CASSANDRA-10380:
---

I managed to reproduce the bug using Cassandra 2.2.5.
The table schema is:
{code:sql}
CREATE TABLE my_table (
u text,
t timeuuid,
PRIMARY KEY (u, t)
) WITH CLUSTERING ORDER BY (t DESC)
AND bloom_filter_fp_chance = 0.01
AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}'
AND comment = ''
AND compaction = {'min_threshold': '2', 'class': 
'org.apache.cassandra.db.compaction.DateTieredCompactionStrategy', 
'base_time_seconds': '1'}
AND compression = {'sstable_compression': 
'org.apache.cassandra.io.compress.LZ4Compressor'}
AND dclocal_read_repair_chance = 0.0
AND default_time_to_live = 10
AND gc_grace_seconds = 0
AND max_index_interval = 2048
AND memtable_flush_period_in_ms = 0
AND min_index_interval = 128
AND read_repair_chance = 0.0
AND speculative_retry = '99.0PERCENTILE';
{code}
A query with wrong result:
{code:sql}
> select count(*) from my_table where u = 'user-0' limit 1;

 count
---
 8

(1 rows)
{code}

> SELECT count within a partition does not respect LIMIT
> --
>
> Key: CASSANDRA-10380
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10380
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
>Reporter: Adam Holmberg
>Assignee: Benjamin Lerer
>Priority: Minor
> Attachments: 10380.txt
>
>
> {code}
> cassandra@cqlsh> create KEYSPACE test WITH replication = {'class': 
> 'SimpleStrategy', 'replication_factor': '1'};
> cassandra@cqlsh> use test;
> cassandra@cqlsh:test> create table t (k int, c int, v int, primary key (k, 
> c));
> cassandra@cqlsh:test> INSERT INTO t (k, c, v) VALUES (0, 0, 0);
> cassandra@cqlsh:test> INSERT INTO t (k, c, v) VALUES (0, 1, 0);
> cassandra@cqlsh:test> INSERT INTO t (k, c, v) VALUES (0, 2, 0);
> cassandra@cqlsh:test> select * from t where k = 0;
>  k | c | v
> ---+---+---
>  0 | 0 | 0
>  0 | 1 | 0
>  0 | 2 | 0
> (3 rows)
> cassandra@cqlsh:test> select count(*) from t where k = 0 limit 2;
>  count
> ---
>  3
> (1 rows)
> {code}
> Expected: count should return 2, according to limit.
> Actual: count of all rows in partition
> This manifests in 3.0, does not appear in 2.2



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


[jira] [Commented] (CASSANDRA-8216) Select Count with Limit returns wrong value

2016-04-11 Thread Vadim TSes'ko (JIRA)

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

Vadim TSes'ko commented on CASSANDRA-8216:
--

I managed to reproduce the bug using Cassandra 2.2.5.
The table schema is:
{code:sql}
CREATE TABLE my_table (
u text,
t timeuuid,
PRIMARY KEY (u, t)
) WITH CLUSTERING ORDER BY (t DESC)
AND bloom_filter_fp_chance = 0.01
AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}'
AND comment = ''
AND compaction = {'min_threshold': '2', 'class': 
'org.apache.cassandra.db.compaction.DateTieredCompactionStrategy', 
'base_time_seconds': '1'}
AND compression = {'sstable_compression': 
'org.apache.cassandra.io.compress.LZ4Compressor'}
AND dclocal_read_repair_chance = 0.0
AND default_time_to_live = 10
AND gc_grace_seconds = 0
AND max_index_interval = 2048
AND memtable_flush_period_in_ms = 0
AND min_index_interval = 128
AND read_repair_chance = 0.0
AND speculative_retry = '99.0PERCENTILE';
{code}
A query with wrong result:
{code:sql}
> select count(*) from my_table where u = 'user-0' limit 1;

 count
---
 8

(1 rows)
{code}

> Select Count with Limit returns wrong value
> ---
>
> Key: CASSANDRA-8216
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8216
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Philip Thompson
>Assignee: Benjamin Lerer
>  Labels: qa-resolved
> Fix For: 2.2.0 beta 1
>
>
> The dtest cql_tests.py:TestCQL.select_count_paging_test is failing on trunk 
> HEAD but not 2.1-HEAD.
> The query {code} select count(*) from test where field3 = false limit 1; 
> {code} is returning 2, where obviously it should only return 1 because of the 
> limit. This may end up having the same root cause of #8214, I will be 
> bisecting them both soon.



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


[jira] [Commented] (CASSANDRA-10988) ClassCastException in SelectStatement

2016-04-07 Thread Vadim TSes'ko (JIRA)

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

Vadim TSes'ko commented on CASSANDRA-10988:
---

We managed to reproduce the bug using Cassandra 2.2.5 with the following table 
schema (unset parameters use default values):
{code:sql}
CREATE TABLE mytable (
u text,
t timeuuid,
PRIMARY KEY (u, t)
) WITH COMPACT STORAGE
AND CLUSTERING ORDER BY (t DESC)
AND compaction = {'min_threshold': '2', 'class': 
'org.apache.cassandra.db.compaction.DateTieredCompactionStrategy', 
'base_time_seconds': '1'}
AND compression = {'sstable_compression': 
'org.apache.cassandra.io.compress.LZ4Compressor'}
AND dclocal_read_repair_chance = 0.0
AND default_time_to_live = 10
AND gc_grace_seconds = 0
AND read_repair_chance = 0.0;
{code}
And the following query:
{code:sql}
SELECT COUNT(1) as cnt
FROM mytable
WHERE u = :user AND t > :timestamp
ORDER BY t DESC
LIMIT 110;
{code}
Removing {{COMPACT STORAGE}} definitely helps, so it is somehow connected.

> ClassCastException in SelectStatement
> -
>
> Key: CASSANDRA-10988
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10988
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
>Reporter: Vassil Hristov
>
> After we've upgraded our cluster to version 2.1.11, we started getting the 
> bellow exceptions for some of our queries. Issue seems to be very similar to 
> CASSANDRA-7284.
> {code:java}
> java.lang.ClassCastException: 
> org.apache.cassandra.db.composites.Composites$EmptyComposite cannot be cast 
> to org.apache.cassandra.db.composites.CellName
> at 
> org.apache.cassandra.db.composites.AbstractCellNameType.cellFromByteBuffer(AbstractCellNameType.java:188)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.db.composites.AbstractSimpleCellNameType.makeCellName(AbstractSimpleCellNameType.java:125)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.db.composites.AbstractCellNameType.makeCellName(AbstractCellNameType.java:254)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.makeExclusiveSliceBound(SelectStatement.java:1197)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.applySliceRestriction(SelectStatement.java:1205)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.processColumnFamily(SelectStatement.java:1283)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:1250)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:299)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:276)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:224)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:67)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:238)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:493)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:138)
>  ~[apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:439)
>  [apache-cassandra-2.1.11.jar:2.1.11]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:335)
>  [apache-cassandra-2.1.11.jar:2.1.11]
> at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
> at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
> at 
> io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
> at 
> io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_66]
> at 
> 

[jira] [Comment Edited] (CASSANDRA-7868) Sporadic CL switch from LOCAL_QUORUM to ALL

2014-09-12 Thread Vadim TSes'ko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14128754#comment-14128754
 ] 

Vadim TSes'ko edited comment on CASSANDRA-7868 at 9/12/14 2:30 PM:
---

We suspected {{allowRemoteDcForLocalConsistencyLevel = true}} in 
{{DCAwareRoundRobinPolicy}}, but after turning it off the problem is still 
reproduced.


was (Author: incubos):
The cause is {{allowRemoteDcForLocalConsistencyLevel = true}} in 
{{DCAwareRoundRobinPolicy}}.

 Sporadic CL switch from LOCAL_QUORUM to ALL
 ---

 Key: CASSANDRA-7868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7868
 Project: Cassandra
  Issue Type: Bug
 Environment: Client: cassandra-java-driver 2.0.4
 Server: 2.0.9
Reporter: Dmitry Schitinin

 Hi!
 We have keyspace described as
 {code}
 CREATE KEYSPACE subscriptions WITH replication = {
   'class': 'NetworkTopologyStrategy',
   'FOL': '3',
   'SAS': '3',
   'AMS': '0',
   'IVA': '3',
   'UGR': '0'
 } AND durable_writes = 'false';
 {code}
 There is simple table 
 {code}
 CREATE TABLE processed_documents (
   id text,
   PRIMARY KEY ((id))
 ) WITH
   bloom_filter_fp_chance=0.01 AND
   caching='KEYS_ONLY' AND
   comment='' AND
   dclocal_read_repair_chance=0.00 AND
   gc_grace_seconds=864000 AND
   index_interval=128 AND
   read_repair_chance=0.10 AND
   replicate_on_write='true' AND
   populate_io_cache_on_flush='false' AND
   default_time_to_live=0 AND
   speculative_retry='99.0PERCENTILE' AND
   memtable_flush_period_in_ms=0 AND
   compaction={'class': 'SizeTieredCompactionStrategy'} AND
   compression={'sstable_compression': 'LZ4Compressor'};
 {code}
 in the keyspace.
 On client we execute next prepared statement:
 {code}
 session.prepare(
 SELECT id FROM processed_documents WHERE id IN :ids
 ).setConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM)
 {code}
 Used Cassandra session has next main properties:
   * Load balancing policy - DCAwareRoundRobinPolicy(localDc, 
 usedHostPerRemoteDc = 3, allowRemoteDcForLocalConsistencyLevel = true)
   * Retry policy - DefaultRetryPolicy
   * Query options - QueryOptions with set consistency level to 
 ConsistencyLevel.LOCAL_QUORUM
 Our problem is next.
 Since some moment there are next errors in the client application log:
 {code}
 com.datastax.driver.core.exceptions.ReadTimeoutException: Cassandra timeout 
 during read query at consistency ALL (9 responses were required but only 8 
 replica responded)
 at 
 com.datastax.driver.core.exceptions.ReadTimeoutException.copy(ReadTimeoutException.java:69)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.Responses$Error.asException(Responses.java:94) 
 ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.DefaultResultSetFuture.onSet(DefaultResultSetFuture.java:108)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.RequestHandler.setFinalResult(RequestHandler.java:235)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.RequestHandler.onSet(RequestHandler.java:379) 
 ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.Connection$Dispatcher.messageReceived(Connection.java:571)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) 
 ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:70)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) 
 ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:70)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
  ~[netty-3.9.0.Final.jar:na]
 at 
 

[jira] [Commented] (CASSANDRA-7868) Sporadic CL switch from LOCAL_QUORUM to ALL

2014-09-10 Thread Vadim TSes'ko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14128754#comment-14128754
 ] 

Vadim TSes'ko commented on CASSANDRA-7868:
--

The cause is {{allowRemoteDcForLocalConsistencyLevel = true}} in 
{{DCAwareRoundRobinPolicy}}.

 Sporadic CL switch from LOCAL_QUORUM to ALL
 ---

 Key: CASSANDRA-7868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7868
 Project: Cassandra
  Issue Type: Bug
 Environment: Client: cassandra-java-driver 2.0.4
 Server: 2.0.9
Reporter: Dmitry Schitinin

 Hi!
 We have keyspace described as
 {code}
 CREATE KEYSPACE subscriptions WITH replication = {
   'class': 'NetworkTopologyStrategy',
   'FOL': '3',
   'SAS': '3',
   'AMS': '0',
   'IVA': '3',
   'UGR': '0'
 } AND durable_writes = 'false';
 {code}
 There is simple table 
 {code}
 CREATE TABLE processed_documents (
   id text,
   PRIMARY KEY ((id))
 ) WITH
   bloom_filter_fp_chance=0.01 AND
   caching='KEYS_ONLY' AND
   comment='' AND
   dclocal_read_repair_chance=0.00 AND
   gc_grace_seconds=864000 AND
   index_interval=128 AND
   read_repair_chance=0.10 AND
   replicate_on_write='true' AND
   populate_io_cache_on_flush='false' AND
   default_time_to_live=0 AND
   speculative_retry='99.0PERCENTILE' AND
   memtable_flush_period_in_ms=0 AND
   compaction={'class': 'SizeTieredCompactionStrategy'} AND
   compression={'sstable_compression': 'LZ4Compressor'};
 {code}
 in the keyspace.
 On client we execute next prepared statement:
 {code}
 session.prepare(
 SELECT id FROM processed_documents WHERE id IN :ids
 ).setConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM)
 {code}
 Used Cassandra session has next main properties:
   * Load balancing policy - DCAwareRoundRobinPolicy(localDc, 
 usedHostPerRemoteDc = 3, allowRemoteDcForLocalConsistencyLevel = true)
   * Retry policy - DefaultRetryPolicy
   * Query options - QueryOptions with set consistency level to 
 ConsistencyLevel.LOCAL_QUORUM
 Our problem is next.
 Since some moment there are next errors in the client application log:
 {code}
 com.datastax.driver.core.exceptions.ReadTimeoutException: Cassandra timeout 
 during read query at consistency ALL (9 responses were required but only 8 
 replica responded)
 at 
 com.datastax.driver.core.exceptions.ReadTimeoutException.copy(ReadTimeoutException.java:69)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.Responses$Error.asException(Responses.java:94) 
 ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.DefaultResultSetFuture.onSet(DefaultResultSetFuture.java:108)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.RequestHandler.setFinalResult(RequestHandler.java:235)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.RequestHandler.onSet(RequestHandler.java:379) 
 ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 com.datastax.driver.core.Connection$Dispatcher.messageReceived(Connection.java:571)
  ~[cassandra-driver-core-2.0.2.jar:na]
 at 
 org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) 
 ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:70)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) 
 ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:70)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
  ~[netty-3.9.0.Final.jar:na]
 at 
 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) 
 ~[netty-3.9.0.Final.jar:na]
 at