2021-11-16 12:01:30 +00:00
|
|
|
#include <algorithm>
|
|
|
|
#include <chrono>
|
|
|
|
#include <unordered_set>
|
|
|
|
|
2021-11-15 14:52:52 +00:00
|
|
|
#include <base/getFQDNOrHostName.h>
|
|
|
|
#include <base/logger_useful.h>
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
#include <Common/DNSResolver.h>
|
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
|
|
|
#include <Common/ZooKeeper/Types.h>
|
2021-11-15 14:52:52 +00:00
|
|
|
#include <Common/setThreadName.h>
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
#include <Core/ServerUUID.h>
|
|
|
|
|
|
|
|
#include <Interpreters/Cluster.h>
|
|
|
|
#include <Interpreters/ClusterDiscovery.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2021-11-15 14:52:52 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
constexpr size_t MAX_QUEUE_SIZE = 16;
|
|
|
|
constexpr UInt64 QUEUE_OP_TIMEOUT_MS = 1000;
|
|
|
|
|
|
|
|
fs::path getReplicasListPath(const String & zk_root)
|
|
|
|
{
|
|
|
|
return fs::path(zk_root + "/replicas");
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
ClusterDiscovery::ClusterDiscovery(
|
|
|
|
const Poco::Util::AbstractConfiguration & config,
|
|
|
|
ContextMutablePtr context_,
|
|
|
|
const String & config_prefix)
|
|
|
|
: context(context_)
|
|
|
|
, node_name(toString(ServerUUID::get()))
|
|
|
|
, server_port(context->getTCPPort())
|
2021-11-15 14:52:52 +00:00
|
|
|
, queue(std::make_shared<UpdateQueue>(MAX_QUEUE_SIZE))
|
2021-11-11 09:03:53 +00:00
|
|
|
, log(&Poco::Logger::get("ClusterDiscovery"))
|
|
|
|
{
|
|
|
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
|
|
|
config.keys(config_prefix, config_keys);
|
|
|
|
|
|
|
|
for (const auto & key : config_keys)
|
|
|
|
{
|
|
|
|
String path = config.getString(config_prefix + "." + key + ".path");
|
|
|
|
trimRight(path, '/');
|
2021-11-16 12:01:30 +00:00
|
|
|
clusters_info.emplace(key, ClusterInfo(key, path));
|
2021-11-11 09:03:53 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
/// List node in zookeper for cluster
|
2021-11-16 09:02:44 +00:00
|
|
|
Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk,
|
|
|
|
const String & zk_root,
|
|
|
|
const String & cluster_name,
|
|
|
|
int * version,
|
|
|
|
bool set_callback)
|
|
|
|
{
|
|
|
|
auto watch_callback = [cluster_name, queue=queue, log=log](const Coordination::WatchResponse &)
|
|
|
|
{
|
|
|
|
if (!queue->tryPush(cluster_name, QUEUE_OP_TIMEOUT_MS))
|
|
|
|
{
|
|
|
|
if (queue->isFinished())
|
|
|
|
return;
|
|
|
|
LOG_WARNING(log, "Cannot push update request for cluster '{}'", cluster_name);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
Coordination::Stat stat;
|
|
|
|
Strings nodes = zk->getChildrenWatch(getReplicasListPath(zk_root), &stat, set_callback ? watch_callback : Coordination::WatchCallback{});
|
|
|
|
if (version)
|
|
|
|
*version = stat.cversion;
|
|
|
|
return nodes;
|
|
|
|
}
|
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
/// Reads node information from scpecified zookeper nodes
|
|
|
|
ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes)
|
2021-11-11 09:03:53 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
NodesInfo result;
|
|
|
|
for (const auto & node_uuid : nodes)
|
2021-11-11 09:03:53 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
bool ok = zk->tryGet(getReplicasListPath(zk_root) / node_uuid, result[node_uuid]);
|
2021-11-16 09:02:44 +00:00
|
|
|
if (!ok)
|
2021-11-11 09:03:53 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
result.erase(node_uuid);
|
2021-11-11 09:03:53 +00:00
|
|
|
LOG_WARNING(log, "Cluster configuration was changed during update, skip nonexisting node");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
/// Checks if custer nodes set is changed.
|
|
|
|
/// Returs true if update required.
|
|
|
|
/// It performs only shallow check (set of nodes' uuids).
|
|
|
|
/// So, if node's hostname are changed, then cluster won't be updated.
|
|
|
|
bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & nodes)
|
2021-11-15 14:52:52 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
bool has_difference = node_uuids.size() != nodes.size() ||
|
|
|
|
std::any_of(node_uuids.begin(), node_uuids.end(), [&nodes] (auto u) { return !nodes.contains(u); });
|
2021-11-16 09:02:44 +00:00
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
return has_difference;
|
|
|
|
}
|
2021-11-16 09:02:44 +00:00
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
ClusterPtr ClusterDiscovery::getCluster(const ClusterInfo & cluster_info)
|
|
|
|
{
|
|
|
|
Strings replica_adresses;
|
|
|
|
replica_adresses.reserve(cluster_info.nodes_info.size());
|
|
|
|
for (const auto & node : cluster_info.nodes_info)
|
|
|
|
replica_adresses.emplace_back(node.second);
|
2021-11-11 09:03:53 +00:00
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
std::vector<std::vector<String>> shards = {replica_adresses};
|
2021-11-11 09:03:53 +00:00
|
|
|
|
|
|
|
bool secure = false;
|
|
|
|
auto maybe_secure_port = context->getTCPPortSecure();
|
|
|
|
auto cluster = std::make_shared<Cluster>(
|
|
|
|
context->getSettings(),
|
|
|
|
shards,
|
|
|
|
context->getUserName(),
|
|
|
|
"",
|
|
|
|
(secure ? (maybe_secure_port ? *maybe_secure_port : DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()),
|
|
|
|
false /* treat_local_as_remote */,
|
|
|
|
context->getApplicationType() == Context::ApplicationType::LOCAL /* treat_local_port_as_remote */,
|
|
|
|
secure);
|
2021-11-16 12:01:30 +00:00
|
|
|
return cluster;
|
|
|
|
}
|
2021-11-11 09:03:53 +00:00
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
/// Reads data from zookeeper and tries to update cluster.
|
|
|
|
/// Returns true on success (or no update required).
|
|
|
|
bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info)
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Updating cluster '{}'", cluster_info.name);
|
|
|
|
|
|
|
|
auto zk = context->getZooKeeper();
|
|
|
|
|
|
|
|
int start_version;
|
|
|
|
Strings node_uuids = getNodeNames(zk, cluster_info.zk_root, cluster_info.name, &start_version, false);
|
|
|
|
|
|
|
|
if (node_uuids.empty())
|
|
|
|
{
|
|
|
|
LOG_ERROR(log, "Can't find any node in cluster '{}', will register again", cluster_info.name);
|
|
|
|
registerInZk(zk, cluster_info);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
auto & nodes_info = cluster_info.nodes_info;
|
|
|
|
if (!needUpdate(node_uuids, nodes_info))
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "No update required for cluster '{}'", cluster_info.name);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
nodes_info = getNodes(zk, cluster_info.zk_root, node_uuids);
|
|
|
|
|
|
|
|
if (nodes_info.empty())
|
|
|
|
return false;
|
|
|
|
|
|
|
|
int current_version;
|
|
|
|
getNodeNames(zk, cluster_info.zk_root, cluster_info.name, ¤t_version, true);
|
|
|
|
|
|
|
|
if (current_version != start_version)
|
|
|
|
{
|
|
|
|
nodes_info.clear();
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
auto cluster = getCluster(cluster_info);
|
|
|
|
context->setCluster(cluster_info.name, cluster);
|
2021-11-16 09:02:44 +00:00
|
|
|
return true;
|
2021-11-11 09:03:53 +00:00
|
|
|
}
|
|
|
|
|
2021-11-16 09:02:44 +00:00
|
|
|
bool ClusterDiscovery::updateCluster(const String & cluster_name)
|
2021-11-15 14:52:52 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
auto cluster_info = clusters_info.find(cluster_name);
|
|
|
|
if (cluster_info == clusters_info.end())
|
|
|
|
{
|
|
|
|
LOG_ERROR(log, "Unknown cluster '{}'", cluster_name);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return updateCluster(cluster_info->second);
|
|
|
|
}
|
|
|
|
|
|
|
|
void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info)
|
|
|
|
{
|
|
|
|
String node_path = getReplicasListPath(info.zk_root) / node_name;
|
|
|
|
zk->createAncestors(node_path);
|
|
|
|
|
|
|
|
String payload = getFQDNOrHostName() + ":" + toString(server_port);
|
|
|
|
|
|
|
|
zk->createOrUpdate(node_path, payload, zkutil::CreateMode::Ephemeral);
|
|
|
|
LOG_DEBUG(log, "Current node {} registered in cluster {}", node_name, info.name);
|
2021-11-15 14:52:52 +00:00
|
|
|
}
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
void ClusterDiscovery::start()
|
|
|
|
{
|
|
|
|
auto zk = context->getZooKeeper();
|
2021-11-15 14:52:52 +00:00
|
|
|
|
|
|
|
LOG_TRACE(log, "Starting working thread");
|
|
|
|
main_thread = ThreadFromGlobalPool([this] { runMainThread(); });
|
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
for (auto & [_, info] : clusters_info)
|
2021-11-11 09:03:53 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
registerInZk(zk, info);
|
|
|
|
updateCluster(info);
|
2021-11-11 09:03:53 +00:00
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
if (!updateCluster(info))
|
|
|
|
LOG_WARNING(log, "Error on updating cluster '{}'", info.name);
|
2021-11-11 09:03:53 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-15 14:52:52 +00:00
|
|
|
void ClusterDiscovery::runMainThread()
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Worker thread started");
|
2021-11-16 12:01:30 +00:00
|
|
|
// setThreadName("ClusterDiscovery");
|
|
|
|
|
|
|
|
using namespace std::chrono_literals;
|
|
|
|
constexpr UInt64 full_update_interval = std::chrono::milliseconds(5min).count();
|
|
|
|
|
|
|
|
std::unordered_map<String, Stopwatch> last_cluster_update;
|
|
|
|
for (const auto & [cluster_name, _] : clusters_info)
|
|
|
|
last_cluster_update.emplace(cluster_name, Stopwatch());
|
|
|
|
Stopwatch last_full_update;
|
|
|
|
|
|
|
|
pcg64 rng(randomSeed());
|
2021-11-15 14:52:52 +00:00
|
|
|
|
|
|
|
while (!stop_flag)
|
|
|
|
{
|
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
String cluster_name;
|
|
|
|
if (queue->tryPop(cluster_name, QUEUE_OP_TIMEOUT_MS))
|
|
|
|
{
|
|
|
|
if (updateCluster(cluster_name))
|
|
|
|
last_cluster_update[cluster_name].restart();
|
|
|
|
else
|
|
|
|
LOG_WARNING(log, "Error on updating cluster '{}', configuration changed during update, will retry", cluster_name);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
auto jitter = std::uniform_real_distribution<>(1.0, 2.0)(rng);
|
|
|
|
if (last_full_update.elapsedMilliseconds() > UInt64(full_update_interval * jitter))
|
|
|
|
{
|
|
|
|
for (const auto & lastupd : last_cluster_update)
|
2021-11-16 09:02:44 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
if (lastupd.second.elapsedMilliseconds() > full_update_interval)
|
|
|
|
{
|
|
|
|
if (updateCluster(lastupd.first))
|
|
|
|
last_cluster_update[lastupd.first].restart();
|
|
|
|
else
|
|
|
|
LOG_WARNING(log, "Error on updating cluster '{}'", lastupd.first);
|
|
|
|
}
|
2021-11-16 09:02:44 +00:00
|
|
|
}
|
2021-11-16 12:01:30 +00:00
|
|
|
last_full_update.restart();
|
2021-11-15 14:52:52 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
LOG_TRACE(log, "Worker thread stopped");
|
|
|
|
}
|
|
|
|
|
|
|
|
void ClusterDiscovery::shutdown()
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Shutting down");
|
|
|
|
|
|
|
|
stop_flag.exchange(true);
|
|
|
|
queue->clearAndFinish();
|
|
|
|
if (main_thread.joinable())
|
|
|
|
main_thread.join();
|
|
|
|
}
|
|
|
|
|
|
|
|
ClusterDiscovery::~ClusterDiscovery()
|
|
|
|
{
|
|
|
|
ClusterDiscovery::shutdown();
|
|
|
|
}
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
}
|