[
https://issues.apache.org/jira/browse/CASSANDRA-20798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Ling Mao updated CASSANDRA-20798:
---------------------------------
Description:
h4. How to reproduce
{code:java}
CREATE KEYSPACE c_simple_3_1 WITH replication = {'class': 'SimpleStrategy',
'replication_factor' : '3/1'};
use c_simple_3_1;
CREATE TABLE users (
user_id varchar PRIMARY KEY,
first varchar,
last varchar,
age int
) WITH read_repair = 'NONE';
cqlsh:c_simple_3_1> consistency;
Current consistency level is ONE.
INSERT INTO users (user_id, first, last, age)
VALUES ('jsmith', 'John', 'Smith', 42);
INSERT INTO users (user_id, first, last, age)
VALUES ('foo', 'foo', 'foo', 18);
INSERT INTO users (user_id, first, last, age)
VALUES ('bar', 'bar', 'bar', 19);
INSERT INTO users (user_id, first, last, age)
VALUES ('abc', 'abc', 'abc', 20);
cqlsh:c_simple_3_1> SELECT * FROM users;
NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host:
127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error]
message="java.lang.AssertionError">})
cqlsh:c_simple_3_1> SELECT * FROM users where user_id = 'jsmith';
user_id | age | first | last
---------+-----+-------+-------
jsmith | 42 | John | Smith
(1 rows)
cqlsh:c_simple_3_1> SELECT * FROM users where user_id = 'jsmith';
NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host:
127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error]
message="java.lang.AssertionError">})
{code}
h4. Exception stack
{code:java}
ERROR [Native-Transport-Requests-1] 2025-07-23 19:06:16,161
ExceptionHandlers.java:246 - Unexpected exception during request; channel =
[id: 0x65a448a1, L:/127.0.0.1:9042 - R:/127.0.0.1:55121]
java.lang.AssertionError: null
at
org.apache.cassandra.exceptions.UnavailableException.create(UnavailableException.java:39)
at
org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicas(ReplicaPlans.java:200)
at
org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicasForRead(ReplicaPlans.java:139)
at
org.apache.cassandra.locator.ReplicaPlans.forRangeRead(ReplicaPlans.java:964)
at
org.apache.cassandra.locator.ReplicaPlans.forRangeRead(ReplicaPlans.java:945)
at
org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:89)
at
org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:46)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at com.google.common.collect.Iterators$PeekingImpl.hasNext(Iterators.java:1191)
at
org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:61)
at
org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:34)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at
org.apache.cassandra.service.reads.range.RangeCommandIterator.computeNext(RangeCommandIterator.java:126)
at
org.apache.cassandra.service.reads.range.RangeCommandIterator.computeNext(RangeCommandIterator.java:74)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:90)
at
org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:1053)
at
org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:636)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:610)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:404)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:154)
at
org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:304)
at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:399)
at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:386)
at
org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:117)
at org.apache.cassandra.transport.Message$Request.execute(Message.java:259)
at
org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:423)
at
org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:442)
at
org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:469)
at
org.apache.cassandra.transport.Dispatcher$RequestProcessor.run(Dispatcher.java:314)
at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:99)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:150)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829){code}
h4. Analysis
It's unexpected that executing the same CQL query against the same coordinator
can return different results under transient replication mode.
After some debugging, we found the root cause is that the candidate replicas
are selected in a non-deterministic order.
If a transient replica happens to be the first element in the candidates list,
it may violate the requirement that at least one full replica must be involved
in the read, leading to inconsistent behavior.
{code:java}
EndpointsForToken candidates = candidatesForRead(keyspace, indexQueryPlan,
consistencyLevel, forTokenReadLive.all());
EndpointsForToken contacts = contactForRead(metadata.locator,
replicationStrategy, consistencyLevel,
retry.equals(AlwaysSpeculativeRetryPolicy.INSTANCE), candidates);{code}
{code:java}
In `org.apache.cassandra.locator.ReplicaPlans#assureSufficientLiveReplicas`:
default:
int live = allLive.size();
int full = Replicas.countFull(allLive);
if (live < blockFor || full < blockForFullReplicas)
{
if (logger.isTraceEnabled())
logger.trace("Live nodes {} do not satisfy ConsistencyLevel ({}
required)", Iterables.toString(allLive), blockFor);
throw UnavailableException.create(consistencyLevel, blockFor,
blockForFullReplicas, live, full);
}
#################################
In this case, if the list contains only transient replicas, `full` will be 0.
If `blockForFullReplicas` is 1, the condition `full(0) <
blockForFullReplicas(1)` is satisfied, triggering an `UnavailableException`.
{code}
h4. Solution
To solve this, we introduce a method _*reorderWithOneFullReplicaFirst*_ that
attempts to ensure at least one full replica is placed at the head of the
replica lis, without disturbing the rest of the ordering (which is typically
proximity-based)
was:
h4. How to reproduce
{code:java}
CREATE KEYSPACE c_simple_3_1 WITH replication = {'class': 'SimpleStrategy',
'replication_factor' : '3/1'};
use c_simple_3_1;
CREATE TABLE users (
user_id varchar PRIMARY KEY,
first varchar,
last varchar,
age int
) WITH read_repair = 'NONE';
cqlsh:c_simple_3_1> consistency;
Current consistency level is ONE.
INSERT INTO users (user_id, first, last, age)
VALUES ('jsmith', 'John', 'Smith', 42);
INSERT INTO users (user_id, first, last, age)
VALUES ('foo', 'foo', 'foo', 18);
INSERT INTO users (user_id, first, last, age)
VALUES ('bar', 'bar', 'bar', 19);
INSERT INTO users (user_id, first, last, age)
VALUES ('abc', 'abc', 'abc', 20);
cqlsh:c_simple_3_1> SELECT * FROM users;
NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host:
127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error]
message="java.lang.AssertionError">})
cqlsh:c_simple_3_1> SELECT * FROM users where user_id = 'jsmith';
user_id | age | first | last
---------+-----+-------+-------
jsmith | 42 | John | Smith
(1 rows)
cqlsh:c_simple_3_1> SELECT * FROM users where user_id = 'jsmith';
NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host:
127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error]
message="java.lang.AssertionError">})
{code}
h4. Exception stack
{code:java}
ERROR [Native-Transport-Requests-1] 2025-07-23 19:06:16,161
ExceptionHandlers.java:246 - Unexpected exception during request; channel =
[id: 0x65a448a1, L:/127.0.0.1:9042 - R:/127.0.0.1:55121]
java.lang.AssertionError: null
at
org.apache.cassandra.exceptions.UnavailableException.create(UnavailableException.java:39)
at
org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicas(ReplicaPlans.java:200)
at
org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicasForRead(ReplicaPlans.java:139)
at
org.apache.cassandra.locator.ReplicaPlans.forRangeRead(ReplicaPlans.java:964)
at
org.apache.cassandra.locator.ReplicaPlans.forRangeRead(ReplicaPlans.java:945)
at
org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:89)
at
org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:46)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at com.google.common.collect.Iterators$PeekingImpl.hasNext(Iterators.java:1191)
at
org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:61)
at
org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:34)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at
org.apache.cassandra.service.reads.range.RangeCommandIterator.computeNext(RangeCommandIterator.java:126)
at
org.apache.cassandra.service.reads.range.RangeCommandIterator.computeNext(RangeCommandIterator.java:74)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:90)
at
org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:1053)
at
org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:636)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:610)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:404)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:154)
at
org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:304)
at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:399)
at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:386)
at
org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:117)
at org.apache.cassandra.transport.Message$Request.execute(Message.java:259)
at
org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:423)
at
org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:442)
at
org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:469)
at
org.apache.cassandra.transport.Dispatcher$RequestProcessor.run(Dispatcher.java:314)
at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:99)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:150)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829){code}
h4. Analysis
It's *{*}unexpected{*}* that executing the same CQL query against the same
coordinator can return *{*}different results{*}* under *{*}transient
replication{*}* mode.
After some debugging, we found the root cause is that the *{*}candidate
replicas are selected in a non-deterministic order{*}*.
If a *{*}transient replica{*}* happens to be the *{*}first element{*}* in the
candidates list, it may *{*}violate the requirement that at least one full
replica must be involved in the read{*}*, leading to inconsistent behavior.
{code:java}
EndpointsForToken candidates = candidatesForRead(keyspace, indexQueryPlan,
consistencyLevel, forTokenReadLive.all());
EndpointsForToken contacts = contactForRead(metadata.locator,
replicationStrategy, consistencyLevel,
retry.equals(AlwaysSpeculativeRetryPolicy.INSTANCE), candidates);{code}
{code:java}
In `org.apache.cassandra.locator.ReplicaPlans#assureSufficientLiveReplicas`:
default:
int live = allLive.size();
int full = Replicas.countFull(allLive);
if (live < blockFor || full < blockForFullReplicas)
{
if (logger.isTraceEnabled())
logger.trace("Live nodes {} do not satisfy ConsistencyLevel ({}
required)", Iterables.toString(allLive), blockFor);
throw UnavailableException.create(consistencyLevel, blockFor,
blockForFullReplicas, live, full);
}
#################################
In this case, if the list contains only transient replicas, `full` will be 0.
If `blockForFullReplicas` is 1, the condition `full(0) <
blockForFullReplicas(1)` is satisfied, triggering an `UnavailableException`.
{code}
h4. Solution
To solve this, we introduce a method `reorderWithOneFullReplicaFirst(...)` that
attempts to ensure *{*}at least one full replica is placed at the head of the
replica list{*}*, without disturbing the rest of the ordering (which is
typically proximity-based)
> Fix non deterministic reads in transient replication
> ----------------------------------------------------
>
> Key: CASSANDRA-20798
> URL: https://issues.apache.org/jira/browse/CASSANDRA-20798
> Project: Apache Cassandra
> Issue Type: Improvement
> Components: Feature/Transient Replication
> Reporter: Ling Mao
> Assignee: Ling Mao
> Priority: Normal
> Time Spent: 10m
> Remaining Estimate: 0h
>
> h4. How to reproduce
> {code:java}
> CREATE KEYSPACE c_simple_3_1 WITH replication = {'class': 'SimpleStrategy',
> 'replication_factor' : '3/1'};
> use c_simple_3_1;
> CREATE TABLE users (
> user_id varchar PRIMARY KEY,
> first varchar,
> last varchar,
> age int
> ) WITH read_repair = 'NONE';
> cqlsh:c_simple_3_1> consistency;
> Current consistency level is ONE.
> INSERT INTO users (user_id, first, last, age)
> VALUES ('jsmith', 'John', 'Smith', 42);
> INSERT INTO users (user_id, first, last, age)
> VALUES ('foo', 'foo', 'foo', 18);
> INSERT INTO users (user_id, first, last, age)
> VALUES ('bar', 'bar', 'bar', 19);
> INSERT INTO users (user_id, first, last, age)
> VALUES ('abc', 'abc', 'abc', 20);
> cqlsh:c_simple_3_1> SELECT * FROM users;
> NoHostAvailable: ('Unable to complete the operation against any hosts',
> {<Host: 127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server
> error] message="java.lang.AssertionError">})
> cqlsh:c_simple_3_1> SELECT * FROM users where user_id = 'jsmith';
> user_id | age | first | last
> ---------+-----+-------+-------
> jsmith | 42 | John | Smith
> (1 rows)
> cqlsh:c_simple_3_1> SELECT * FROM users where user_id = 'jsmith';
> NoHostAvailable: ('Unable to complete the operation against any hosts',
> {<Host: 127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server
> error] message="java.lang.AssertionError">})
> {code}
> h4. Exception stack
> {code:java}
> ERROR [Native-Transport-Requests-1] 2025-07-23 19:06:16,161
> ExceptionHandlers.java:246 - Unexpected exception during request; channel =
> [id: 0x65a448a1, L:/127.0.0.1:9042 - R:/127.0.0.1:55121]
> java.lang.AssertionError: null
> at
> org.apache.cassandra.exceptions.UnavailableException.create(UnavailableException.java:39)
> at
> org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicas(ReplicaPlans.java:200)
> at
> org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicasForRead(ReplicaPlans.java:139)
> at
> org.apache.cassandra.locator.ReplicaPlans.forRangeRead(ReplicaPlans.java:964)
> at
> org.apache.cassandra.locator.ReplicaPlans.forRangeRead(ReplicaPlans.java:945)
> at
> org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:89)
> at
> org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:46)
> at
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> at
> com.google.common.collect.Iterators$PeekingImpl.hasNext(Iterators.java:1191)
> at
> org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:61)
> at
> org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:34)
> at
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> at
> org.apache.cassandra.service.reads.range.RangeCommandIterator.computeNext(RangeCommandIterator.java:126)
> at
> org.apache.cassandra.service.reads.range.RangeCommandIterator.computeNext(RangeCommandIterator.java:74)
> at
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> at
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:90)
> at
> org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:1053)
> at
> org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:636)
> at
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:610)
> at
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:404)
> at
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:154)
> at
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:304)
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:399)
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:386)
> at
> org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:117)
> at org.apache.cassandra.transport.Message$Request.execute(Message.java:259)
> at
> org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:423)
> at
> org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:442)
> at
> org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:469)
> at
> org.apache.cassandra.transport.Dispatcher$RequestProcessor.run(Dispatcher.java:314)
> at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:99)
> at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
> at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:150)
> at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.base/java.lang.Thread.run(Thread.java:829){code}
> h4. Analysis
> It's unexpected that executing the same CQL query against the same
> coordinator can return different results under transient replication mode.
> After some debugging, we found the root cause is that the candidate replicas
> are selected in a non-deterministic order.
> If a transient replica happens to be the first element in the candidates
> list, it may violate the requirement that at least one full replica must be
> involved in the read, leading to inconsistent behavior.
> {code:java}
> EndpointsForToken candidates = candidatesForRead(keyspace, indexQueryPlan,
> consistencyLevel, forTokenReadLive.all());
> EndpointsForToken contacts = contactForRead(metadata.locator,
> replicationStrategy, consistencyLevel,
> retry.equals(AlwaysSpeculativeRetryPolicy.INSTANCE), candidates);{code}
> {code:java}
> In `org.apache.cassandra.locator.ReplicaPlans#assureSufficientLiveReplicas`:
> default:
> int live = allLive.size();
> int full = Replicas.countFull(allLive);
> if (live < blockFor || full < blockForFullReplicas)
> {
> if (logger.isTraceEnabled())
> logger.trace("Live nodes {} do not satisfy ConsistencyLevel ({}
> required)", Iterables.toString(allLive), blockFor);
> throw UnavailableException.create(consistencyLevel, blockFor,
> blockForFullReplicas, live, full);
> }
> #################################
> In this case, if the list contains only transient replicas, `full` will be 0.
> If `blockForFullReplicas` is 1, the condition `full(0) <
> blockForFullReplicas(1)` is satisfied, triggering an `UnavailableException`.
> {code}
> h4. Solution
> To solve this, we introduce a method _*reorderWithOneFullReplicaFirst*_ that
> attempts to ensure at least one full replica is placed at the head of the
> replica lis, without disturbing the rest of the ordering (which is typically
> proximity-based)
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]