Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix snapshot never clear #323

Merged
merged 3 commits into from
Jun 21, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
22 changes: 17 additions & 5 deletions src/Service/NuRaftLogSnapshot.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -917,10 +917,15 @@ size_t KeeperSnapshotManager::removeSnapshots()
{
Int64 remove_count = static_cast<Int64>(snapshots.size()) - static_cast<Int64>(keep_max_snapshot_count);

LOG_INFO(log, "There are {} snapshots, we will try to move {remove_count} of them", snapshots.size(), remove_count);

while (remove_count > 0)
{
auto it = snapshots.begin();
ulong remove_log_index = it->first;
uint128_t remove_term_log_index = it->first;
auto [log_term, log_index] = getTermLogFromSnapshotStoreMapKey(remove_term_log_index);
LOG_INFO(log, "Remove snapshot with term {} log index {}", log_term, log_index);

Poco::File dir_obj(snap_dir);
if (dir_obj.exists())
{
Expand All @@ -939,17 +944,19 @@ size_t KeeperSnapshotManager::removeSnapshots()
LOG_ERROR(log, "Can't parse object info from file name {}", file);
continue;
}
if (remove_log_index == s_obj.log_last_index)
auto key = getSnapshotStoreMapKey(s_obj);
if (remove_term_log_index == key)
{
LOG_INFO(
log,
"remove_count {}, snapshot size {}, remove log index {}, file {}",
"remove_count {}, snapshot size {}, remove term with term {} log index {}, file {}",
remove_count,
snapshots.size(),
remove_log_index,
log_term,
log_index,
file);
Poco::File(snap_dir + "/" + file).remove();
if (snapshots.find(remove_log_index) != snapshots.end())
if (snapshots.find(key) != snapshots.end())
{
snapshots.erase(it);
}
Expand All @@ -958,6 +965,11 @@ size_t KeeperSnapshotManager::removeSnapshots()
}
remove_count--;
}

if (snapshots.size() > keep_max_snapshot_count)
LOG_ERROR(log, "Snapshots size() is still large than keep_max_snapshot_count {}, it's a bug",
snapshots.size(), keep_max_snapshot_count);

return snapshots.size();
}

Expand Down
5 changes: 5 additions & 0 deletions src/Service/NuRaftLogSnapshot.h
Original file line number Diff line number Diff line change
Expand Up @@ -285,6 +285,11 @@ inline uint128_t getSnapshotStoreMapKey(const SnapObject & obj)
return getSnapshotStoreMapKeyImpl(obj.log_last_term, obj.log_last_index);
}

inline std::pair<uint64_t, uint64_t> getTermLogFromSnapshotStoreMapKey(uint128_t & key)
{
return {static_cast<uint64_t>(key >> 64), static_cast<uint64_t>(key & 0xFFFFFFFFFFFFFFFF)};
}

// Map key is term << 64 | log
using KeeperSnapshotStoreMap = std::map<uint128_t, ptr<KeeperSnapshotStore>>;

Expand Down
4 changes: 4 additions & 0 deletions tests/integration/helpers/cluster_service.py
Original file line number Diff line number Diff line change
Expand Up @@ -777,6 +777,10 @@ def file_exists(self, path):
return self.exec_in_container(
["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n'

def list_path(self, path):
return self.exec_in_container(
["bash", "-c", "ls {}".format(path)])

def copy_file_to_container(self, local_path, dest_path):
container_id = self.get_docker_handle().id
return self.cluster.copy_file_to_container(container_id, local_path, dest_path)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,8 +2,8 @@
<keeper>
<my_id>1</my_id>
<host>node</host>
<log_dir>./raft_log</log_dir>
<snapshot_dir>./snapshot_log</snapshot_dir>
<log_dir>/raft_log</log_dir>
<snapshot_dir>/snapshot</snapshot_dir>
<snapshot_create_interval>86400</snapshot_create_interval>
<forwarding_port>8102</forwarding_port>
<port>8101</port>
Expand All @@ -15,6 +15,7 @@
<max_session_timeout_ms>80000</max_session_timeout_ms>
<operation_timeout_ms>1000</operation_timeout_ms>
<snapshot_distance>10</snapshot_distance>
<max_stored_snapshots>1</max_stored_snapshots>
<election_timeout_lower_bound_ms>1000</election_timeout_lower_bound_ms>
<election_timeout_upper_bound_ms>2000</election_timeout_upper_bound_ms>
<min_session_timeout_ms>1000</min_session_timeout_ms>
Expand Down
5 changes: 3 additions & 2 deletions tests/integration/test_snapshots/configs/enable_keeper.xml
Original file line number Diff line number Diff line change
Expand Up @@ -2,8 +2,8 @@
<keeper>
<my_id>1</my_id>
<host>node</host>
<log_dir>./raft_log</log_dir>
<snapshot_dir>./snapshot_log</snapshot_dir>
<log_dir>/raft_log</log_dir>
<snapshot_dir>/snapshot</snapshot_dir>
<snapshot_create_interval>86400</snapshot_create_interval>
<forwarding_port>8102</forwarding_port>
<port>8101</port>
Expand All @@ -15,6 +15,7 @@
<max_session_timeout_ms>80000</max_session_timeout_ms>
<operation_timeout_ms>1000</operation_timeout_ms>
<snapshot_distance>10</snapshot_distance>
<max_stored_snapshots>1</max_stored_snapshots>
<election_timeout_lower_bound_ms>1000</election_timeout_lower_bound_ms>
<election_timeout_upper_bound_ms>2000</election_timeout_upper_bound_ms>
<min_session_timeout_ms>1000</min_session_timeout_ms>
Expand Down
38 changes: 38 additions & 0 deletions tests/integration/test_snapshots/test.py
Original file line number Diff line number Diff line change
Expand Up @@ -142,3 +142,41 @@ def test_restart_with_no_log(started_cluster, node):
assert node_zk2.exists("/test_restart_with_no_log")
finally:
close_zk_clients([node_zk, node_zk2])


def get_snapshots(node):
files = node.list_path('/snapshot').split()

snapshots = dict()
for file_name in files:
print(file_name)
file_name_splits = file_name.split('_')
assert(len(file_name_splits) == 5)
key = '_'.join(file_name_splits[:4])
value = snapshots.get(key, [])
value.append(file_name_splits[4])
snapshots[key] = value
return snapshots


@pytest.mark.parametrize(
'node',
[
cluster.add_instance('node7', main_configs=['configs/enable_keeper.xml'], with_zookeeper=True, stay_alive=True),
cluster.add_instance('node8', main_configs=['configs/enable_async_snapshot_keeper.xml'], with_zookeeper=True, stay_alive=True)
]
)
def test_snapshot_clear(started_cluster, node):
node_zk = node_zk2 = None
try:
node_zk = node.get_fake_zk()

for i in range(5):
node_zk.create(f"/test_node_clear_{i}", b"test")
node.send_4lw_cmd(cmd="csnp")
# wait for snapshot to be taken
time.sleep(1)
snapshots = get_snapshots(node)
assert (len(snapshots) == 1)
finally:
close_zk_clients([node_zk, node_zk2])
Loading