Compare commits

...

3 Commits

Author SHA1 Message Date
Piotr Smaron
d4c28690e1 db: fail reads and writes with local consistencty level to a DC with RF=0
When read or write operations are performed on a DC with RF=0 with LOCAL_QUORUM
or LOCAL_ONE consistency level, Cassandra throws `Unavailable` exception.
Scylla allowed such read operations and failed write operations with a cryptic:
"broken promise" error. This occured because the initial availability
check passed (quorum of 0 requires 0 replicas), but execution failed
later when no replicas existed to process the mutation.

This patch adds an explicit RF=0 validation for LOCAL_ONE and LOCAL_QUORUM that
throws before attempting operation execution.

The change also requires `test_query_dc_with_rf_0_does_not_crash_db` to be
upgraded. This testcase was asserting somewhat similar scenario, but wasn't
taking into account the whole matrix of combinations:
- scenarios: successful vs unsuccesful operation outcome
- local consistency levels: LOCAL_QUORUM & LOCAL_ONE
- operations: SELECT (read) & INSERT (write)

and so it's been extended to cover both the pre-existing and the current issues
and the whole matrix of combinations.

Fixes: scylladb/scylladb#27893
2026-01-22 12:49:45 +01:00
Piotr Smaron
9475659ae8 db: consistency_level: split local_quorum_for()
The core of `local_quorum_for()` has been extracted to
`get_replication_factor_for_dc()`, which is going to be used later,
while `local_quorum_for()` itself has been recreated using the exracted
part.
2026-01-22 12:49:23 +01:00
Piotr Smaron
0b3ee197b6 db: consistency_level: fix nrs -> nts abbreviation
`network_topology_strategy` was abbreviated with `nrs`, and not `nts`. I
think someone incorrectly assumed it's 'network Replication strategy', hence
nrs.
2026-01-22 12:48:37 +01:00
2 changed files with 66 additions and 22 deletions

View File

