2021-11-16 12:01:30 +00:00
|
|
|
#include <algorithm>
|
2021-11-17 11:32:20 +00:00
|
|
|
#include <atomic>
|
2021-11-16 12:01:30 +00:00
|
|
|
#include <chrono>
|
2021-11-17 11:32:20 +00:00
|
|
|
#include <condition_variable>
|
|
|
|
#include <unordered_map>
|
2021-11-16 12:01:30 +00:00
|
|
|
#include <unordered_set>
|
|
|
|
|
2021-11-15 14:52:52 +00:00
|
|
|
#include <base/getFQDNOrHostName.h>
|
|
|
|
#include <base/logger_useful.h>
|
|
|
|
|
2021-11-17 11:32:20 +00:00
|
|
|
#include <Common/Exception.h>
|
2021-11-11 09:03:53 +00:00
|
|
|
#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>
|
|
|
|
|
2021-11-17 14:16:49 +00:00
|
|
|
#include <Poco/Exception.h>
|
|
|
|
#include <Poco/JSON/JSON.h>
|
|
|
|
#include <Poco/JSON/Object.h>
|
|
|
|
#include <Poco/JSON/Parser.h>
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2021-11-17 11:32:20 +00:00
|
|
|
namespace ErrorCodes
|
2021-11-15 14:52:52 +00:00
|
|
|
{
|
2021-11-17 11:32:20 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
2021-11-15 14:52:52 +00:00
|
|
|
|
2021-11-17 11:32:20 +00:00
|
|
|
namespace
|
|
|
|
{
|
2021-11-15 14:52:52 +00:00
|
|
|
|
2021-11-16 13:39:54 +00:00
|
|
|
fs::path getShardsListPath(const String & zk_root)
|
2021-11-15 14:52:52 +00:00
|
|
|
{
|
2021-11-16 13:39:54 +00:00
|
|
|
return fs::path(zk_root + "/shards");
|
2021-11-15 14:52:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2021-11-17 11:32:20 +00:00
|
|
|
/*
|
|
|
|
* Holds boolean flags for fixed set of keys.
|
|
|
|
* Flags can be concurrently set from different threads, and consumer can wait for it.
|
|
|
|
*/
|
|
|
|
template <typename T>
|
|
|
|
class ClusterDiscovery::ConcurrentFlags
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
template <typename It>
|
|
|
|
ConcurrentFlags(It begin, It end)
|
|
|
|
{
|
|
|
|
for (auto it = begin; it != end; ++it)
|
|
|
|
flags.emplace(*it, false);
|
|
|
|
}
|
|
|
|
|
2021-11-18 08:57:26 +00:00
|
|
|
void set(const T & key)
|
2021-11-17 11:32:20 +00:00
|
|
|
{
|
2021-11-18 08:57:26 +00:00
|
|
|
setFlag(key, true);
|
2021-11-17 11:32:20 +00:00
|
|
|
cv.notify_one();
|
|
|
|
}
|
|
|
|
|
2021-11-18 08:57:26 +00:00
|
|
|
void unset(const T & key) { setFlag(key, false); }
|
2021-11-17 11:32:20 +00:00
|
|
|
|
|
|
|
void wait(std::chrono::milliseconds timeout)
|
|
|
|
{
|
|
|
|
std::unique_lock<std::mutex> lk(mu);
|
|
|
|
cv.wait_for(lk, timeout);
|
|
|
|
}
|
|
|
|
|
2021-11-18 08:57:26 +00:00
|
|
|
const std::unordered_map<T, std::atomic_bool> & get() { return flags; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
|
|
|
|
void setFlag(const T & key, bool value)
|
2021-11-17 11:32:20 +00:00
|
|
|
{
|
2021-11-18 08:57:26 +00:00
|
|
|
auto it = flags.find(key);
|
|
|
|
if (it == flags.end())
|
|
|
|
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key);
|
|
|
|
it->second = value;
|
2021-11-17 11:32:20 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
std::condition_variable cv;
|
|
|
|
std::mutex mu;
|
|
|
|
|
|
|
|
std::unordered_map<T, std::atomic_bool> flags;
|
|
|
|
};
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
ClusterDiscovery::ClusterDiscovery(
|
|
|
|
const Poco::Util::AbstractConfiguration & config,
|
2021-11-18 08:57:26 +00:00
|
|
|
ContextPtr context_,
|
2021-11-11 09:03:53 +00:00
|
|
|
const String & config_prefix)
|
2021-11-18 08:57:26 +00:00
|
|
|
: context(Context::createCopy(context_))
|
2021-11-25 13:45:38 +00:00
|
|
|
, current_node_name(toString(ServerUUID::get()))
|
2021-11-11 09:03:53 +00:00
|
|
|
, server_port(context->getTCPPort())
|
|
|
|
, log(&Poco::Logger::get("ClusterDiscovery"))
|
|
|
|
{
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_DEBUG(log, "Cluster discovery is enabled");
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
|
|
|
config.keys(config_prefix, config_keys);
|
|
|
|
|
|
|
|
for (const auto & key : config_keys)
|
|
|
|
{
|
2021-11-18 09:45:57 +00:00
|
|
|
String prefix = config_prefix + "." + key + ".discovery";
|
|
|
|
if (!config.has(prefix))
|
|
|
|
continue;
|
|
|
|
String path = config.getString(prefix + ".path");
|
2021-11-11 09:03:53 +00:00
|
|
|
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-17 11:32:20 +00:00
|
|
|
clusters_to_update = std::make_shared<UpdateFlags>(config_keys.begin(), config_keys.end());
|
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)
|
|
|
|
{
|
2021-11-17 11:32:20 +00:00
|
|
|
auto watch_callback = [cluster_name, clusters_to_update=clusters_to_update](auto) { clusters_to_update->set(cluster_name); };
|
2021-11-16 09:02:44 +00:00
|
|
|
|
|
|
|
Coordination::Stat stat;
|
2021-11-16 13:39:54 +00:00
|
|
|
Strings nodes = zk->getChildrenWatch(getShardsListPath(zk_root), &stat, set_callback ? watch_callback : Coordination::WatchCallback{});
|
2021-11-16 09:02:44 +00:00
|
|
|
if (version)
|
|
|
|
*version = stat.cversion;
|
|
|
|
return nodes;
|
|
|
|
}
|
|
|
|
|
2021-11-17 11:32:20 +00:00
|
|
|
/// Reads node information from specified zookeeper nodes
|
|
|
|
/// On error returns empty result
|
2021-11-16 13:39:54 +00:00
|
|
|
ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids)
|
2021-11-11 09:03:53 +00:00
|
|
|
{
|
2021-11-16 12:01:30 +00:00
|
|
|
NodesInfo result;
|
2021-11-16 13:39:54 +00:00
|
|
|
for (const auto & node_uuid : node_uuids)
|
2021-11-11 09:03:53 +00:00
|
|
|
{
|
2021-11-16 13:39:54 +00:00
|
|
|
String payload;
|
2021-11-25 13:45:38 +00:00
|
|
|
bool ok = zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload) &&
|
|
|
|
NodeInfo::parse(payload, result[node_uuid]);
|
|
|
|
if (!ok)
|
2021-11-17 14:16:49 +00:00
|
|
|
{
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_WARNING(log, "Can't get data from node '{}' in '{}'", node_uuid, zk_root);
|
2021-11-16 13:39:54 +00:00
|
|
|
return {};
|
2021-11-17 14:16:49 +00:00
|
|
|
}
|
2021-11-11 09:03:53 +00:00
|
|
|
}
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2021-11-16 13:39:54 +00:00
|
|
|
/// Checks if cluster nodes set is changed.
|
|
|
|
/// Returns true if update required.
|
2021-11-16 12:01:30 +00:00
|
|
|
/// 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-17 13:47:40 +00:00
|
|
|
{
|
2021-11-25 13:45:38 +00:00
|
|
|
/// Just to log updated nodes, suboptimal, but should be ok for expected update sizes
|
2021-11-17 13:47:40 +00:00
|
|
|
std::set<String> new_names(node_uuids.begin(), node_uuids.end());
|
|
|
|
std::set<String> old_names;
|
|
|
|
for (const auto & [name, _] : nodes)
|
|
|
|
old_names.emplace(name);
|
|
|
|
|
|
|
|
auto format_cluster_update = [](const std::set<String> & s1, const std::set<String> & s2)
|
|
|
|
{
|
|
|
|
std::vector<String> diff;
|
|
|
|
std::set_difference(s1.begin(), s1.end(), s2.begin(), s2.end(), std::back_inserter(diff));
|
|
|
|
|
|
|
|
constexpr size_t max_to_show = 3;
|
|
|
|
size_t sz = diff.size();
|
2021-11-25 13:45:38 +00:00
|
|
|
bool need_crop = sz > max_to_show;
|
|
|
|
if (need_crop)
|
2021-11-17 13:47:40 +00:00
|
|
|
diff.resize(max_to_show);
|
2021-11-25 13:45:38 +00:00
|
|
|
|
|
|
|
if (sz == 0)
|
|
|
|
return fmt::format("{} nodes", sz);
|
2021-11-29 11:25:52 +00:00
|
|
|
return fmt::format("{} node{} [{}{}]", sz, sz != 1 ? "s" : "", fmt::join(diff, ", "), need_crop ? ",..." : "");
|
2021-11-17 13:47:40 +00:00
|
|
|
};
|
|
|
|
|
2021-11-29 11:25:52 +00:00
|
|
|
LOG_TRACE(log, "Cluster update: added {}, removed {}",
|
2021-11-17 13:47:40 +00:00
|
|
|
format_cluster_update(new_names, old_names),
|
|
|
|
format_cluster_update(old_names, new_names));
|
|
|
|
}
|
2021-11-16 12:01:30 +00:00
|
|
|
return has_difference;
|
|
|
|
}
|
2021-11-16 09:02:44 +00:00
|
|
|
|
2021-11-19 09:42:00 +00:00
|
|
|
ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info)
|
2021-11-16 12:01:30 +00:00
|
|
|
{
|
|
|
|
Strings replica_adresses;
|
|
|
|
replica_adresses.reserve(cluster_info.nodes_info.size());
|
2021-11-19 07:21:40 +00:00
|
|
|
|
|
|
|
std::optional<bool> secure;
|
2021-11-16 13:39:54 +00:00
|
|
|
for (const auto & [_, node] : cluster_info.nodes_info)
|
2021-11-19 07:21:40 +00:00
|
|
|
{
|
|
|
|
if (secure && secure.value() != node.secure)
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Nodes in cluster '{}' has different 'secure' value", cluster_info.name);
|
|
|
|
}
|
|
|
|
|
|
|
|
secure = node.secure;
|
2021-11-16 13:39:54 +00:00
|
|
|
replica_adresses.emplace_back(node.address);
|
2021-11-19 07:21:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// TODO(vdimir@) save custom params from config to zookeeper and use here (like secure), split by shards
|
2021-11-16 12:01:30 +00:00
|
|
|
std::vector<std::vector<String>> shards = {replica_adresses};
|
2021-11-11 09:03:53 +00:00
|
|
|
|
2021-11-19 07:21:40 +00:00
|
|
|
auto secure_port_opt = context->getTCPPortSecure();
|
2021-11-11 09:03:53 +00:00
|
|
|
auto cluster = std::make_shared<Cluster>(
|
|
|
|
context->getSettings(),
|
|
|
|
shards,
|
2021-11-19 07:21:40 +00:00
|
|
|
/* username= */ context->getUserName(),
|
|
|
|
/* password= */ "",
|
|
|
|
/* clickhouse_port= */ (secure.value_or(false) ? secure_port_opt.value_or(DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()),
|
|
|
|
/* treat_local_as_remote= */ false,
|
|
|
|
/* treat_local_port_as_remote= */ context->getApplicationType() == Context::ApplicationType::LOCAL,
|
|
|
|
secure.value_or(false));
|
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);
|
2021-11-29 11:25:52 +00:00
|
|
|
auto & nodes_info = cluster_info.nodes_info;
|
2021-11-16 12:01:30 +00:00
|
|
|
|
2021-11-25 13:45:38 +00:00
|
|
|
if (std::find(node_uuids.begin(), node_uuids.end(), current_node_name) == node_uuids.end())
|
2021-11-16 12:01:30 +00:00
|
|
|
{
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_ERROR(log, "Can't find current node in cluster '{}', will register again", cluster_info.name);
|
2021-11-16 12:01:30 +00:00
|
|
|
registerInZk(zk, cluster_info);
|
2021-11-29 11:25:52 +00:00
|
|
|
nodes_info.clear();
|
2021-11-16 12:01:30 +00:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
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())
|
2021-11-25 13:45:38 +00:00
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Can't get nodes info for '{}'", cluster_info.name);
|
2021-11-16 12:01:30 +00:00
|
|
|
return false;
|
2021-11-25 13:45:38 +00:00
|
|
|
}
|
2021-11-16 12:01:30 +00:00
|
|
|
|
|
|
|
int current_version;
|
|
|
|
getNodeNames(zk, cluster_info.zk_root, cluster_info.name, ¤t_version, true);
|
|
|
|
|
|
|
|
if (current_version != start_version)
|
|
|
|
{
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_TRACE(log, "Cluster '{}' configuration changed during update", cluster_info.name);
|
2021-11-16 12:01:30 +00:00
|
|
|
nodes_info.clear();
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_TRACE(log, "Updating system.clusters record for '{}' with {} nodes", cluster_info.name, cluster_info.nodes_info.size());
|
|
|
|
|
2021-11-19 09:42:00 +00:00
|
|
|
auto cluster = makeCluster(cluster_info);
|
2021-11-16 12:01:30 +00:00
|
|
|
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)
|
|
|
|
{
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_DEBUG(log, "Registering current node {} in cluster {}", current_node_name, info.name);
|
2021-11-19 09:42:00 +00:00
|
|
|
|
2021-11-25 13:45:38 +00:00
|
|
|
String node_path = getShardsListPath(info.zk_root) / current_node_name;
|
2021-11-16 12:01:30 +00:00
|
|
|
zk->createAncestors(node_path);
|
|
|
|
|
2021-11-17 14:16:49 +00:00
|
|
|
NodeInfo self_node(getFQDNOrHostName() + ":" + toString(server_port));
|
2021-11-16 12:01:30 +00:00
|
|
|
|
2021-11-17 14:16:49 +00:00
|
|
|
zk->createOrUpdate(node_path, self_node.serialize(), zkutil::CreateMode::Ephemeral);
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_DEBUG(log, "Current node {} registered in cluster {}", current_node_name, info.name);
|
2021-11-15 14:52:52 +00:00
|
|
|
}
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
void ClusterDiscovery::start()
|
|
|
|
{
|
2021-11-18 08:57:26 +00:00
|
|
|
if (clusters_info.empty())
|
2021-11-11 09:03:53 +00:00
|
|
|
{
|
2021-11-18 08:57:26 +00:00
|
|
|
LOG_DEBUG(log, "No defined clusters for discovery");
|
|
|
|
return;
|
2021-11-11 09:03:53 +00:00
|
|
|
}
|
2021-11-18 08:57:26 +00:00
|
|
|
LOG_TRACE(log, "Starting working thread");
|
2021-11-16 12:01:30 +00:00
|
|
|
|
2021-11-19 09:42:00 +00:00
|
|
|
auto zk = context->getZooKeeper();
|
|
|
|
for (auto & [_, info] : clusters_info)
|
2021-11-18 08:57:26 +00:00
|
|
|
{
|
2021-11-19 09:42:00 +00:00
|
|
|
registerInZk(zk, info);
|
|
|
|
if (!updateCluster(info))
|
2021-11-18 08:57:26 +00:00
|
|
|
{
|
2021-11-19 09:42:00 +00:00
|
|
|
LOG_WARNING(log, "Error on updating cluster '{}', will retry", info.name);
|
|
|
|
clusters_to_update->set(info.name);
|
2021-11-18 08:57:26 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-19 09:42:00 +00:00
|
|
|
main_thread = ThreadFromGlobalPool([this]
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
runMainThread();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log, "Caught exception in cluster discovery runMainThread");
|
|
|
|
}
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
void ClusterDiscovery::runMainThread()
|
|
|
|
{
|
|
|
|
setThreadName("ClusterDiscover");
|
|
|
|
LOG_DEBUG(log, "Worker thread started");
|
|
|
|
|
2021-11-16 12:01:30 +00:00
|
|
|
using namespace std::chrono_literals;
|
2021-11-15 14:52:52 +00:00
|
|
|
|
|
|
|
while (!stop_flag)
|
|
|
|
{
|
2021-11-17 11:32:20 +00:00
|
|
|
/// if some cluster update was ended with error on previous iteration, we will retry after timeout
|
|
|
|
clusters_to_update->wait(5s);
|
|
|
|
for (const auto & [cluster_name, need_update] : clusters_to_update->get())
|
2021-11-15 14:52:52 +00:00
|
|
|
{
|
2021-11-17 11:32:20 +00:00
|
|
|
if (!need_update)
|
|
|
|
continue;
|
2021-11-25 13:45:38 +00:00
|
|
|
|
|
|
|
if (updateCluster(cluster_name))
|
|
|
|
{
|
2021-11-17 11:32:20 +00:00
|
|
|
clusters_to_update->unset(cluster_name);
|
2021-11-25 13:45:38 +00:00
|
|
|
LOG_DEBUG(log, "Cluster '{}' updated successfully", cluster_name);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "Cluster '{}' are not updated, will retry", cluster_name);
|
|
|
|
}
|
2021-11-15 14:52:52 +00:00
|
|
|
}
|
|
|
|
}
|
2021-11-18 08:57:26 +00:00
|
|
|
LOG_DEBUG(log, "Worker thread stopped");
|
2021-11-15 14:52:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void ClusterDiscovery::shutdown()
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Shutting down");
|
|
|
|
|
|
|
|
stop_flag.exchange(true);
|
|
|
|
if (main_thread.joinable())
|
|
|
|
main_thread.join();
|
|
|
|
}
|
|
|
|
|
|
|
|
ClusterDiscovery::~ClusterDiscovery()
|
|
|
|
{
|
|
|
|
ClusterDiscovery::shutdown();
|
|
|
|
}
|
|
|
|
|
2021-11-17 14:16:49 +00:00
|
|
|
bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
Poco::JSON::Parser parser;
|
|
|
|
auto json = parser.parse(data).extract<Poco::JSON::Object::Ptr>();
|
|
|
|
|
|
|
|
result.address = json->getValue<std::string>("address");
|
2021-11-19 07:21:40 +00:00
|
|
|
result.secure = json->optValue<bool>("secure", false);
|
2021-11-17 14:16:49 +00:00
|
|
|
}
|
|
|
|
catch (Poco::Exception & e)
|
|
|
|
{
|
|
|
|
LOG_WARNING(
|
|
|
|
&Poco::Logger::get("ClusterDiscovery"),
|
|
|
|
"Can't parse '{}' from node: {}",
|
|
|
|
data.size() < 1024 ? data : "[data too long]", e.displayText());
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
String ClusterDiscovery::NodeInfo::serialize() const
|
|
|
|
{
|
|
|
|
Poco::JSON::Object json;
|
|
|
|
json.set("address", address);
|
|
|
|
|
|
|
|
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
|
|
|
oss.exceptions(std::ios::failbit);
|
|
|
|
Poco::JSON::Stringifier::stringify(json, oss);
|
|
|
|
return oss.str();
|
|
|
|
}
|
|
|
|
|
2021-11-11 09:03:53 +00:00
|
|
|
}
|