repair: Allow min max range to be updated for repair history
It is observed that: repair - repair[667d4a59-63fb-4ca6-8feb-98da49946d8b]: Failed to update system.repair_history table of node d27de212-6f32-4649ad76-a9ef1165fdcb: seastar::rpc::remote_verb_error (repair[667d4a59-63fb-4ca6-8feb-98da49946d8b]: range (minimum token,maximum token) is not in the format of (start, end]) This is because repair checks the end of the range to be repaired needs to be inclusive. When small_table_optimization is enabled for regular repair, a (minimum token,maximum token) will be used. To fix, we can relax the check of (start, end] for the min max range. Fixes #27220 Closes scylladb/scylladb#27357
This commit is contained in:
@@ -2529,7 +2529,7 @@ future<repair_update_system_table_response> repair_service::repair_update_system
|
||||
}
|
||||
}
|
||||
if (req.range.end()) {
|
||||
if (!req.range.end()->is_inclusive()) {
|
||||
if (!req.range.end()->is_inclusive() && req.range.end()->value() != dht::maximum_token()) {
|
||||
is_valid_range = false;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -339,3 +339,17 @@ async def test_repair_timtestamp_difference(manager):
|
||||
|
||||
logger.info("Checking timestamps after repair")
|
||||
check({host1: update2_timestamp, host2: update2_timestamp})
|
||||
|
||||
@pytest.mark.asyncio
|
||||
async def test_small_table_optimization_repair(manager):
|
||||
servers = await manager.servers_add(2, auto_rack_dc="dc1")
|
||||
|
||||
cql = manager.get_cql()
|
||||
|
||||
cql.execute("CREATE KEYSPACE ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2} AND TABLETS = {'enabled': false}")
|
||||
cql.execute("CREATE TABLE ks.tbl (pk int, ck int, PRIMARY KEY (pk, ck)) WITH tombstone_gc = {'mode': 'repair'}")
|
||||
|
||||
await manager.api.repair(servers[0].ip_addr, "ks", "tbl", small_table_optimization=True)
|
||||
|
||||
rows = await cql.run_async(f"SELECT * from system.repair_history")
|
||||
assert len(rows) == 1
|
||||
|
||||
@@ -457,7 +457,7 @@ class ScyllaRESTAPIClient:
|
||||
data = await self.client.get_json("/raft/leader_host", host=node_ip, params=params)
|
||||
return HostID(data)
|
||||
|
||||
async def repair(self, node_ip: str, keyspace: str, table: str, ranges: str = '') -> None:
|
||||
async def repair(self, node_ip: str, keyspace: str, table: str, ranges: str = '', small_table_optimization: bool = False) -> None:
|
||||
"""Repair the given table and wait for it to complete"""
|
||||
vnode_keyspaces = await self.client.get_json(f"/storage_service/keyspaces", host=node_ip, params={"replication": "vnodes"})
|
||||
if keyspace in vnode_keyspaces:
|
||||
@@ -465,6 +465,8 @@ class ScyllaRESTAPIClient:
|
||||
params = {"columnFamilies": table, "ranges": ranges}
|
||||
else:
|
||||
params = {"columnFamilies": table}
|
||||
if small_table_optimization:
|
||||
params["small_table_optimization"] = "true"
|
||||
sequence_number = await self.client.post_json(f"/storage_service/repair_async/{keyspace}", host=node_ip, params=params)
|
||||
status = await self.client.get_json(f"/storage_service/repair_status", host=node_ip, params={"id": str(sequence_number)})
|
||||
if status != 'SUCCESSFUL':
|
||||
|
||||
Reference in New Issue
Block a user