mirror of
https://github.com/scylladb/scylladb.git
synced 2026-06-04 22:13:19 +00:00
tablets: Make sure topology has enough endpoints for RF
When creating a keyspace, scylla allows setting RF value smaller than there are nodes in the DC. With vnodes, when new nodes are bootstrapped, new tokens are inserted thus catching up with RF. With tablets, it's not the case as replica set remains unchanged. With tablets it's good chance not to mimic the vnodes behavior and require as many nodes to be up and running as the requested RF is. This patch implementes this in a lazy manned -- when creating a keyspace RF can be any, but when a new table is created the topology should meet RF requirements. If not met, user can bootstrap new nodes or ALTER KEYSPACE. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
This commit is contained in:
30
test/topology_custom/test_tablets.py
Normal file
30
test/topology_custom/test_tablets.py
Normal file
@@ -0,0 +1,30 @@
|
||||
#
|
||||
# Copyright (C) 2024-present ScyllaDB
|
||||
#
|
||||
# SPDX-License-Identifier: AGPL-3.0-or-later
|
||||
#
|
||||
from cassandra.protocol import ConfigurationException
|
||||
from test.pylib.manager_client import ManagerClient
|
||||
import pytest
|
||||
import logging
|
||||
import asyncio
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
|
||||
|
||||
@pytest.mark.asyncio
|
||||
async def test_tablet_replication_factor_enough_nodes(manager: ManagerClient):
|
||||
cfg = {'enable_user_defined_functions': False,
|
||||
'experimental_features': ['tablets', 'consistent-topology-changes']}
|
||||
servers = await manager.servers_add(2, config=cfg)
|
||||
|
||||
cql = manager.get_cql()
|
||||
res = await cql.run_async("SELECT data_center FROM system.local")
|
||||
this_dc = res[0].data_center
|
||||
|
||||
await cql.run_async(f"CREATE KEYSPACE test WITH replication = {{'class': 'NetworkTopologyStrategy', '{this_dc}': 3}}")
|
||||
with pytest.raises(ConfigurationException, match=f"Datacenter {this_dc} doesn't have enough nodes"):
|
||||
await cql.run_async("CREATE TABLE test.test (pk int PRIMARY KEY, c int);")
|
||||
|
||||
await cql.run_async(f"ALTER KEYSPACE test WITH replication = {{'class': 'NetworkTopologyStrategy', '{this_dc}': 2}}")
|
||||
await cql.run_async("CREATE TABLE test.test (pk int PRIMARY KEY, c int);")
|
||||
Reference in New Issue
Block a user