From d18eb9479f616cb6ceac4a5f09c91a3ac7822d79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Paszkowski?= Date: Tue, 5 Aug 2025 13:28:59 +0200 Subject: [PATCH] cql/statement: Create keyspace_metadata with correct initial_tablets count In `ks_prop_defs::as_ks_metadata(...)` a default initial tablets count is set to 0, when tablets are enabled and the replication strategy is NetworkReplicationStrategy. This effectively sets _uses_tablets = false in abstract_replication_strategy for the remaining strategies when no `tablets = {...}` options are specified. As a consequence, it is possible to create vnode-based keyspaces even when tablets are enforced with `tablets_mode_for_new_keyspaces`. The patch sets a default initial tablets count to zero regardless of the chosen replication strategy. Then each of the replication strategy validates the options and raises a configuration exception when tablets are not supported. All tests are altered in the following way: + whenever it was correct, SimpleStrategy was replaced with NetworkTopologyStrategy + otherwise, tablets were explicitly disabled with ` AND tablets = {'enabled': false}` Fixes https://github.com/scylladb/scylladb/issues/25340 Closes scylladb/scylladb#25342 --- cql3/statements/ks_prop_defs.cc | 6 +- test/cluster/mv/test_mv_delete_partitions.py | 2 +- test/cluster/mv/test_mv_fail_building.py | 4 +- test/cluster/test_aggregation.py | 2 +- test/cluster/test_audit.py | 10 +-- ...est_commitlog_segment_data_resurrection.py | 4 +- test/cluster/test_hints.py | 25 ++++--- test/cluster/test_ip_mappings.py | 2 +- test/cluster/test_lwt_semaphore.py | 2 +- test/cluster/test_node_isolation.py | 2 +- ...ode_shutdown_waits_for_pending_requests.py | 4 +- test/cluster/test_query_rebounce.py | 2 +- test/cluster/test_raft_no_quorum.py | 6 +- test/cluster/test_raft_snapshot_truncation.py | 6 +- test/cluster/test_shutdown_hang.py | 8 +-- .../test_sstable_compression_config.py | 6 +- test/cluster/test_tablets.py | 68 +++++++++++-------- .../validation/operations/create_test.py | 8 +-- .../insert_update_if_condition_test.py | 4 +- test/cqlpy/nodetool.py | 8 +++ test/cqlpy/test_describe.py | 4 +- .../test_guardrail_replication_strategy.py | 29 +++++--- test/cqlpy/test_keyspace.py | 7 +- test/cqlpy/test_name.py | 2 +- test/cqlpy/test_permissions.py | 4 +- test/cqlpy/test_tablets.py | 2 +- 26 files changed, 132 insertions(+), 95 deletions(-) diff --git a/cql3/statements/ks_prop_defs.cc b/cql3/statements/ks_prop_defs.cc index bd9eab4e28..79449bb458 100644 --- a/cql3/statements/ks_prop_defs.cc +++ b/cql3/statements/ks_prop_defs.cc @@ -411,10 +411,10 @@ bool ks_prop_defs::get_durable_writes() const { lw_shared_ptr ks_prop_defs::as_ks_metadata(sstring ks_name, const locator::token_metadata& tm, const gms::feature_service& feat, const db::config& cfg) { auto sc = get_replication_strategy_class().value(); - // if tablets options have not been specified, but tablets are globally enabled, set the value to 0 for N.T.S. only + // if tablets options have not been specified, but tablets are globally enabled, set the value to 0. The strategy will + // validate it and throw an error if it does not support tablets. auto enable_tablets = feat.tablets && cfg.enable_tablets_by_default(); - std::optional default_initial_tablets = enable_tablets && locator::abstract_replication_strategy::to_qualified_class_name(sc) == "org.apache.cassandra.locator.NetworkTopologyStrategy" - ? std::optional(0) : std::nullopt; + std::optional default_initial_tablets = enable_tablets ? std::optional(0) : std::nullopt; auto initial_tablets = get_initial_tablets(default_initial_tablets, cfg.enforce_tablets()); bool uses_tablets = initial_tablets.has_value(); bool rack_list_enabled = utils::get_local_injector().enter("create_with_numeric") ? false : feat.rack_list_rf; diff --git a/test/cluster/mv/test_mv_delete_partitions.py b/test/cluster/mv/test_mv_delete_partitions.py index 3daf32e302..f09bc4fa99 100644 --- a/test/cluster/mv/test_mv_delete_partitions.py +++ b/test/cluster/mv/test_mv_delete_partitions.py @@ -62,7 +62,7 @@ async def test_delete_partition_rows_from_table_with_mv(manager: ManagerClient) node_count = 2 await manager.servers_add(node_count, config={'error_injections_at_startup': ['view_update_limit', 'delay_before_remote_view_update']}) cql = manager.get_cql() - async with new_test_keyspace(manager, "WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}") as ks: + async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") as ks: await cql.run_async(f"CREATE TABLE {ks}.tab (key int, c int, PRIMARY KEY (key, c))") await insert_with_concurrency(cql, f"{ks}.tab", 200, 100) diff --git a/test/cluster/mv/test_mv_fail_building.py b/test/cluster/mv/test_mv_fail_building.py index be3dd6b5e7..e3afc35ad4 100644 --- a/test/cluster/mv/test_mv_fail_building.py +++ b/test/cluster/mv/test_mv_fail_building.py @@ -19,9 +19,9 @@ from cassandra.query import SimpleStatement # type: ignore @pytest.mark.skip_mode(mode='release', reason="error injections aren't enabled in release mode") async def test_mv_fail_building(manager: ManagerClient) -> None: node_count = 3 - servers = await manager.servers_add(node_count) + servers = await manager.servers_add(node_count, auto_rack_dc="dc") cql = manager.get_cql() - async with new_test_keyspace(manager, "WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3}") as ks: + async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3}") as ks: await cql.run_async(f"CREATE TABLE {ks}.tab (key int, c int, PRIMARY KEY (key, c))") # Insert initial rows for building an index for i in range(10): diff --git a/test/cluster/test_aggregation.py b/test/cluster/test_aggregation.py index ddd57d1d4e..84f0b4e825 100644 --- a/test/cluster/test_aggregation.py +++ b/test/cluster/test_aggregation.py @@ -40,7 +40,7 @@ async def test_cancel_mapreduce(manager: ManagerClient): [host1] = filter(lambda host: host.address == s1.ip_addr, hosts) host_id2 = await manager.get_host_id(s2.server_id) - async with new_test_keyspace(manager, "WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1}") as ks: + async with new_test_keyspace(manager, f"WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'dc1': {[s2.rack]}}}") as ks: async with new_test_table(manager, ks, "pk int PRIMARY KEY, v int") as t: # Distribute data across the nodes. for _ in range(250): diff --git a/test/cluster/test_audit.py b/test/cluster/test_audit.py index cae327f989..5a5056d607 100644 --- a/test/cluster/test_audit.py +++ b/test/cluster/test_audit.py @@ -460,7 +460,7 @@ class AuditBackendSyslog(AuditBackend): date = datetime.datetime(2000, 1, 1, 0, 0) node = match.group("node").split(":")[0] - statement = match.group("query").replace("\\", "") + statement = match.group("query").replace("\\", "") source = match.group("client_ip").split(":")[0] event_time = uuid.UUID(int=idx) t = self.named_tuple_factory(date, node, event_time, match.group("category"), match.group("cl"), match.group("error") == "true", match.group("keyspace"), statement, source, match.group("table"), match.group("username")) @@ -829,7 +829,7 @@ class CQLAuditTester(AuditTester): self.execute_and_validate_new_audit_entry( session, - "CREATE KEYSPACE ks WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND DURABLE_WRITES = true", + "CREATE KEYSPACE ks WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND tablets = {'enabled': false} AND DURABLE_WRITES = true", category="DDL", ) self.execute_and_validate_new_audit_entry( @@ -853,7 +853,7 @@ class CQLAuditTester(AuditTester): keyspaces = audit_settings["audit_keyspaces"].split(",") if "audit_keyspaces" in audit_settings else [] assert "ks2" not in keyspaces query_sequence = [ - "CREATE KEYSPACE ks2 WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND DURABLE_WRITES = true", + "CREATE KEYSPACE ks2 WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND tablets = {'enabled': false} AND DURABLE_WRITES = true", 'USE "ks2"', "ALTER KEYSPACE ks2 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'dc1' : 1 } AND DURABLE_WRITES = false", "DROP KEYSPACE ks2", @@ -968,7 +968,7 @@ class CQLAuditTester(AuditTester): keyspaces = audit_settings["audit_keyspaces"].split(",") if "audit_keyspaces" in audit_settings else [] assert "ks2" not in keyspaces query_sequence = [ - "CREATE KEYSPACE ks2 WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND DURABLE_WRITES = true", + "CREATE KEYSPACE ks2 WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND tablets = {'enabled': false} AND DURABLE_WRITES = true", f"CREATE TABLE ks2.{first_table} (k int PRIMARY KEY, v1 int)", f"ALTER TABLE ks2.{first_table} ADD v2 int", f"INSERT INTO ks2.{first_table} (k, v1, v2) VALUES (1, 1, 1)", @@ -1009,7 +1009,7 @@ class CQLAuditTester(AuditTester): session = await self.prepare(create_keyspace=False, audit_settings=audit_settings) - session.execute("CREATE KEYSPACE ks WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND DURABLE_WRITES = true") + session.execute("CREATE KEYSPACE ks WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} AND tablets = {'enabled': false} AND DURABLE_WRITES = true") session.execute("USE ks") diff --git a/test/cluster/test_commitlog_segment_data_resurrection.py b/test/cluster/test_commitlog_segment_data_resurrection.py index 659fb116b5..7228769f13 100644 --- a/test/cluster/test_commitlog_segment_data_resurrection.py +++ b/test/cluster/test_commitlog_segment_data_resurrection.py @@ -53,8 +53,8 @@ async def test_pinned_cl_segment_doesnt_resurrect_data(manager: ManagerClient): def get_cl_segments(): return {os.path.basename(s) for s in glob.glob(os.path.join(cl_path, "CommitLog-*"))} - async with new_test_keyspace(manager, "with replication = {'class': 'SimpleStrategy', 'replication_factor': 1}") as ks1, \ - new_test_keyspace(manager, "with replication = {'class': 'SimpleStrategy', 'replication_factor': 1}") as ks2: + async with new_test_keyspace(manager, "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") as ks1, \ + new_test_keyspace(manager, "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") as ks2: tbl1 = f"{ks1}.tbl1" tbl2 = f"{ks2}.tbl2" await cql.run_async(f"create table {tbl1} (pk int, ck int, primary key(pk, ck))") diff --git a/test/cluster/test_hints.py b/test/cluster/test_hints.py index a69198cf88..b4a30ebaf3 100644 --- a/test/cluster/test_hints.py +++ b/test/cluster/test_hints.py @@ -19,6 +19,7 @@ from test.pylib.tablets import get_tablet_replicas from test.pylib.scylla_cluster import ReplaceConfig from test.pylib.util import gather_safely, wait_for +from test.cqlpy import nodetool from test.cluster.util import get_topology_coordinator, find_server_by_host_id, keyspace_has_tablets, new_test_keyspace, new_test_table @@ -97,11 +98,11 @@ async def test_limited_concurrency_of_writes(manager: ManagerClient): """ node1 = await manager.server_add(config={ "error_injections_at_startup": ["decrease_max_size_of_hints_in_progress"] - }) - node2 = await manager.server_add() + }, property_file = {"dc":"dc1", "rack":"rack1"}) + node2 = await manager.server_add(property_file = {"dc":"dc1", "rack":"rack2"}) cql = await manager.get_cql_exclusive(node1) - async with new_test_keyspace(manager, "WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2}") as ks: + async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2}") as ks: table = f"{ks}.t" await cql.run_async(f"CREATE TABLE {table} (pk int primary key, v int)") @@ -128,10 +129,10 @@ async def test_sync_point(manager: ManagerClient): live nodes. """ node_count = 3 - [node1, node2, node3] = await manager.servers_add(node_count) + [node1, node2, node3] = await manager.servers_add(node_count, auto_rack_dc="dc1") cql = manager.get_cql() - async with new_test_keyspace(manager, "WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3}") as ks: + async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3}") as ks: table = f"{ks}.t" await cql.run_async(f"CREATE TABLE {table} (pk int primary key, v int)") @@ -297,12 +298,13 @@ async def test_draining_hints(manager: ManagerClient): This test verifies that all hints are drained when a node is being decommissioned. """ - s1, s2, _ = await manager.servers_add(3) + s1, s2, s3 = await manager.servers_add(3, auto_rack_dc="dc") + cql = manager.get_cql() await manager.api.set_logger_level(s1.ip_addr, "hints_manager", "trace") - await cql.run_async("CREATE KEYSPACE ks WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 3}") + await cql.run_async("CREATE KEYSPACE ks WITH REPLICATION = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3}") await cql.run_async("CREATE TABLE ks.t (pk int PRIMARY KEY, v int)") await manager.server_stop_gracefully(s2.server_id) @@ -314,6 +316,7 @@ async def test_draining_hints(manager: ManagerClient): sync_point = await create_sync_point(manager.api.client, s1.ip_addr) await manager.server_start(s2.server_id) + await cql.run_async(f"ALTER KEYSPACE ks WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'dc': {[s2.rack, s3.rack]}}}") async with asyncio.TaskGroup() as tg: _ = tg.create_task(manager.decommission_node(s1.server_id, timeout=60)) _ = tg.create_task(await_sync_point(manager.api.client, s1.ip_addr, sync_point, 60)) @@ -325,15 +328,14 @@ async def test_canceling_hint_draining(manager: ManagerClient): This test verifies that draining hints is canceled as soon as we issue a shutdown, but it's resumed after starting the node again. """ + s1, s2, s3 = await manager.servers_add(3, auto_rack_dc="dc") - s1, s2, _ = await manager.servers_add(3) cql = manager.get_cql() - host_id2 = await manager.get_host_id(s2.server_id) await manager.api.set_logger_level(s1.ip_addr, "hints_manager", "trace") - await cql.run_async("CREATE KEYSPACE ks WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 3}") + await cql.run_async("CREATE KEYSPACE ks WITH REPLICATION = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3}") await cql.run_async("CREATE TABLE ks.t (pk int PRIMARY KEY, v int)") await manager.server_stop_gracefully(s2.server_id) @@ -345,6 +347,9 @@ async def test_canceling_hint_draining(manager: ManagerClient): sync_point = await create_sync_point(manager.api.client, s1.ip_addr) await manager.api.enable_injection(s1.ip_addr, "hinted_handoff_pause_hint_replay", False, {}) + nodetool.excludenode(cql, host_id2) + await cql.run_async(f"ALTER KEYSPACE ks WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'dc': {[s1.rack, s3.rack]}}}") + await manager.remove_node(s1.server_id, s2.server_id) await manager.server_stop_gracefully(s1.server_id) diff --git a/test/cluster/test_ip_mappings.py b/test/cluster/test_ip_mappings.py index baef4a8e57..8c934ceb8a 100644 --- a/test/cluster/test_ip_mappings.py +++ b/test/cluster/test_ip_mappings.py @@ -23,7 +23,7 @@ async def test_broken_bootstrap(manager: ManagerClient): server_a = await manager.server_add() server_b = await manager.server_add(start=False) - async with new_test_keyspace(manager, "WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1}") as ks: + async with new_test_keyspace(manager, "WITH REPLICATION = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") as ks: table = f"{ks}.test" await manager.cql.run_async(f"CREATE TABLE {table} (a int PRIMARY KEY, b int)") for i in range(100): diff --git a/test/cluster/test_lwt_semaphore.py b/test/cluster/test_lwt_semaphore.py index dc698e3113..20ccfed698 100644 --- a/test/cluster/test_lwt_semaphore.py +++ b/test/cluster/test_lwt_semaphore.py @@ -21,7 +21,7 @@ async def test_cas_semaphore(manager): host = await wait_for_cql_and_get_hosts(manager.cql, {servers[0]}, time.time() + 60) - async with new_test_keyspace(manager, "WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1}") as ks: + async with new_test_keyspace(manager, "WITH REPLICATION = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") as ks: table = f"{ks}.test" await manager.cql.run_async(f"CREATE TABLE {table} (a int PRIMARY KEY, b int)") diff --git a/test/cluster/test_node_isolation.py b/test/cluster/test_node_isolation.py index 6ca257f644..a97187e518 100644 --- a/test/cluster/test_node_isolation.py +++ b/test/cluster/test_node_isolation.py @@ -28,7 +28,7 @@ async def test_banned_node_notification(manager: ManagerClient) -> None: config = { 'failure_detector_timeout_in_ms': 2000 } - srvs = await manager.servers_add(3, config=config) + srvs = await manager.servers_add(3, config=config, auto_rack_dc="dc") cql = manager.get_cql() # Pause one of the servers so other nodes mark it as dead and we can remove it. diff --git a/test/cluster/test_node_shutdown_waits_for_pending_requests.py b/test/cluster/test_node_shutdown_waits_for_pending_requests.py index 1f4ddfd8d0..c70911ed4f 100644 --- a/test/cluster/test_node_shutdown_waits_for_pending_requests.py +++ b/test/cluster/test_node_shutdown_waits_for_pending_requests.py @@ -21,14 +21,14 @@ async def test_node_shutdown_waits_for_pending_requests(manager: ManagerClient) """Reproducer for #16382""" logger.info('start two nodes') - servers = await manager.servers_add(servers_num=2) + servers = await manager.servers_add(servers_num=2, auto_rack_dc="dc") cql = manager.get_cql() logger.info(f'wait for host for the node {servers[0]}, servers {servers}') h0 = (await wait_for_cql_and_get_hosts(cql, [servers[0]], time.time() + 60))[0] logger.info('create keyspace and table') - async with new_test_keyspace(manager, "with replication = {'class': 'SimpleStrategy', 'replication_factor': 2}") as ks: + async with new_test_keyspace(manager, "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2}") as ks: await cql.run_async(f'create table {ks}.test_table (pk int primary key)') logger.info('insert test row into the table') diff --git a/test/cluster/test_query_rebounce.py b/test/cluster/test_query_rebounce.py index 4125d561c2..2d8381aaba 100644 --- a/test/cluster/test_query_rebounce.py +++ b/test/cluster/test_query_rebounce.py @@ -45,7 +45,7 @@ async def test_query_rebounce(manager: ManagerClient): servers = await manager.running_servers() cql = manager.get_cql() - async with new_test_keyspace(manager, "with replication = {'class': 'SimpleStrategy', 'replication_factor': 1} and tablets = {'enabled': false};") as ks: + async with new_test_keyspace(manager, "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1};") as ks: await cql.run_async(f"create table {ks}.lwt (a int, b int, primary key(a));") await cql.run_async(f"insert into {ks}.lwt (a,b ) values (1, 10);") diff --git a/test/cluster/test_raft_no_quorum.py b/test/cluster/test_raft_no_quorum.py index 96db88bfdb..e0ba398f89 100644 --- a/test/cluster/test_raft_no_quorum.py +++ b/test/cluster/test_raft_no_quorum.py @@ -196,13 +196,13 @@ async def test_cannot_run_operations(manager: ManagerClient, raft_op_timeout: in 'value': '500' } ] - })] + }, property_file={"dc": "dc1", "rack": "rack1"})] logger.info("starting second and third nodes (followers)") - servers += await manager.servers_add(servers_num=2) + servers += await manager.servers_add(servers_num=2, property_file={"dc": "dc1", "rack": "rack2"}) logger.info('create keyspace and table') - ks = await create_new_test_keyspace(manager.get_cql(), "with replication = {'class': 'SimpleStrategy', 'replication_factor': 2}") + ks = await create_new_test_keyspace(manager.get_cql(), "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2}") await manager.get_cql().run_async(f'create table {ks}.test_table (pk int primary key)') logger.info("stopping the second and third nodes") diff --git a/test/cluster/test_raft_snapshot_truncation.py b/test/cluster/test_raft_snapshot_truncation.py index b4dabbac13..8be00e4010 100644 --- a/test/cluster/test_raft_snapshot_truncation.py +++ b/test/cluster/test_raft_snapshot_truncation.py @@ -49,7 +49,7 @@ async def test_raft_snapshot_truncation(manager: ManagerClient): logger.info(f"Log size on {s1}: {log_size}") assert (log_size > 0) - ks = await create_new_test_keyspace(cql, "with replication = {'class': 'SimpleStrategy', 'replication_factor': 1}") + ks = await create_new_test_keyspace(cql, "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") log_size = await get_raft_log_size(cql, h1) logger.info(f"After add keyspace Log size on {s1}: {log_size}") @@ -80,11 +80,11 @@ async def test_raft_snapshot_truncation(manager: ManagerClient): await asyncio.gather(*errs) original_snap_id = await get_raft_snap_id(cql, h1) - + # Create 3 keyspaces. keyspaces = [] for i in range(3): - keyspaces.append(await create_new_test_keyspace(cql, "with replication = {'class': 'SimpleStrategy', 'replication_factor': 1}")) + keyspaces.append(await create_new_test_keyspace(cql, "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}")) # Drop 2 keyspaces. for i in range(2): diff --git a/test/cluster/test_shutdown_hang.py b/test/cluster/test_shutdown_hang.py index 933849244c..da04bc0bf5 100644 --- a/test/cluster/test_shutdown_hang.py +++ b/test/cluster/test_shutdown_hang.py @@ -25,15 +25,15 @@ logger = logging.getLogger(__name__) async def test_hints_manager_shutdown_hang(manager: ManagerClient) -> None: """Reproducer for #8079""" s1 = await manager.server_add(config={ - 'error_injections_at_startup': ['decrease_hints_flush_period'] - }) - s2 = await manager.server_add() + 'error_injections_at_startup': ['decrease_hints_flush_period'], + }, property_file={"dc": "dc1", "rack": "rack1"}) + s2 = await manager.server_add(property_file={"dc": "dc1", "rack": "rack2"}) await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) cql = manager.get_cql() logger.info("Create keyspace and table") - async with new_test_keyspace(manager, "with replication = {'class': 'SimpleStrategy', 'replication_factor': 2}") as ks: + async with new_test_keyspace(manager, "with replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2}") as ks: await cql.run_async(f"create table {ks}.t (pk int primary key)") logger.info(f"Stop {s2}") diff --git a/test/cluster/test_sstable_compression_config.py b/test/cluster/test_sstable_compression_config.py index 18ce010ac6..8edb53171b 100644 --- a/test/cluster/test_sstable_compression_config.py +++ b/test/cluster/test_sstable_compression_config.py @@ -128,7 +128,7 @@ async def test_default_compression_on_upgrade(manager: ManagerClient, scylla_202 logger.info("Creating a test keyspace") cql = manager.get_cql() - await cql.run_async("CREATE KEYSPACE test_ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}") + await cql.run_async("CREATE KEYSPACE test_ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") await create_table_and_check_compression(cql, "test_ks", "table_before_upgrade", "org.apache.cassandra.io.compress.LZ4Compressor", "before upgrade") @@ -252,7 +252,7 @@ async def test_cql_base_tables_respect_compression_config(manager: ManagerClient await wait_for_cql_and_get_hosts(cql, [server], time.time() + 60) ks = f"cql_aux_test_{int(time.time())}" - await cql.run_async(f"CREATE KEYSPACE {ks} WITH replication = {{'class': 'SimpleStrategy', 'replication_factor': 1}}") + await cql.run_async(f"CREATE KEYSPACE {ks} WITH replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}}") await cql.run_async(f"CREATE TABLE {ks}.base (pk int PRIMARY KEY, v int)") try: @@ -294,7 +294,7 @@ async def test_cql_aux_tables_respect_compression_config(manager: ManagerClient) await wait_for_cql_and_get_hosts(cql, [server], time.time() + 60) ks = f"cql_aux_test_{int(time.time())}" - await cql.run_async(f"CREATE KEYSPACE {ks} WITH replication = {{'class': 'SimpleStrategy', 'replication_factor': 1}}") + await cql.run_async(f"CREATE KEYSPACE {ks} WITH replication = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}}") await cql.run_async(f"CREATE TABLE {ks}.base (pk int PRIMARY KEY, v int) WITH compression = {{ 'sstable_compression': 'DeflateCompressor' }}") await cql.run_async(f"CREATE MATERIALIZED VIEW {ks}.mv AS SELECT * FROM {ks}.base WHERE pk IS NOT NULL AND v IS NOT NULL PRIMARY KEY (v, pk)") diff --git a/test/cluster/test_tablets.py b/test/cluster/test_tablets.py index 7db2cba5bd..509843803f 100644 --- a/test/cluster/test_tablets.py +++ b/test/cluster/test_tablets.py @@ -890,46 +890,58 @@ async def test_read_of_pending_replica_during_migration(manager: ManagerClient, # This test checks that --enable-tablets option and the TABLETS parameters of the CQL CREATE KEYSPACE -# statemement are mutually correct from the "the least surprising behavior" concept. See comments inside -# the test code for more details. -@pytest.mark.parametrize("with_tablets", [True, False]) +# statement are mutually correct. +@pytest.mark.parametrize("tablets_mode_for_new_keyspaces", ["enabled", "disabled", "enforced"]) +@pytest.mark.parametrize("cql_tablets_params", ["enabled", "disabled", None]) @pytest.mark.parametrize("replication_strategy", ["NetworkTopologyStrategy", "SimpleStrategy", "EverywhereStrategy", "LocalStrategy"]) @pytest.mark.asyncio -async def test_keyspace_creation_cql_vs_config_sanity(manager: ManagerClient, with_tablets, replication_strategy): - cfg = {'tablets_mode_for_new_keyspaces': 'enabled' if with_tablets else 'disabled'} +async def test_keyspace_creation_cql_vs_config_sanity(manager: ManagerClient, tablets_mode_for_new_keyspaces, cql_tablets_params, replication_strategy): + cfg = {'tablets_mode_for_new_keyspaces': tablets_mode_for_new_keyspaces} server = await manager.server_add(config=cfg) cql = manager.get_cql() - # Tablets are only possible when the replication strategy is NetworkTopology - tablets_possible = (replication_strategy == 'NetworkTopologyStrategy') - tablets_enabled_by_default = tablets_possible and with_tablets + """ + tablets_mode_for_new_keyspaces | enabled | disabled | enforced | + |--------------------|--------------------|--------------------| + AND tablets = {} | enabled | disabled | enabled | disabled | enabled | disabled | + | None | | | None | None | | + -------------------------------|---------|----------|---------|----------|--------------------| + "NetworkTopologyStrategy" | OK | OK | OK | OK | OK | Error | + "SimpleStrategy" | Error | OK | Error | OK | Error | Error | + "EverywhereStrategy" | Error | OK | Error | OK | Error | Error | + "LocalStrategy" | Error | OK | Error | OK | Error | Error | + """ + no_misconfiguration = ( + (replication_strategy == "NetworkTopologyStrategy" and (tablets_mode_for_new_keyspaces != "enforced" or cql_tablets_params != "disabled")) + or (tablets_mode_for_new_keyspaces == "enabled" and cql_tablets_params == "disabled") + or (tablets_mode_for_new_keyspaces == "disabled" and cql_tablets_params != "enabled") + ) + expect_tablets = (replication_strategy == "NetworkTopologyStrategy") and ( + (tablets_mode_for_new_keyspaces == "enforced" and cql_tablets_params != "disabled") + or (tablets_mode_for_new_keyspaces == "enabled" and cql_tablets_params != "disabled") + or (tablets_mode_for_new_keyspaces == "disabled" and cql_tablets_params == "enabled") + ) - # First, check if a kesypace is able to be created with default CQL statement that - # doesn't contain tablets parameters. When possible, tablets should be activated - async with new_test_keyspace(manager, f"WITH replication = {{'class': '{replication_strategy}', 'replication_factor': 1}}") as ks: - res = cql.execute(f"SELECT initial_tablets FROM system_schema.scylla_keyspaces WHERE keyspace_name = '{ks}'").one() - if tablets_enabled_by_default: - assert res.initial_tablets == 0 - else: - assert res is None - - # Next, check that explicit CQL request for enabling tablets can only be satisfied when - # tablets are possible. Tablets must be activated in this case - if tablets_possible: + if no_misconfiguration: expectation = does_not_raise() else: expectation = pytest.raises(ConfigurationException) + with expectation: - ks = await create_new_test_keyspace(cql, f"WITH replication = {{'class': '{replication_strategy}', 'replication_factor': 1}} AND TABLETS = {{'enabled': true}}") - res = cql.execute(f"SELECT initial_tablets FROM system_schema.scylla_keyspaces WHERE keyspace_name = '{ks}'").one() - assert res.initial_tablets == 0 + tablets_opt = "" + if cql_tablets_params == "enabled": + tablets_opt = f"AND TABLETS = {{'enabled': true }}" + if cql_tablets_params == "disabled": + tablets_opt = f"AND TABLETS = {{'enabled': false }}" + + ks = await create_new_test_keyspace(cql, f"WITH replication = {{'class': '{replication_strategy}', 'replication_factor': 1}} {tablets_opt}") + res = cql.execute(f"SELECT initial_tablets FROM system_schema.scylla_keyspaces WHERE keyspace_name = '{ks}'") + if expect_tablets: + assert res.one().initial_tablets == 0 + else: + assert not res await cql.run_async(f"drop keyspace {ks}") - # Finally, check that explicitly disabling tablets in CQL results in vnode-based keyspace - # whenever tablets are enabled or not in config - async with new_test_keyspace(manager, f"WITH replication = {{'class': '{replication_strategy}', 'replication_factor': 1}} AND TABLETS = {{'enabled': false}}") as ks: - res = cql.execute(f"SELECT initial_tablets FROM system_schema.scylla_keyspaces WHERE keyspace_name = '{ks}'").one() - assert res is None @pytest.mark.asyncio @pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode') diff --git a/test/cqlpy/cassandra_tests/validation/operations/create_test.py b/test/cqlpy/cassandra_tests/validation/operations/create_test.py index bfcf7e0445..f9a6fcc5cd 100644 --- a/test/cqlpy/cassandra_tests/validation/operations/create_test.py +++ b/test/cqlpy/cassandra_tests/validation/operations/create_test.py @@ -296,14 +296,14 @@ def testKeyspace(cql): n = unique_name() assertInvalidThrow(cql, n, SyntaxException, "CREATE KEYSPACE %s testXYZ ") - execute(cql, n, "CREATE KEYSPACE %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }") + execute(cql, n, "CREATE KEYSPACE %s WITH replication = { 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1 }") execute(cql, n, "DROP KEYSPACE %s") # The original Cassandra test checked that a 53-character keyspace name doesn't work # but Scylla increased the 48-character limit to 192 characters, so this test was modified # to check length of 500, to produce the same outcome on Scylla and Cassandra. too_long_keyspace_name = "k" * 500 - assertInvalid(cql, "", - f"CREATE KEYSPACE {too_long_keyspace_name} WITH replication = {{ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }}") + assertInvalid(cql, "", + f"CREATE KEYSPACE {too_long_keyspace_name} WITH replication = {{ 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1 }}") # FIXME: Cassandra throws InvalidRequest here, but Scylla uses # ConfigurationException. We shouldn't have done that... But I consider @@ -311,7 +311,7 @@ def testKeyspace(cql): # Maybe we should reconsider, and not allow ConfigurationException... assertInvalidThrow(cql, "", (InvalidRequest, ConfigurationException), "DROP KEYSPACE non_existing") - execute(cql, n, "CREATE KEYSPACE %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }") + execute(cql, n, "CREATE KEYSPACE %s WITH replication = { 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1 }") # clean-up execute(cql, n, "DROP KEYSPACE %s") diff --git a/test/cqlpy/cassandra_tests/validation/operations/insert_update_if_condition_test.py b/test/cqlpy/cassandra_tests/validation/operations/insert_update_if_condition_test.py index fb25a0cea2..0a04473f95 100644 --- a/test/cqlpy/cassandra_tests/validation/operations/insert_update_if_condition_test.py +++ b/test/cqlpy/cassandra_tests/validation/operations/insert_update_if_condition_test.py @@ -302,13 +302,13 @@ def testCasAndListIndex(cql, test_keyspace, is_scylla): def testDropCreateKeyspaceIfNotExists(cql, test_keyspace): keyspace = unique_name() # create and confirm - execute(cql, test_keyspace, "CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} and durable_writes = true ") + execute(cql, test_keyspace, "CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = { 'class':'NetworkTopologyStrategy', 'replication_factor':1} and durable_writes = true ") assertRows(execute(cql, test_keyspace, "select durable_writes from system_schema.keyspaces where keyspace_name = ?", keyspace), row(true)) # unsuccessful create since it's already there, confirm settings don't change - execute(cql, test_keyspace, "CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = {'class':'SimpleStrategy', 'replication_factor':1} and durable_writes = false ") + execute(cql, test_keyspace, "CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = {'class':'NetworkTopologyStrategy', 'replication_factor':1} and durable_writes = false ") assertRows(execute(cql, test_keyspace, "select durable_writes from system_schema.keyspaces where keyspace_name = ?", keyspace), diff --git a/test/cqlpy/nodetool.py b/test/cqlpy/nodetool.py index 86cc87f8d9..5ef9c77713 100644 --- a/test/cqlpy/nodetool.py +++ b/test/cqlpy/nodetool.py @@ -190,6 +190,14 @@ def parse_keyspace_table(name, separator_chars = '.'): m = re.match(pat, name) return m.group('keyspace'), m.group('table') + +def excludenode(cql, excluded_host_id: str): + if has_rest_api(cql): + requests.post(f'{rest_api_url(cql)}/storage_service/exclude_node/', params={'hosts': excluded_host_id}) + else: + run_nodetool(cql, "excludenode", excluded_host_id) + + class no_autocompaction_context: """Disable autocompaction for the enclosed scope, for the provided keyspace(s) or keyspace.table(s). """ diff --git a/test/cqlpy/test_describe.py b/test/cqlpy/test_describe.py index 6aba9aa7b1..816667a5d9 100644 --- a/test/cqlpy/test_describe.py +++ b/test/cqlpy/test_describe.py @@ -17,6 +17,7 @@ from contextlib import ExitStack from .util import new_type, unique_name, new_test_table, new_test_keyspace, new_function, new_aggregate, \ new_cql, keyspace_has_tablets, unique_name_prefix, new_session, new_user, new_materialized_view, \ new_secondary_index +from .conftest import has_tablets from .test_service_levels import MAX_USER_SERVICE_LEVELS from cassandra.protocol import InvalidRequest, Unauthorized from collections.abc import Iterable @@ -1628,7 +1629,8 @@ def new_random_keyspace(cql): options["replication_factor"] = random.randrange(1, 6) options_str = ", ".join([f"'{k}': '{v}'" for (k, v) in options.items()]) extra = "" - if options["class"] == "NetworkTopologyStrategy" and options["replication_factor"] != 1: + # Cassandra does not have tablets and thus does not even support tablets syntax. + if not has_tablets or options["class"] == "SimpleStrategy" or options["replication_factor"] != 1: extra = " and tablets = { 'enabled': false }" write = random.choice(writes) diff --git a/test/cqlpy/test_guardrail_replication_strategy.py b/test/cqlpy/test_guardrail_replication_strategy.py index 7277e94402..3e4a61c21d 100644 --- a/test/cqlpy/test_guardrail_replication_strategy.py +++ b/test/cqlpy/test_guardrail_replication_strategy.py @@ -2,6 +2,7 @@ import pytest from contextlib import ExitStack import re from .util import unique_name, config_value_context, new_test_keyspace, ScyllaMetrics +from .conftest import has_tablets from cassandra.protocol import ConfigurationException # Tests for the replication_strategy_{warn,fail}_list guardrail. Because @@ -37,6 +38,11 @@ def ks_opts(strategy, rf, dc=None, tablets=True): return opts +def get_replication_strategy_ks_opts(strategy: str, rf: int, dc=None) -> str: + # If tablets syntax is not supported (e.g. in cassandra) we don't add it to the ks options + return ks_opts(strategy, rf, dc=dc, tablets=has_tablets and strategy == 'NetworkTopologyStrategy') + + def create_ks_and_assert_warnings_and_errors(cql, ks_opts, metric_name=None, warnings=[], failures=[]): before = get_metric(cql, metric_name) if metric_name else None @@ -63,13 +69,13 @@ def create_ks_and_assert_warnings_and_errors(cql, ks_opts, metric_name=None, def test_given_default_config_when_creating_ks_should_only_produce_warning_for_simple_strategy(cql, this_dc): - create_ks_and_assert_warnings_and_errors(cql, ks_opts('SimpleStrategy', 3), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts('SimpleStrategy', 3), metric_name='scylla_cql_replication_strategy_warn_list_violations', warnings=[STRATEGY_WARN_RE.format(strategy='SimpleStrategy')]) for strategy, dc in {'NetworkTopologyStrategy': this_dc, 'EverywhereStrategy': 'replication_factor', 'LocalStrategy': 'replication_factor'}.items(): - create_ks_and_assert_warnings_and_errors(cql, ks_opts(strategy, 1, dc=dc), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts(strategy, 1, dc=dc), warnings=[MINIMUM_RF_WARN_RE.format(dc=re.escape(dc), rf=1, threshold=3)]) @@ -80,7 +86,7 @@ def test_given_cleared_guardrails_when_creating_ks_should_not_get_warning_nor_er for strategy, dc in {'SimpleStrategy': 'replication_factor', 'NetworkTopologyStrategy': this_dc, 'EverywhereStrategy': 'replication_factor', 'LocalStrategy': 'replication_factor'}.items(): - create_ks_and_assert_warnings_and_errors(cql, ks_opts(strategy, 1, dc=dc), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts(strategy, 1, dc=dc), warnings=[MINIMUM_RF_WARN_RE.format(dc=re.escape(dc), rf=1, threshold=3)]) @@ -90,7 +96,7 @@ def test_given_non_empty_warn_list_when_creating_ks_should_only_warn_when_listed 'SimpleStrategy,LocalStrategy,NetworkTopologyStrategy,EverywhereStrategy')) for strategy, dc in {'SimpleStrategy': 'replication_factor', 'NetworkTopologyStrategy': this_dc, 'EverywhereStrategy': 'replication_factor', 'LocalStrategy': 'replication_factor'}.items(): - create_ks_and_assert_warnings_and_errors(cql, ks_opts(strategy, 1, dc=dc), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts(strategy, 1, dc=dc), warnings=[STRATEGY_WARN_RE.format(strategy=strategy), MINIMUM_RF_WARN_RE.format(dc=re.escape(dc), rf=1, threshold=3)]) @@ -106,11 +112,11 @@ def test_given_non_empty_warn_and_fail_lists_when_creating_ks_should_fail_query_ 'EverywhereStrategy': 'replication_factor', 'LocalStrategy': 'replication_factor'}.items(): # note: even though warn list is not empty, no warnings should be generated, because failures come first - # we don't want to issue a warning and also fail the query at the same time - create_ks_and_assert_warnings_and_errors(cql, ks_opts(strategy, 1, dc=dc), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts(strategy, 1, dc=dc), failures=[STRATEGY_FAIL_RE.format(strategy=strategy)]) # Verify metric increment and exact error message (docs/cql/guardrails.rst). - create_ks_and_assert_warnings_and_errors(cql, ks_opts('SimpleStrategy', 3), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts('SimpleStrategy', 3), metric_name='scylla_cql_replication_strategy_fail_list_violations', failures=[STRATEGY_FAIL_RE.format(strategy='SimpleStrategy')]) @@ -149,7 +155,7 @@ def test_given_rf_and_strategy_guardrails_when_creating_ks_should_print_2_warnin with ExitStack() as config_modifications: config_modifications.enter_context(config_value_context(cql, 'replication_strategy_warn_list', 'SimpleStrategy')) config_modifications.enter_context(config_value_context(cql, 'minimum_replication_factor_warn_threshold', '3')) - create_ks_and_assert_warnings_and_errors(cql, ks_opts('SimpleStrategy', 1), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts('SimpleStrategy', 1), metric_name='scylla_cql_minimum_replication_factor_warn_violations', warnings=[MINIMUM_RF_WARN_RE.format( dc='replication_factor', rf=1, threshold=3), @@ -160,7 +166,7 @@ def test_given_rf_and_strategy_guardrails_when_violating_fail_rf_limit_and_warn_ with ExitStack() as config_modifications: config_modifications.enter_context(config_value_context(cql, 'replication_strategy_warn_list', 'SimpleStrategy')) config_modifications.enter_context(config_value_context(cql, 'minimum_replication_factor_fail_threshold', '3')) - create_ks_and_assert_warnings_and_errors(cql, ks_opts('SimpleStrategy', 1), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts('SimpleStrategy', 1), metric_name='scylla_cql_minimum_replication_factor_fail_violations', failures=[MINIMUM_RF_FAIL_RE.format( dc='replication_factor', rf=1, threshold=3)]) @@ -170,19 +176,20 @@ def test_given_rf_and_strategy_guardrails_when_violating_fail_strategy_limit_sho with ExitStack() as config_modifications: config_modifications.enter_context(config_value_context(cql, 'replication_strategy_fail_list', 'SimpleStrategy')) config_modifications.enter_context(config_value_context(cql, 'minimum_replication_factor_fail_threshold', '3')) - create_ks_and_assert_warnings_and_errors(cql, ks_opts('SimpleStrategy', 1), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts('SimpleStrategy', 1), failures=[STRATEGY_FAIL_RE.format(strategy='SimpleStrategy')]) def test_given_restrict_replication_simplestrategy_when_it_is_set_should_emulate_old_behavior(cql): with ExitStack() as config_modifications: config_modifications.enter_context(config_value_context(cql, 'restrict_replication_simplestrategy', 'true')) - create_ks_and_assert_warnings_and_errors(cql, ks_opts('SimpleStrategy', 3), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts('SimpleStrategy', 3), failures=[STRATEGY_FAIL_RE.format(strategy='SimpleStrategy')]) config_modifications.enter_context(config_value_context(cql, 'restrict_replication_simplestrategy', 'warn')) - create_ks_and_assert_warnings_and_errors(cql, ks_opts('SimpleStrategy', 3), + create_ks_and_assert_warnings_and_errors(cql, get_replication_strategy_ks_opts('SimpleStrategy', 3), warnings=[STRATEGY_WARN_RE.format(strategy='SimpleStrategy')]) + def test_config_replication_strategy_warn_list_roundtrips_quotes(cql): # Use direct SELECT/UPDATE to avoid trippy config_value_context behavior value = cql.execute("SELECT value FROM system.config WHERE name = 'replication_strategy_warn_list'").one().value diff --git a/test/cqlpy/test_keyspace.py b/test/cqlpy/test_keyspace.py index 1f57b042ec..0c397e4ea1 100644 --- a/test/cqlpy/test_keyspace.py +++ b/test/cqlpy/test_keyspace.py @@ -6,6 +6,7 @@ # ALTER KEYSPACE from .util import new_test_keyspace, unique_name +from .conftest import has_tablets import pytest from cassandra.protocol import SyntaxException, AlreadyExists, InvalidRequest, ConfigurationException from threading import Thread @@ -70,8 +71,10 @@ def test_create_keyspace_if_not_exists(cql, this_dc): # A second invocation with IF NOT EXISTS is fine: cql.execute("CREATE KEYSPACE IF NOT EXISTS test_create_keyspace_if_not_exists WITH REPLICATION = { 'class' : 'NetworkTopologyStrategy', '" + this_dc + "' : 1 }") # It doesn't matter if the second invocation has different parameters, - # they are ignored. - cql.execute("CREATE KEYSPACE IF NOT EXISTS test_create_keyspace_if_not_exists WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 2 }") + # they are ignored. Disable tablets with SimpleStrategy with the tablets + # syntax is supported + tablets_opts = " AND TABLETS = { 'enabled': false }" if has_tablets else "" + cql.execute("CREATE KEYSPACE IF NOT EXISTS test_create_keyspace_if_not_exists WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 2 }" + tablets_opts) cql.execute("DROP KEYSPACE test_create_keyspace_if_not_exists") # We treat ALTER to numeric RF of same count as no-op. diff --git a/test/cqlpy/test_name.py b/test/cqlpy/test_name.py index 7a08a24724..1ea04218e8 100644 --- a/test/cqlpy/test_name.py +++ b/test/cqlpy/test_name.py @@ -38,7 +38,7 @@ def padded_name(length): # Created to avoid passing the same replication option in every tests. @contextmanager def new_keyspace(cql, ks_name=unique_name()): - cql.execute(f"CREATE KEYSPACE {ks_name} WITH REPLICATION = {{'class': 'SimpleStrategy', 'replication_factor': 1}}") + cql.execute(f"CREATE KEYSPACE {ks_name} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}}") try: yield ks_name finally: diff --git a/test/cqlpy/test_permissions.py b/test/cqlpy/test_permissions.py index e4ec53f429..e70f5154b2 100644 --- a/test/cqlpy/test_permissions.py +++ b/test/cqlpy/test_permissions.py @@ -333,7 +333,7 @@ def test_grant_revoke_udf_permissions(cql): # and yet it's not enforced def test_grant_revoke_alter_udf_permissions(cassandra_bug, cql): schema = "a int primary key" - with new_test_keyspace(cql, "WITH REPLICATION = { 'class': 'SimpleStrategy', 'replication_factor': 1 }") as keyspace: + with new_test_keyspace(cql, "WITH REPLICATION = { 'class': 'NetworkTopologyStrategy', 'replication_factor': 1 }") as keyspace: with new_test_table(cql, keyspace, schema) as table: fun_body_lua = "(i int) CALLED ON NULL INPUT RETURNS int LANGUAGE lua AS 'return 42;'" fun_body_java = "(i int) CALLED ON NULL INPUT RETURNS int LANGUAGE java AS 'return 42;'" @@ -358,7 +358,7 @@ def test_grant_revoke_alter_udf_permissions(cassandra_bug, cql): check_enforced(cql, username, permission='ALTER', resource=f'all functions in keyspace {keyspace}', function=lambda: user_session.execute(f"CREATE OR REPLACE FUNCTION {keyspace}.{fun} {fun_body}")) check_enforced(cql, username, permission='ALTER', resource='all functions', - function=lambda: user_session.execute(f"CREATE OR REPLACE FUNCTION {keyspace}.{fun} {fun_body}")) + function=lambda: user_session.execute(f"CREATE OR REPLACE FUNCTION {keyspace}.{fun} {fun_body}")) check_enforced(cql, username, permission='ALTER', resource=f'FUNCTION {keyspace}.{fun}(int)', function=lambda: user_session.execute(f"CREATE OR REPLACE FUNCTION {keyspace}.{fun} {fun_body}")) diff --git a/test/cqlpy/test_tablets.py b/test/cqlpy/test_tablets.py index f7f47ec06a..eaeba18317 100644 --- a/test/cqlpy/test_tablets.py +++ b/test/cqlpy/test_tablets.py @@ -57,7 +57,7 @@ def test_alter_cannot_change_vnodes_to_tablets(cql, skip_without_tablets): # Converting vnodes-based keyspace to tablets-based in not implemented yet def test_alter_vnodes_ks_doesnt_enable_tablets(cql, skip_without_tablets): - ksdef = "WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};" + ksdef = "WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1} AND tablets = {'enabled': false};" with new_test_keyspace(cql, ksdef) as keyspace: cql.execute(f"ALTER KEYSPACE {keyspace} WITH replication = {{'class': 'NetworkTopologyStrategy'}};")