Skip to content

Commit b91dbd1

Browse files
Merge pull request ClickHouse#87751 from ClickHouse/backport/25.8/78873
Backport ClickHouse#78873 to 25.8: Fix RMV tryEnqueueReplicatedDDL shutdown crash again
2 parents a6ee72a + 7e68db4 commit b91dbd1

File tree

4 files changed

+10
-5
lines changed

4 files changed

+10
-5
lines changed

programs/server/Server.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2820,11 +2820,11 @@ try
28202820
else
28212821
LOG_INFO(log, "Closed connections.");
28222822

2823-
global_context->getRefreshSet().joinBackgroundTasks(wait_start + std::chrono::milliseconds(wait_limit_seconds * 1000));
2823+
bool joined_refresh_tasks = global_context->getRefreshSet().joinBackgroundTasks(wait_start + std::chrono::milliseconds(wait_limit_seconds * 1000));
28242824

28252825
dns_cache_updater.reset();
28262826

2827-
if (current_connections)
2827+
if (current_connections || !joined_refresh_tasks)
28282828
{
28292829
/// There is no better way to force connections to close in Poco.
28302830
/// Otherwise connection handlers will continue to live

src/Storages/MaterializedView/RefreshSet.cpp

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ bool RefreshSet::refreshesStopped() const
203203
return refreshes_stopped.load();
204204
}
205205

206-
void RefreshSet::joinBackgroundTasks(std::chrono::steady_clock::time_point deadline)
206+
bool RefreshSet::joinBackgroundTasks(std::chrono::steady_clock::time_point deadline)
207207
{
208208
std::vector<RefreshTaskPtr> remaining_tasks;
209209
std::chrono::steady_clock::time_point stopped_at;
@@ -234,7 +234,11 @@ void RefreshSet::joinBackgroundTasks(std::chrono::steady_clock::time_point deadl
234234
names += remaining_tasks[i]->getInfo().view_id.getNameForLogs();
235235
}
236236
LOG_ERROR(getLogger("RefreshSet"), "{} view refreshes failed to stop in {:.3}s: {}", remaining_tasks.size(), elapsed_seconds, names);
237+
238+
return false;
237239
}
240+
241+
return true;
238242
}
239243

240244
RefreshSet::Handle::Handle(RefreshSet * parent_set_, StorageID id_, std::optional<StorageID> inner_table_id_, RefreshTaskList::iterator iter_, RefreshTaskList::iterator inner_table_iter_, std::vector<StorageID> dependencies_)

src/Storages/MaterializedView/RefreshSet.h

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -68,7 +68,8 @@ class RefreshSet
6868
bool refreshesStopped() const;
6969

7070
/// Called during shutdown, after setRefreshesStopped(true).
71-
void joinBackgroundTasks(std::chrono::steady_clock::time_point deadline);
71+
/// Returns false if some tasks are still running after deadline.
72+
bool joinBackgroundTasks(std::chrono::steady_clock::time_point deadline);
7273

7374
private:
7475
using TaskMap = std::unordered_map<StorageID, RefreshTaskList, StorageID::DatabaseAndTableNameHash, StorageID::DatabaseAndTableNameEqual>;

src/Storages/MaterializedView/RefreshTask.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1047,7 +1047,7 @@ void RefreshTask::interruptExecution()
10471047
if (execution.executor)
10481048
{
10491049
execution.executor->cancel();
1050-
LOG_DEBUG(log, "Cancelling refresh");
1050+
LOG_DEBUG(log, "Cancelling refresh in {}", set_handle.getID().getFullNameNotQuoted());
10511051
}
10521052
}
10531053

0 commit comments

Comments
 (0)