mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
update ClusterDiscovery::ConcurrentFlags::set
This commit is contained in:
parent
a76e93209d
commit
3934e99dce
@ -60,20 +60,17 @@ public:
|
||||
|
||||
void set(const T & key)
|
||||
{
|
||||
setFlag(key, true);
|
||||
cv.notify_one();
|
||||
}
|
||||
|
||||
/// `need_update` expected to be value from `flags` corresponding to some key
|
||||
void set(std::atomic_bool & need_update)
|
||||
{
|
||||
auto it = flags.find(key);
|
||||
if (it == flags.end())
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key);
|
||||
it->second = true;
|
||||
any_need_update = true;
|
||||
need_update = true;
|
||||
cv.notify_one();
|
||||
}
|
||||
|
||||
/// waits unit at least one flag is set
|
||||
/// caller should handle all set flags (or set it again manually)
|
||||
/// note: keys of returen map should not be changed!
|
||||
std::unordered_map<T, std::atomic_bool> & wait(std::atomic_bool & finished, std::chrono::milliseconds timeout)
|
||||
{
|
||||
std::unique_lock<std::mutex> lk(mu);
|
||||
@ -89,16 +86,6 @@ public:
|
||||
void notify() { cv.notify_one(); }
|
||||
|
||||
private:
|
||||
void setFlag(const T & key, bool value)
|
||||
{
|
||||
any_need_update = any_need_update || value;
|
||||
|
||||
auto it = flags.find(key);
|
||||
if (it == flags.end())
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key);
|
||||
it->second = value;
|
||||
}
|
||||
|
||||
std::condition_variable cv;
|
||||
std::mutex mu;
|
||||
|
||||
@ -395,7 +382,8 @@ bool ClusterDiscovery::runMainThread()
|
||||
}
|
||||
else
|
||||
{
|
||||
clusters_to_update->set(need_update);
|
||||
/// no need to trigger convar, will retry after timeout in `wait`
|
||||
need_update = true;
|
||||
LOG_WARNING(log, "Cluster '{}' wasn't updated, will retry", cluster_name);
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user