mirror of
https://github.com/scylladb/scylladb.git
synced 2026-05-02 22:25:48 +00:00
Merge 'Make node replace procedure work with Raft' from Kamil Braun
We need to obtain the Raft ID of the replaced node during the shadow round and place it in the address map. It won't be placed by the regular gossiping route if we're replacing using the same IP, because we override the application state of the replaced node. Even if we replace a node with a different IP, it is not guaranteed that background gossiping manages update the address map before we need it, especially in tests where we set ring_delay to 0 and disable wait_for_gossip_to_settle. The shadow round, on the other hand, performs a synchronous request (and if it fails during bootstrap, bootstrap will fail - because we also won't be able to obtain the tokens and Host ID of the replaced node). Fetch the Raft ID of the replaced node in `prepare_replacement_info`, which runs the shadow round. Return it in `replacement_info`. Then `join_token_ring` passes it to `setup_group0`, which stores it in the address map. It does that after `join_group0` so the entry is non-expiring (the replaced node is a member of group 0). Later in the replace procedure, we call `remove_from_group0` for the replaced node. `remove_from_group0` will be able to reverse-translate the IP of the replaced node to its Raft ID using the address map. Also remove an unconditional 60 seconds sleep from the replace code. Make it dependent on ring_delay. Enable the replace tests. Modify some code related to removing servers from group 0 which depended on storing IP addresses in the group 0 configuration. Closes #12172 * github.com:scylladb/scylladb: test/topology: enable replace tests service/raft: report an error when Raft ID can't be found in `raft_group0::remove_from_group0` service: handle replace correctly with Raft enabled gms/gossiper: fetch RAFT_SERVER_ID during shadow round service: storage_service: sleep 2*ring_delay instead of BROADCAST_INTERVAL before replace
This commit is contained in:
@@ -60,6 +60,7 @@ async def test_remove_node_add_column(manager, random_tables):
|
||||
await manager.remove_node(servers[0].server_id, servers[1].server_id) # Remove [1]
|
||||
await table.add_column()
|
||||
await random_tables.verify_schema()
|
||||
# TODO: check that group 0 no longer contains the removed node (#12153)
|
||||
|
||||
|
||||
@pytest.mark.asyncio
|
||||
@@ -92,24 +93,23 @@ async def test_decommission_node_add_column(manager, random_tables):
|
||||
await manager.decommission_node(decommission_target.server_id)
|
||||
await table.add_column()
|
||||
await random_tables.verify_schema()
|
||||
# TODO: check that group 0 no longer contains the decommissioned node (#12153)
|
||||
|
||||
@pytest.mark.asyncio
|
||||
@pytest.mark.skip(reason="Replace operation sleeps for 60 seconds")
|
||||
async def test_replace_different_ip(manager: ManagerClient, random_tables) -> None:
|
||||
servers = await manager.running_servers()
|
||||
await manager.server_stop(servers[0].server_id)
|
||||
replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False)
|
||||
await manager.server_add(replace_cfg)
|
||||
# TODO: check that group 0 no longer contains the replaced node
|
||||
# TODO: check that group 0 no longer contains the replaced node (#12153)
|
||||
|
||||
@pytest.mark.asyncio
|
||||
@pytest.mark.skip(reason="As above + the new node cannot join group 0")
|
||||
async def replace_reuse_ip(manager: ManagerClient, random_tables) -> None:
|
||||
async def test_replace_reuse_ip(manager: ManagerClient, random_tables) -> None:
|
||||
servers = await manager.running_servers()
|
||||
await manager.server_stop(servers[0].server_id)
|
||||
replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = True)
|
||||
await manager.server_add(replace_cfg)
|
||||
# TODO: check that group 0 no longer contains the replaced node
|
||||
# TODO: check that group 0 no longer contains the replaced node (#12153)
|
||||
|
||||
|
||||
@pytest.mark.asyncio
|
||||
@@ -162,6 +162,7 @@ async def test_remove_node_with_concurrent_ddl(manager, random_tables):
|
||||
await manager.wait_for_host_down(initiator_ip, target_ip)
|
||||
logger.info(f'do_remove_node [{i}], invoking remove_node')
|
||||
await manager.remove_node(initiator_ip, target_ip, target_host_id)
|
||||
# TODO: check that group 0 no longer contains the removed node (#12153)
|
||||
logger.info(f'do_remove_node [{i}], remove_node done')
|
||||
new_server_ip = await manager.server_add()
|
||||
logger.info(f'do_remove_node [{i}], server_add [{new_server_ip}] done')
|
||||
|
||||
Reference in New Issue
Block a user