Thanks for the link, I hadn't seen that before. It's unfortunate that they don't explain what they mean by "closest replica". The nodes in the remote DC should not be regarded as "closest". Also, it's not clear what the black arrows mean… the coordinator sends the read to all three replicas, but only one of them responds?
Reading further (assuming this article from 2012 is still accurate for Cassandra 3.0 http://www.datastax.com/dev/blog/dynamic-snitching-in-cassandra-past-present-and-future), it appears that by "closest replica" what they really mean is the replica chosen by the "dynamic snitch". The structure of the documentation https://docs.datastax.com/en/cassandra/3.0/cassandra/architecture/archSnitchesAbout.html is misleading in this regard: it puts the "Dynamic snitching" section side by side with the other snitch implementations, implying that it's one of the choices you can configure as a snitch, which is why I hadn't read that section (I didn't want a snitch that "monitors the performance of reads"). Instead, the info about the dynamic snitch should be in the top-level page. In any case, the dynamic snitch is apparently governed by read latency, the node's state, and whether the node is doing a compaction ("severity"). So why is it routing requests to nodes with latency that's ~20 times larger? I don't know, but I wish it wasn't. I guess it's important to differentiate between that and the load balancing policy called LatencyAwarePolicy… even if you're not using the LatencyAwarePolicy, internally the snitch is still doing stuff based on latency. This is also unfortunate because it makes the DCAwareRoundRobinPolicy useless for anything but local consistency levels, and (if you read it at face value) contradicts the description in the documentation that "This policy queries nodes of the local data-center in a round-robin fashion; optionally, it can also try a configurable number of hosts in remote data centers if all local hosts failed." Also, if you're right that the requests are getting routed to the remote DC, then those requests aren't showing up in my graph of read request rate… which is problematic because I'm not getting an accurate representation of what's actually happening. I can't find any other metric beyond org.apache.cassandra.metrics.ClientRequest.* which might include these internal read requests. I am wondering if perhaps there's a mistake with the way that the dynamic snitch measures latency… if it's only measuring requests coming from clients, then if a remote node happens to win the dynamic snitch's favor momentarily, the latency of the local node will increase (because it's querying the remote node), and then the dynamic snitch will see that the local node is performing poorly, and will continue directing traffic to the remote cluster. Or, perhaps they're measuring the latency of each node based not on how long the client request takes but based on how long the internal request takes… which, again, could mislead the snitch into thinking that the remote host is providing a better deal to the client than it really is. It seems like a mistake that the dynamic switch would think that a remote node will be faster or less work than the local node which actually contains a copy of the data being queried. Looks like I'm not the only one who's run into this: https://issues.apache.org/jira/browse/CASSANDRA-6908 I think I'm going to try setting the system property "cassandra.ignore_dynamic_snitch_severity" to "true" and see what happens. That or "dynamic_snitch: false"… it's not documented in https://docs.datastax.com/en/cassandra/3.0/cassandra/configuration/configCassandra_yaml.html but it appears to be a valid config option. From: Eric Plowe <eric.pl...@gmail.com<mailto:eric.pl...@gmail.com>> Reply-To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" <user@cassandra.apache.org<mailto:user@cassandra.apache.org>> Date: Wednesday, March 22, 2017 at 11:44 AM To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" <user@cassandra.apache.org<mailto:user@cassandra.apache.org>> Subject: Re: ONE has much higher latency than LOCAL_ONE Yes, your request from the client is going to the LocalDC that you've defined for the data center aware load balancing policy, but with a consistency level of ONE, there is a chance for the coordinator (the node your client has connected to) to route the request across DC's. Please see: https://docs.datastax.com/en/cassandra/2.1/cassandra/dml/dmlClientRequestsRead.html#dmlClientRequestsRead__two-dc-one "A two datacenter cluster with a consistency level of ONE "In a multiple datacenter cluster with a replication factor of 3, and a read consistency of ONE, the closest replica for the given row, regardless of datacenter, is contacted to fulfill the read request. In the background a read repair is potentially initiated, based on the read_repair_chance setting of the table, for the other replicas." A two datacenter cluster with a consistency level of LOCAL_ONE <https://docs.datastax.com/en/cassandra/2.1/cassandra/dml/dmlClientRequestsRead.html#dmlClientRequestsRead__two-dc-local-one> In a multiple datacenter cluster with a replication factor of 3, and a read consistency of LOCAL_ONE, the closest replica for the given row in the same datacenter as the coordinator node is contacted to fulfill the read request. In the background a read repair is potentially initiated, based on the read_repair_chance setting of the table, for the other replicas." Dynamic snitching also comes into play with reads. Just because your client is using TokenAware, and should connect to the appropriate replica node (which now is your coordinator) it can route your read request away from what it believes to be poorly performing nodes to another replica which could be in the other DC with CL = ONE. Read more about dynamic snitch here: https://docs.datastax.com/en/cassandra/2.1/cassandra/architecture/architectureSnitchDynamic_c.html Regards, Eric Plowe