@@ -31,19 +31,23 @@ size_t quorum_for(const locator::effective_replication_map& erm) {
return replication_factor ? (replication_factor / 2) + 1 : 0;
}
size_t local_quorum_for(const locator::effective_replication_map& erm, const sstring& dc) {
static size_t get_replication_factor_for_dc(const locator::effective_replication_map& erm, const sstring& dc) {
using namespace locator;
const auto& rs = erm.get_replication_strategy();
if (rs.get_type() == replication_strategy_type::network_topology) {
const network_topology_strategy* nrs =
const network_topology_strategy* nts =
static_cast<const network_topology_strategy*>(&rs);
size_t replication_factor = nrs->get_replication_factor(dc);
return replication_factor ? (replication_factor / 2) + 1 : 0;
return nts->get_replication_factor(dc);
}
return quorum_for(erm);
return erm.get_replication_factor();
}
size_t local_quorum_for(const locator::effective_replication_map& erm, const sstring& dc) {
auto rf = get_replication_factor_for_dc(erm, dc);
return rf ? (rf / 2) + 1 : 0;
}
size_t block_for_local_serial(const locator::effective_replication_map& erm) {
@@ -188,18 +192,30 @@ void assure_sufficient_live_nodes(
return pending <= live ? live - pending : 0;
};
auto make_rf_zero_error_msg = [cl] (const sstring& local_dc) {
return format("Cannot achieve consistency level {} in datacenter '{}' with replication factor 0. "
"Ensure the keyspace is replicated to this datacenter or use a non-local consistency level.", cl, local_dc);
};
const auto& topo = erm.get_topology();
const sstring& local_dc = topo.get_datacenter();
switch (cl) {
case consistency_level::ANY:
// local hint is acceptable, and local node is always live
break;
case consistency_level::LOCAL_ONE:
if (size_t local_rf = get_replication_factor_for_dc(erm, local_dc); local_rf == 0) {
throw exceptions::unavailable_exception(make_rf_zero_error_msg(local_dc), cl, 1, 0);
}
if (topo.count_local_endpoints(live_endpoints) < topo.count_local_endpoints(pending_endpoints) + 1) {
throw exceptions::unavailable_exception(cl, 1, 0);
}
break;
case consistency_level::LOCAL_QUORUM: {
if (size_t local_rf = get_replication_factor_for_dc(erm, local_dc); local_rf == 0) {
throw exceptions::unavailable_exception(make_rf_zero_error_msg(local_dc), cl, need, 0);
}
size_t local_live = topo.count_local_endpoints(live_endpoints);
size_t pending = topo.count_local_endpoints(pending_endpoints);
if (local_live < need + pending) {

View File

@@ -13,7 +13,8 @@ import pytest
from cassandra.policies import WhiteListRoundRobinPolicy
from test.cqlpy import nodetool
from cassandra import ConsistencyLevel
from cassandra import ConsistencyLevel, Unavailable
from cassandra.cluster import NoHostAvailable
from cassandra.protocol import InvalidRequest, ConfigurationException
from cassandra.query import SimpleStatement
from test.pylib.async_cql import _wrap_future
@@ -113,14 +114,19 @@ async def test_putget_2dc_with_rf(
@pytest.mark.asyncio
async def test_query_dc_with_rf_0_does_not_crash_db(request: pytest.FixtureRequest, manager: ManagerClient):
"""Test querying dc with CL=LOCAL_QUORUM when RF=0 for this dc, does not crash the node and returns None
Covers https://github.com/scylladb/scylla/issues/8354"""
async def test_read_or_write_to_dc_with_rf_0_fails(request: pytest.FixtureRequest, manager: ManagerClient):
"""
Verifies that operations using local consistency levels (LOCAL_QUORUM, LOCAL_ONE) fail
with a clear, actionable error message when the datacenter has replication factor 0.
Covers:
- https://github.com/scylladb/scylla/issues/8354 - Operations should not crash the DB
- https://github.com/scylladb/scylladb/issues/27893 - Should give a clear error about RF=0
"""
servers = []
ks = "test_ks"
table_name = "test_table_name"
expected = ["k1", "value1"]
dc_replication = {'dc2': 0}
dc_replication = {'dc1': 1, 'dc2': 0}
columns = [Column("name", TextType), Column("value", TextType)]
for i in [1, 2]:
@@ -136,18 +142,40 @@ async def test_query_dc_with_rf_0_does_not_crash_db(request: pytest.FixtureReque
random_tables = RandomTables(request.node.name, manager, ks, 1, dc_replication)
await random_tables.add_table(ncolumns=2, columns=columns, pks=1, name=table_name)
dc1_connection.execute(
f"INSERT INTO {ks}.{table_name} ({columns[0].name}, {columns[1].name}) VALUES ('{expected[0]}', '{expected[1]}');")
select_query = SimpleStatement(f"SELECT * from {ks}.{table_name};",
consistency_level=ConsistencyLevel.LOCAL_QUORUM)
nodetool.flush(dc1_connection, "{ks}.{table_name}")
first_node_results = list(dc1_connection.execute(select_query).one())
second_node_result = dc2_connection.execute(select_query).one()
assert first_node_results == expected, \
f"Expected {expected} from {select_query.query_string}, but got {first_node_results}"
assert second_node_result is None, \
f"Expected no results from {select_query.query_string}, but got {second_node_result}"
# sanity check: operations from dc1 (RF > 0) should succeed with all local consistency levels
local_cls = [ConsistencyLevel.LOCAL_QUORUM, ConsistencyLevel.LOCAL_ONE]
for i, cl in enumerate(local_cls):
dc1_connection.execute(SimpleStatement(
f"INSERT INTO {ks}.{table_name} ({columns[0].name}, {columns[1].name}) VALUES ('k{i}', 'value{i}')", consistency_level=cl))
nodetool.flush(dc1_connection, f"{ks}.{table_name}")
select_query = SimpleStatement(f"SELECT * FROM {ks}.{table_name} WHERE {columns[0].name} = 'k{i}'", consistency_level=cl)
# asserting SELECT's results may seem excessive, but it could happen that it'd fail and return nothing and not throw,
# in which case it'd silently return nothing, which we don't want, and hence are asserting it's *really* working
result = list(dc1_connection.execute(select_query).one())
expected_row = [f"k{i}", f"value{i}"]
assert result == expected_row, \
f"Expected {expected_row} with CL={cl} from dc1, but got {result}"
def assert_operation_fails_with_rf0_error(cl: ConsistencyLevel, operation: str) -> None:
with pytest.raises((Unavailable, NoHostAvailable)) as exc_info:
dc2_connection.execute(SimpleStatement(operation, consistency_level=cl))
error_msg = str(exc_info.value)
assert "Cannot achieve consistency level LOCAL_" in error_msg and "use a non-local consistency level" in error_msg, \
f"Expected error indicating RF=0 and datacenter with CL={cl}, but received: {exc_info.value}"
# SELECT & INSERT from dc2 (with RF=0) using local CLs should fail with a clear error message
# indicating the replication factor is 0 and suggesting to use a non-local CL
for i, cl in enumerate(local_cls):
assert_operation_fails_with_rf0_error(cl,
f"SELECT * FROM {ks}.{table_name}")
assert_operation_fails_with_rf0_error(cl,
f"INSERT INTO {ks}.{table_name} ({columns[0].name}, {columns[1].name}) VALUES ('k_fail_{i}', 'value_fail_{i}')")
@pytest.mark.asyncio
async def test_create_and_alter_keyspace_with_altering_rf_and_racks(manager: ManagerClient):