Attempt to generalize ZooKeeper interface [#CLICKHOUSE-2]

This commit is contained in:
Alexey Milovidov 2018-08-25 04:58:14 +03:00
parent 4d343a441c
commit 3475ef5304
41 changed files with 491 additions and 508 deletions

View File

@ -363,8 +363,8 @@ struct TaskCluster
struct MultiTransactionInfo struct MultiTransactionInfo
{ {
int32_t code; int32_t code;
zkutil::Requests requests; Coordination::Requests requests;
zkutil::Responses responses; Coordination::Responses responses;
}; };
@ -373,7 +373,7 @@ struct MultiTransactionInfo
static MultiTransactionInfo checkNoNodeAndCommit( static MultiTransactionInfo checkNoNodeAndCommit(
const zkutil::ZooKeeperPtr & zookeeper, const zkutil::ZooKeeperPtr & zookeeper,
const String & checking_node_path, const String & checking_node_path,
zkutil::RequestPtr && op) Coordination::RequestPtr && op)
{ {
MultiTransactionInfo info; MultiTransactionInfo info;
info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent)); info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent));
@ -742,7 +742,7 @@ public:
{ {
auto zookeeper = context.getZooKeeper(); auto zookeeper = context.getZooKeeper();
task_description_watch_callback = [this] (const ZooKeeperImpl::ZooKeeper::WatchResponse &) task_description_watch_callback = [this] (const Coordination::WatchResponse &)
{ {
UInt64 version = ++task_descprtion_version; UInt64 version = ++task_descprtion_version;
LOG_DEBUG(log, "Task description should be updated, local version " << version); LOG_DEBUG(log, "Task description should be updated, local version " << version);
@ -902,7 +902,7 @@ public:
task_description_watch_zookeeper = zookeeper; task_description_watch_zookeeper = zookeeper;
String task_config_str; String task_config_str;
zkutil::Stat stat; Coordination::Stat stat;
int code; int code;
zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code);
@ -1052,7 +1052,7 @@ protected:
{ {
updateConfigIfNeeded(); updateConfigIfNeeded();
zkutil::Stat stat; Coordination::Stat stat;
zookeeper->get(workers_version_path, &stat); zookeeper->get(workers_version_path, &stat);
auto version = stat.version; auto version = stat.version;
zookeeper->get(workers_path, &stat); zookeeper->get(workers_path, &stat);
@ -1070,16 +1070,16 @@ protected:
} }
else else
{ {
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version));
ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral));
zkutil::Responses responses; Coordination::Responses responses;
auto code = zookeeper->tryMulti(ops, responses); auto code = zookeeper->tryMulti(ops, responses);
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
return std::make_shared<zkutil::EphemeralNodeHolder>(current_worker_path, *zookeeper, false, false, description); return std::make_shared<zkutil::EphemeralNodeHolder>(current_worker_path, *zookeeper, false, false, description);
if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) if (code == Coordination::ZBADVERSION)
{ {
++num_bad_version_errors; ++num_bad_version_errors;
@ -1093,7 +1093,7 @@ protected:
} }
} }
else else
throw zkutil::KeeperException(code); throw Coordination::Exception(code);
} }
} }
} }
@ -1157,7 +1157,7 @@ protected:
zxid2.push_back(res.stat.pzxid); zxid2.push_back(res.stat.pzxid);
} }
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name
<< ". Will recheck the partition. Error: " << e.displayText()); << ". Will recheck the partition. Error: " << e.displayText());
@ -1242,9 +1242,9 @@ protected:
{ {
cleaner_holder = zkutil::EphemeralNodeHolder::create(dirt_cleaner_path, *zookeeper, host_id); cleaner_holder = zkutil::EphemeralNodeHolder::create(dirt_cleaner_path, *zookeeper, host_id);
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (e.code == Coordination::ZNODEEXISTS)
{ {
LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep");
std::this_thread::sleep_for(default_sleep_time); std::this_thread::sleep_for(default_sleep_time);
@ -1254,7 +1254,7 @@ protected:
throw; throw;
} }
zkutil::Stat stat; Coordination::Stat stat;
if (zookeeper->exists(current_partition_active_workers_dir, &stat)) if (zookeeper->exists(current_partition_active_workers_dir, &stat))
{ {
if (stat.numChildren != 0) if (stat.numChildren != 0)
@ -1291,7 +1291,7 @@ protected:
} }
/// Remove the locking node /// Remove the locking node
zkutil::Requests requests; Coordination::Requests requests;
requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1)); requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1));
requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1)); requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1));
zookeeper->multi(requests); zookeeper->multi(requests);
@ -1503,8 +1503,8 @@ protected:
auto create_is_dirty_node = [&] () auto create_is_dirty_node = [&] ()
{ {
auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent); auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code && code != Coordination::ZNODEEXISTS)
throw zkutil::KeeperException(code, is_dirty_flag_path); throw Coordination::Exception(code, is_dirty_flag_path);
}; };
/// Returns SELECT query filtering current partition and applying user filter /// Returns SELECT query filtering current partition and applying user filter
@ -1552,9 +1552,9 @@ protected:
{ {
partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id); partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id);
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (e.code == Coordination::ZNODEEXISTS)
{ {
LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path);
return PartitionTaskStatus::Active; return PartitionTaskStatus::Active;
@ -1605,7 +1605,7 @@ protected:
if (count != 0) if (count != 0)
{ {
zkutil::Stat stat_shards; Coordination::Stat stat_shards;
zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards);
if (stat_shards.numChildren == 0) if (stat_shards.numChildren == 0)
@ -1698,7 +1698,7 @@ protected:
output = io_insert.out; output = io_insert.out;
} }
std::future<zkutil::ExistsResponse> future_is_dirty_checker; std::future<Coordination::ExistsResponse> future_is_dirty_checker;
Stopwatch watch(CLOCK_MONOTONIC_COARSE); Stopwatch watch(CLOCK_MONOTONIC_COARSE);
constexpr size_t check_period_milliseconds = 500; constexpr size_t check_period_milliseconds = 500;
@ -1716,9 +1716,9 @@ protected:
/// Otherwise, the insertion will slow a little bit /// Otherwise, the insertion will slow a little bit
if (watch.elapsedMilliseconds() >= check_period_milliseconds) if (watch.elapsedMilliseconds() >= check_period_milliseconds)
{ {
zkutil::ExistsResponse status = future_is_dirty_checker.get(); Coordination::ExistsResponse status = future_is_dirty_checker.get();
if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE) if (status.error != Coordination::ZNONODE)
throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
} }
@ -2023,13 +2023,13 @@ private:
/// Auto update config stuff /// Auto update config stuff
UInt64 task_descprtion_current_version = 1; UInt64 task_descprtion_current_version = 1;
std::atomic<UInt64> task_descprtion_version{1}; std::atomic<UInt64> task_descprtion_version{1};
zkutil::WatchCallback task_description_watch_callback; Coordination::WatchCallback task_description_watch_callback;
/// ZooKeeper session used to set the callback /// ZooKeeper session used to set the callback
zkutil::ZooKeeperPtr task_description_watch_zookeeper; zkutil::ZooKeeperPtr task_description_watch_zookeeper;
ConfigurationPtr task_cluster_initial_config; ConfigurationPtr task_cluster_initial_config;
ConfigurationPtr task_cluster_current_config; ConfigurationPtr task_cluster_current_config;
zkutil::Stat task_descprtion_current_stat; Coordination::Stat task_descprtion_current_stat;
std::unique_ptr<TaskCluster> task_cluster; std::unique_ptr<TaskCluster> task_cluster;

View File

@ -540,7 +540,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
if (!fallback_to_preprocessed) if (!fallback_to_preprocessed)
throw; throw;
const auto * zk_exception = dynamic_cast<const zkutil::KeeperException *>(ex.nested()); const auto * zk_exception = dynamic_cast<const Coordination::Exception *>(ex.nested());
if (!zk_exception) if (!zk_exception)
throw; throw;

View File

@ -25,9 +25,9 @@ namespace Coordination
Exception::Exception(const std::string & msg, const int32_t code, int) Exception::Exception(const std::string & msg, const int32_t code, int)
: DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code) : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code)
{ {
if (ZooKeeper::isUserError(code)) if (Coordination::isUserError(code))
ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions); ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions);
else if (ZooKeeper::isHardwareError(code)) else if (Coordination::isHardwareError(code))
ProfileEvents::increment(ProfileEvents::ZooKeeperHardwareExceptions); ProfileEvents::increment(ProfileEvents::ZooKeeperHardwareExceptions);
else else
ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions); ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions);

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Core/Types.h> #include <Core/Types.h>
#include <Common/Exception.h>
#include <vector> #include <vector>
#include <memory> #include <memory>

View File

@ -20,7 +20,7 @@ public:
size_t result = 0; size_t result = 0;
std::string result_str; std::string result_str;
zkutil::Stat stat; Coordination::Stat stat;
bool success = false; bool success = false;
auto zookeeper = zookeeper_holder->getZooKeeper(); auto zookeeper = zookeeper_holder->getZooKeeper();
@ -29,11 +29,11 @@ public:
if (zookeeper->tryGet(path, result_str, &stat)) if (zookeeper->tryGet(path, result_str, &stat))
{ {
result = std::stol(result_str) + 1; result = std::stol(result_str) + 1;
success = zookeeper->trySet(path, std::to_string(result), stat.version) == ZooKeeperImpl::ZooKeeper::ZOK; success = zookeeper->trySet(path, std::to_string(result), stat.version) == Coordination::ZOK;
} }
else else
{ {
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZooKeeperImpl::ZooKeeper::ZOK; success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == Coordination::ZOK;
} }
} }
while (!success); while (!success);

View File

@ -1,32 +1,32 @@
#pragma once #pragma once
#include "Types.h" #include <Common/ZooKeeper/Types.h>
namespace zkutil namespace zkutil
{ {
using KeeperException = ZooKeeperImpl::Exception; using KeeperException = Coordination::Exception;
class KeeperMultiException : public KeeperException class KeeperMultiException : public KeeperException
{ {
public: public:
Requests requests; Coordination::Requests requests;
Responses responses; Coordination::Responses responses;
size_t failed_op_index = 0; size_t failed_op_index = 0;
std::string getPathForFirstFailedOp() const; std::string getPathForFirstFailedOp() const;
/// If it is user error throws KeeperMultiException else throws ordinary KeeperException /// If it is user error throws KeeperMultiException else throws ordinary KeeperException
/// If it is ZOK does nothing /// If it is ZOK does nothing
static void check(int32_t code, const Requests & requests, const Responses & responses); static void check(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses);
KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses); KeeperMultiException(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses);
private: private:
static size_t getFailedOpIndex(int32_t code, const Responses & responses); static size_t getFailedOpIndex(int32_t code, const Coordination::Responses & responses);
}; };
} }

View File

@ -122,7 +122,7 @@ private:
{ {
DB::tryLogCurrentException(log); DB::tryLogCurrentException(log);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == Coordination::ZSESSIONEXPIRED)
return; return;
} }
catch (...) catch (...)

View File

@ -18,17 +18,17 @@ bool Lock::tryLock()
std::string dummy; std::string dummy;
int32_t code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy); int32_t code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy);
if (code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code == Coordination::ZNODEEXISTS)
{ {
locked.reset(nullptr); locked.reset(nullptr);
} }
else if (code == ZooKeeperImpl::ZooKeeper::ZOK) else if (code == Coordination::ZOK)
{ {
locked.reset(new ZooKeeperHandler(zookeeper)); locked.reset(new ZooKeeperHandler(zookeeper));
} }
else else
{ {
throw zkutil::KeeperException(code); throw Coordination::Exception(code);
} }
} }
return bool(locked); return bool(locked);
@ -50,7 +50,7 @@ Lock::Status Lock::tryCheck() const
auto zookeeper = zookeeper_holder->getZooKeeper(); auto zookeeper = zookeeper_holder->getZooKeeper();
Status lock_status; Status lock_status;
Stat stat; Coordination::Stat stat;
std::string dummy; std::string dummy;
bool result = zookeeper->tryGet(lock_path, dummy, &stat); bool result = zookeeper->tryGet(lock_path, dummy, &stat);
if (!result) if (!result)

View File

@ -1,16 +1,16 @@
#pragma once #pragma once
#include <common/Types.h> #include <common/Types.h>
#include <future> #include <future>
#include <memory> #include <memory>
#include <vector> #include <vector>
#include <Common/ZooKeeper/ZooKeeperImpl.h> #include <Common/ZooKeeper/IKeeper.h>
#include <Poco/Event.h> #include <Poco/Event.h>
namespace zkutil namespace zkutil
{ {
using Stat = ZooKeeperImpl::ZooKeeper::Stat;
using Strings = std::vector<std::string>; using Strings = std::vector<std::string>;
@ -24,44 +24,14 @@ namespace CreateMode
using EventPtr = std::shared_ptr<Poco::Event>; using EventPtr = std::shared_ptr<Poco::Event>;
/// Callback to call when the watch fires.
/// Because callbacks are called in the single "completion" thread internal to libzookeeper,
/// they must execute as quickly as possible (preferably just set some notification).
using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback;
using Request = ZooKeeperImpl::ZooKeeper::Request;
using Response = ZooKeeperImpl::ZooKeeper::Response;
using RequestPtr = ZooKeeperImpl::ZooKeeper::RequestPtr;
using ResponsePtr = ZooKeeperImpl::ZooKeeper::ResponsePtr;
using Requests = ZooKeeperImpl::ZooKeeper::Requests;
using Responses = ZooKeeperImpl::ZooKeeper::Responses;
using CreateRequest = ZooKeeperImpl::ZooKeeper::CreateRequest;
using RemoveRequest = ZooKeeperImpl::ZooKeeper::RemoveRequest;
using ExistsRequest = ZooKeeperImpl::ZooKeeper::ExistsRequest;
using GetRequest = ZooKeeperImpl::ZooKeeper::GetRequest;
using SetRequest = ZooKeeperImpl::ZooKeeper::SetRequest;
using ListRequest = ZooKeeperImpl::ZooKeeper::ListRequest;
using CheckRequest = ZooKeeperImpl::ZooKeeper::CheckRequest;
using CreateResponse = ZooKeeperImpl::ZooKeeper::CreateResponse;
using RemoveResponse = ZooKeeperImpl::ZooKeeper::RemoveResponse;
using ExistsResponse = ZooKeeperImpl::ZooKeeper::ExistsResponse;
using GetResponse = ZooKeeperImpl::ZooKeeper::GetResponse;
using SetResponse = ZooKeeperImpl::ZooKeeper::SetResponse;
using ListResponse = ZooKeeperImpl::ZooKeeper::ListResponse;
using CheckResponse = ZooKeeperImpl::ZooKeeper::CheckResponse;
/// Gets multiple asynchronous results /// Gets multiple asynchronous results
/// Each pair, the first is path, the second is response eg. CreateResponse, RemoveResponse /// Each pair, the first is path, the second is response eg. CreateResponse, RemoveResponse
template <typename R> template <typename R>
using AsyncResponses = std::vector<std::pair<std::string, std::future<R>>>; using AsyncResponses = std::vector<std::pair<std::string, std::future<R>>>;
RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode); Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode);
RequestPtr makeRemoveRequest(const std::string & path, int version); Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version);
RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version); Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
RequestPtr makeCheckRequest(const std::string & path, int version); Coordination::RequestPtr makeCheckRequest(const std::string & path, int version);
} }

View File

@ -1,4 +1,5 @@
#include "ZooKeeper.h" #include "ZooKeeper.h"
#include "ZooKeeperImpl.h"
#include "KeeperException.h" #include "KeeperException.h"
#include <random> #include <random>
@ -50,16 +51,16 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
chroot = chroot_; chroot = chroot_;
if (hosts.empty()) if (hosts.empty())
throw KeeperException("No addresses passed to ZooKeeper constructor.", ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS); throw KeeperException("No addresses passed to ZooKeeper constructor.", Coordination::ZBADARGUMENTS);
std::vector<std::string> addresses_strings; std::vector<std::string> addresses_strings;
boost::split(addresses_strings, hosts, boost::is_any_of(",")); boost::split(addresses_strings, hosts, boost::is_any_of(","));
ZooKeeperImpl::ZooKeeper::Addresses addresses; Coordination::ZooKeeper::Addresses addresses;
addresses.reserve(addresses_strings.size()); addresses.reserve(addresses_strings.size());
for (const auto & address_string : addresses_strings) for (const auto & address_string : addresses_strings)
addresses.emplace_back(address_string); addresses.emplace_back(address_string);
impl = std::make_unique<ZooKeeperImpl::ZooKeeper>( impl = std::make_unique<Coordination::ZooKeeper>(
addresses, addresses,
chroot, chroot,
identity_.empty() ? "" : "digest", identity_.empty() ? "" : "digest",
@ -71,7 +72,7 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot)); LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
if (!chroot.empty() && !exists("/")) if (!chroot.empty() && !exists("/"))
throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", ZooKeeperImpl::ZooKeeper::ZNONODE); throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", Coordination::ZNONODE);
} }
ZooKeeper::ZooKeeper(const std::string & hosts, const std::string & identity, ZooKeeper::ZooKeeper(const std::string & hosts, const std::string & identity,
@ -112,7 +113,7 @@ struct ZooKeeperArgs
chroot = config.getString(config_name + "." + key); chroot = config.getString(config_name + "." + key);
} }
else else
throw KeeperException(std::string("Unknown key ") + key + " in config file", ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS); throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::ZBADARGUMENTS);
} }
/// Shuffle the hosts to distribute the load among ZooKeeper nodes. /// Shuffle the hosts to distribute the load among ZooKeeper nodes.
@ -129,7 +130,7 @@ struct ZooKeeperArgs
if (!chroot.empty()) if (!chroot.empty())
{ {
if (chroot.front() != '/') if (chroot.front() != '/')
throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot, ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS); throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot, Coordination::ZBADARGUMENTS);
if (chroot.back() == '/') if (chroot.back() == '/')
chroot.pop_back(); chroot.pop_back();
} }
@ -148,22 +149,22 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std
} }
static WatchCallback callbackForEvent(const EventPtr & watch) static Coordination::WatchCallback callbackForEvent(const EventPtr & watch)
{ {
if (!watch) if (!watch)
return {}; return {};
return [watch](const ZooKeeperImpl::ZooKeeper::WatchResponse &) { watch->set(); }; return [watch](const Coordination::WatchResponse &) { watch->set(); };
} }
int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res, int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res,
Stat * stat, Coordination::Stat * stat,
WatchCallback watch_callback) Coordination::WatchCallback watch_callback)
{ {
int32_t code = 0; int32_t code = 0;
Poco::Event event; Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::ListResponse & response) auto callback = [&](const Coordination::ListResponse & response)
{ {
code = response.error; code = response.error;
if (!code) if (!code)
@ -181,7 +182,7 @@ int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res,
} }
Strings ZooKeeper::getChildren( Strings ZooKeeper::getChildren(
const std::string & path, Stat * stat, const EventPtr & watch) const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
{ {
Strings res; Strings res;
check(tryGetChildren(path, res, stat, watch), path); check(tryGetChildren(path, res, stat, watch), path);
@ -189,7 +190,7 @@ Strings ZooKeeper::getChildren(
} }
Strings ZooKeeper::getChildrenWatch( Strings ZooKeeper::getChildrenWatch(
const std::string & path, Stat * stat, WatchCallback watch_callback) const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{ {
Strings res; Strings res;
check(tryGetChildrenWatch(path, res, stat, watch_callback), path); check(tryGetChildrenWatch(path, res, stat, watch_callback), path);
@ -197,22 +198,22 @@ Strings ZooKeeper::getChildrenWatch(
} }
int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res, int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res,
Stat * stat, const EventPtr & watch) Coordination::Stat * stat, const EventPtr & watch)
{ {
int32_t code = getChildrenImpl(path, res, stat, callbackForEvent(watch)); int32_t code = getChildrenImpl(path, res, stat, callbackForEvent(watch));
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE)) if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path); throw KeeperException(code, path);
return code; return code;
} }
int32_t ZooKeeper::tryGetChildrenWatch(const std::string & path, Strings & res, int32_t ZooKeeper::tryGetChildrenWatch(const std::string & path, Strings & res,
Stat * stat, WatchCallback watch_callback) Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{ {
int32_t code = getChildrenImpl(path, res, stat, watch_callback); int32_t code = getChildrenImpl(path, res, stat, watch_callback);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE)) if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path); throw KeeperException(code, path);
return code; return code;
@ -223,7 +224,7 @@ int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data
int32_t code = 0; int32_t code = 0;
Poco::Event event; Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) auto callback = [&](const Coordination::CreateResponse & response)
{ {
code = response.error; code = response.error;
if (!code) if (!code)
@ -247,10 +248,10 @@ int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data,
{ {
int32_t code = createImpl(path, data, mode, path_created); int32_t code = createImpl(path, data, mode, path_created);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || if (!(code == Coordination::ZOK ||
code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == Coordination::ZNONODE ||
code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS || code == Coordination::ZNODEEXISTS ||
code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS)) code == Coordination::ZNOCHILDRENFOREPHEMERALS))
throw KeeperException(code, path); throw KeeperException(code, path);
return code; return code;
@ -267,7 +268,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string &
std::string path_created; std::string path_created;
int32_t code = createImpl(path, data, CreateMode::Persistent, path_created); int32_t code = createImpl(path, data, CreateMode::Persistent, path_created);
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
return; return;
else else
throw KeeperException(code, path); throw KeeperException(code, path);
@ -291,7 +292,7 @@ int32_t ZooKeeper::removeImpl(const std::string & path, int32_t version)
int32_t code = 0; int32_t code = 0;
Poco::Event event; Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) auto callback = [&](const Coordination::RemoveResponse & response)
{ {
if (response.error) if (response.error)
code = response.error; code = response.error;
@ -311,20 +312,20 @@ void ZooKeeper::remove(const std::string & path, int32_t version)
int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version) int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version)
{ {
int32_t code = removeImpl(path, version); int32_t code = removeImpl(path, version);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || if (!(code == Coordination::ZOK ||
code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == Coordination::ZNONODE ||
code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || code == Coordination::ZBADVERSION ||
code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)) code == Coordination::ZNOTEMPTY))
throw KeeperException(code, path); throw KeeperException(code, path);
return code; return code;
} }
int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallback watch_callback) int32_t ZooKeeper::existsImpl(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{ {
int32_t code = 0; int32_t code = 0;
Poco::Event event; Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) auto callback = [&](const Coordination::ExistsResponse & response)
{ {
code = response.error; code = response.error;
if (!code && stat) if (!code && stat)
@ -337,28 +338,28 @@ int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallba
return code; return code;
} }
bool ZooKeeper::exists(const std::string & path, Stat * stat, const EventPtr & watch) bool ZooKeeper::exists(const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
{ {
return existsWatch(path, stat, callbackForEvent(watch)); return existsWatch(path, stat, callbackForEvent(watch));
} }
bool ZooKeeper::existsWatch(const std::string & path, Stat * stat, WatchCallback watch_callback) bool ZooKeeper::existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{ {
int32_t code = existsImpl(path, stat, watch_callback); int32_t code = existsImpl(path, stat, watch_callback);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE)) if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path); throw KeeperException(code, path);
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (code == Coordination::ZNONODE)
return false; return false;
return true; return true;
} }
int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback) int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{ {
int32_t code = 0; int32_t code = 0;
Poco::Event event; Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::GetResponse & response) auto callback = [&](const Coordination::GetResponse & response)
{ {
code = response.error; code = response.error;
if (!code) if (!code)
@ -376,7 +377,7 @@ int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * s
} }
std::string ZooKeeper::get(const std::string & path, Stat * stat, const EventPtr & watch) std::string ZooKeeper::get(const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
{ {
int32_t code = 0; int32_t code = 0;
std::string res; std::string res;
@ -386,7 +387,7 @@ std::string ZooKeeper::get(const std::string & path, Stat * stat, const EventPtr
throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code); throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code);
} }
std::string ZooKeeper::getWatch(const std::string & path, Stat * stat, WatchCallback watch_callback) std::string ZooKeeper::getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{ {
int32_t code = 0; int32_t code = 0;
std::string res; std::string res;
@ -396,31 +397,31 @@ std::string ZooKeeper::getWatch(const std::string & path, Stat * stat, WatchCall
throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code); throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code);
} }
bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat, const EventPtr & watch, int * return_code) bool ZooKeeper::tryGet(const std::string & path, std::string & res, Coordination::Stat * stat, const EventPtr & watch, int * return_code)
{ {
return tryGetWatch(path, res, stat, callbackForEvent(watch), return_code); return tryGetWatch(path, res, stat, callbackForEvent(watch), return_code);
} }
bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback, int * return_code) bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback, int * return_code)
{ {
int32_t code = getImpl(path, res, stat, watch_callback); int32_t code = getImpl(path, res, stat, watch_callback);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE)) if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path); throw KeeperException(code, path);
if (return_code) if (return_code)
*return_code = code; *return_code = code;
return code == ZooKeeperImpl::ZooKeeper::ZOK; return code == Coordination::ZOK;
} }
int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data, int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data,
int32_t version, Stat * stat) int32_t version, Coordination::Stat * stat)
{ {
int32_t code = 0; int32_t code = 0;
Poco::Event event; Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::SetResponse & response) auto callback = [&](const Coordination::SetResponse & response)
{ {
code = response.error; code = response.error;
if (!code && stat) if (!code && stat)
@ -433,7 +434,7 @@ int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data,
return code; return code;
} }
void ZooKeeper::set(const std::string & path, const std::string & data, int32_t version, Stat * stat) void ZooKeeper::set(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat)
{ {
check(trySet(path, data, version, stat), path); check(trySet(path, data, version, stat), path);
} }
@ -441,36 +442,36 @@ void ZooKeeper::set(const std::string & path, const std::string & data, int32_t
void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode) void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode)
{ {
int32_t code = trySet(path, data, -1); int32_t code = trySet(path, data, -1);
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (code == Coordination::ZNONODE)
{ {
create(path, data, mode); create(path, data, mode);
} }
else if (code != ZooKeeperImpl::ZooKeeper::ZOK) else if (code != Coordination::ZOK)
throw KeeperException(code, path); throw KeeperException(code, path);
} }
int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, int32_t ZooKeeper::trySet(const std::string & path, const std::string & data,
int32_t version, Stat * stat) int32_t version, Coordination::Stat * stat)
{ {
int32_t code = setImpl(path, data, version, stat); int32_t code = setImpl(path, data, version, stat);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || if (!(code == Coordination::ZOK ||
code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == Coordination::ZNONODE ||
code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)) code == Coordination::ZBADVERSION))
throw KeeperException(code, path); throw KeeperException(code, path);
return code; return code;
} }
int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses) int32_t ZooKeeper::multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses)
{ {
if (requests.empty()) if (requests.empty())
return ZooKeeperImpl::ZooKeeper::ZOK; return Coordination::ZOK;
int32_t code = 0; int32_t code = 0;
Poco::Event event; Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) auto callback = [&](const Coordination::MultiResponse & response)
{ {
code = response.error; code = response.error;
responses = response.responses; responses = response.responses;
@ -482,18 +483,18 @@ int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses)
return code; return code;
} }
Responses ZooKeeper::multi(const Requests & requests) Coordination::Responses ZooKeeper::multi(const Coordination::Requests & requests)
{ {
Responses responses; Coordination::Responses responses;
int32_t code = multiImpl(requests, responses); int32_t code = multiImpl(requests, responses);
KeeperMultiException::check(code, requests, responses); KeeperMultiException::check(code, requests, responses);
return responses; return responses;
} }
int32_t ZooKeeper::tryMulti(const Requests & requests, Responses & responses) int32_t ZooKeeper::tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses)
{ {
int32_t code = multiImpl(requests, responses); int32_t code = multiImpl(requests, responses);
if (code && !ZooKeeperImpl::ZooKeeper::isUserError(code)) if (code && !Coordination::isUserError(code))
throw KeeperException(code); throw KeeperException(code);
return code; return code;
} }
@ -504,7 +505,7 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path)
Strings children = getChildren(path); Strings children = getChildren(path);
while (!children.empty()) while (!children.empty())
{ {
Requests ops; Coordination::Requests ops;
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{ {
removeChildrenRecursive(path + "/" + children.back()); removeChildrenRecursive(path + "/" + children.back());
@ -518,11 +519,11 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path)
void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path) void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path)
{ {
Strings children; Strings children;
if (tryGetChildren(path, children) != ZooKeeperImpl::ZooKeeper::ZOK) if (tryGetChildren(path, children) != Coordination::ZOK)
return; return;
while (!children.empty()) while (!children.empty())
{ {
Requests ops; Coordination::Requests ops;
Strings batch; Strings batch;
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{ {
@ -530,17 +531,17 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path)
children.pop_back(); children.pop_back();
tryRemoveChildrenRecursive(batch.back()); tryRemoveChildrenRecursive(batch.back());
ZooKeeperImpl::ZooKeeper::RemoveRequest request; Coordination::RemoveRequest request;
request.path = batch.back(); request.path = batch.back();
ops.emplace_back(std::make_shared<ZooKeeperImpl::ZooKeeper::RemoveRequest>(std::move(request))); ops.emplace_back(std::make_shared<Coordination::RemoveRequest>(std::move(request)));
} }
/// Try to remove the children with a faster method - in bulk. If this fails, /// Try to remove the children with a faster method - in bulk. If this fails,
/// this means someone is concurrently removing these children and we will have /// this means someone is concurrently removing these children and we will have
/// to remove them one by one. /// to remove them one by one.
Responses responses; Coordination::Responses responses;
if (tryMulti(ops, responses) != ZooKeeperImpl::ZooKeeper::ZOK) if (tryMulti(ops, responses) != Coordination::ZOK)
for (const std::string & child : batch) for (const std::string & child : batch)
tryRemove(child); tryRemove(child);
} }
@ -576,14 +577,14 @@ void ZooKeeper::waitForDisappear(const std::string & path)
while (true) while (true)
{ {
auto callback = [state](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) auto callback = [state](const Coordination::ExistsResponse & response)
{ {
state->code = response.error; state->code = response.error;
if (state->code) if (state->code)
state->event.set(); state->event.set();
}; };
auto watch = [state](const ZooKeeperImpl::ZooKeeper::WatchResponse & response) auto watch = [state](const Coordination::WatchResponse & response)
{ {
if (!state->code) if (!state->code)
{ {
@ -599,13 +600,13 @@ void ZooKeeper::waitForDisappear(const std::string & path)
impl->exists(path, callback, watch); impl->exists(path, callback, watch);
state->event.wait(); state->event.wait();
if (state->code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (state->code == Coordination::ZNONODE)
return; return;
if (state->code) if (state->code)
throw KeeperException(state->code, path); throw KeeperException(state->code, path);
if (state->event_type == ZooKeeperImpl::ZooKeeper::DELETED) if (state->event_type == Coordination::DELETED)
return; return;
} }
} }
@ -618,7 +619,7 @@ ZooKeeperPtr ZooKeeper::startNewSession() const
std::string ZooKeeper::error2string(int32_t code) std::string ZooKeeper::error2string(int32_t code)
{ {
return ZooKeeperImpl::ZooKeeper::errorMessage(code); return Coordination::errorMessage(code);
} }
bool ZooKeeper::expired() bool ZooKeeper::expired()
@ -632,13 +633,13 @@ Int64 ZooKeeper::getClientID()
} }
std::future<ZooKeeperImpl::ZooKeeper::CreateResponse> ZooKeeper::asyncCreate(const std::string & path, const std::string & data, int32_t mode) std::future<Coordination::CreateResponse> ZooKeeper::asyncCreate(const std::string & path, const std::string & data, int32_t mode)
{ {
/// https://stackoverflow.com/questions/25421346/how-to-create-an-stdfunction-from-a-move-capturing-lambda-expression /// https://stackoverflow.com/questions/25421346/how-to-create-an-stdfunction-from-a-move-capturing-lambda-expression
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::CreateResponse>>(); auto promise = std::make_shared<std::promise<Coordination::CreateResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) mutable auto callback = [promise, path](const Coordination::CreateResponse & response) mutable
{ {
if (response.error) if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -651,12 +652,12 @@ std::future<ZooKeeperImpl::ZooKeeper::CreateResponse> ZooKeeper::asyncCreate(con
} }
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncGet(const std::string & path) std::future<Coordination::GetResponse> ZooKeeper::asyncGet(const std::string & path)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::GetResponse>>(); auto promise = std::make_shared<std::promise<Coordination::GetResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable auto callback = [promise, path](const Coordination::GetResponse & response) mutable
{ {
if (response.error) if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -669,14 +670,14 @@ std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncGet(const std
} }
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncTryGet(const std::string & path) std::future<Coordination::GetResponse> ZooKeeper::asyncTryGet(const std::string & path)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::GetResponse>>(); auto promise = std::make_shared<std::promise<Coordination::GetResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable auto callback = [promise, path](const Coordination::GetResponse & response) mutable
{ {
if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) if (response.error && response.error != Coordination::ZNONODE)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
else else
promise->set_value(response); promise->set_value(response);
@ -686,14 +687,14 @@ std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncTryGet(const
return future; return future;
} }
std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> ZooKeeper::asyncExists(const std::string & path) std::future<Coordination::ExistsResponse> ZooKeeper::asyncExists(const std::string & path)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::ExistsResponse>>(); auto promise = std::make_shared<std::promise<Coordination::ExistsResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable auto callback = [promise, path](const Coordination::ExistsResponse & response) mutable
{ {
if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) if (response.error && response.error != Coordination::ZNONODE)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
else else
promise->set_value(response); promise->set_value(response);
@ -703,12 +704,12 @@ std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> ZooKeeper::asyncExists(con
return future; return future;
} }
std::future<ZooKeeperImpl::ZooKeeper::SetResponse> ZooKeeper::asyncSet(const std::string & path, const std::string & data, int32_t version) std::future<Coordination::SetResponse> ZooKeeper::asyncSet(const std::string & path, const std::string & data, int32_t version)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::SetResponse>>(); auto promise = std::make_shared<std::promise<Coordination::SetResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::SetResponse & response) mutable auto callback = [promise, path](const Coordination::SetResponse & response) mutable
{ {
if (response.error) if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -720,12 +721,12 @@ std::future<ZooKeeperImpl::ZooKeeper::SetResponse> ZooKeeper::asyncSet(const std
return future; return future;
} }
std::future<ZooKeeperImpl::ZooKeeper::ListResponse> ZooKeeper::asyncGetChildren(const std::string & path) std::future<Coordination::ListResponse> ZooKeeper::asyncGetChildren(const std::string & path)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::ListResponse>>(); auto promise = std::make_shared<std::promise<Coordination::ListResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable auto callback = [promise, path](const Coordination::ListResponse & response) mutable
{ {
if (response.error) if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -737,12 +738,12 @@ std::future<ZooKeeperImpl::ZooKeeper::ListResponse> ZooKeeper::asyncGetChildren(
return future; return future;
} }
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncRemove(const std::string & path, int32_t version) std::future<Coordination::RemoveResponse> ZooKeeper::asyncRemove(const std::string & path, int32_t version)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::RemoveResponse>>(); auto promise = std::make_shared<std::promise<Coordination::RemoveResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable
{ {
if (response.error) if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -754,14 +755,14 @@ std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncRemove(con
return future; return future;
} }
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncTryRemove(const std::string & path, int32_t version) std::future<Coordination::RemoveResponse> ZooKeeper::asyncTryRemove(const std::string & path, int32_t version)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::RemoveResponse>>(); auto promise = std::make_shared<std::promise<Coordination::RemoveResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable
{ {
if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE && response.error != ZooKeeperImpl::ZooKeeper::ZBADVERSION && response.error != ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) if (response.error && response.error != Coordination::ZNONODE && response.error != Coordination::ZBADVERSION && response.error != Coordination::ZNOTEMPTY)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
else else
promise->set_value(response); promise->set_value(response);
@ -771,12 +772,12 @@ std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncTryRemove(
return future; return future;
} }
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::tryAsyncMulti(const Requests & ops) std::future<Coordination::MultiResponse> ZooKeeper::tryAsyncMulti(const Coordination::Requests & ops)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::MultiResponse>>(); auto promise = std::make_shared<std::promise<Coordination::MultiResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable auto callback = [promise](const Coordination::MultiResponse & response) mutable
{ {
promise->set_value(response); promise->set_value(response);
}; };
@ -785,12 +786,12 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::tryAsyncMulti(co
return future; return future;
} }
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::asyncMulti(const Requests & ops) std::future<Coordination::MultiResponse> ZooKeeper::asyncMulti(const Coordination::Requests & ops)
{ {
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::MultiResponse>>(); auto promise = std::make_shared<std::promise<Coordination::MultiResponse>>();
auto future = promise->get_future(); auto future = promise->get_future();
auto callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable auto callback = [promise](const Coordination::MultiResponse & response) mutable
{ {
if (response.error) if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); promise->set_exception(std::make_exception_ptr(KeeperException(response.error)));
@ -802,20 +803,20 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::asyncMulti(const
return future; return future;
} }
int32_t ZooKeeper::tryMultiNoThrow(const Requests & requests, Responses & responses) int32_t ZooKeeper::tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses)
{ {
try try
{ {
return multiImpl(requests, responses); return multiImpl(requests, responses);
} }
catch (const ZooKeeperImpl::Exception & e) catch (const Coordination::Exception & e)
{ {
return e.code; return e.code;
} }
} }
size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & responses) size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Coordination::Responses & responses)
{ {
if (responses.empty()) if (responses.empty())
throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR); throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR);
@ -824,7 +825,7 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & re
if (responses[index]->error) if (responses[index]->error)
return index; return index;
if (!ZooKeeperImpl::ZooKeeper::isUserError(code)) if (!Coordination::isUserError(code))
throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(code) + "' is not valid response code for that", throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(code) + "' is not valid response code for that",
DB::ErrorCodes::LOGICAL_ERROR); DB::ErrorCodes::LOGICAL_ERROR);
@ -832,7 +833,7 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & re
} }
KeeperMultiException::KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses) KeeperMultiException::KeeperMultiException(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses)
: KeeperException("Transaction failed", code), : KeeperException("Transaction failed", code),
requests(requests), responses(responses), failed_op_index(getFailedOpIndex(code, responses)) requests(requests), responses(responses), failed_op_index(getFailedOpIndex(code, responses))
{ {
@ -845,21 +846,21 @@ std::string KeeperMultiException::getPathForFirstFailedOp() const
return requests[failed_op_index]->getPath(); return requests[failed_op_index]->getPath();
} }
void KeeperMultiException::check(int32_t code, const Requests & requests, const Responses & responses) void KeeperMultiException::check(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses)
{ {
if (!code) if (!code)
return; return;
if (ZooKeeperImpl::ZooKeeper::isUserError(code)) if (Coordination::isUserError(code))
throw KeeperMultiException(code, requests, responses); throw KeeperMultiException(code, requests, responses);
else else
throw KeeperException(code); throw KeeperException(code);
} }
RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode) Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode)
{ {
auto request = std::make_shared<CreateRequest>(); auto request = std::make_shared<Coordination::CreateRequest>();
request->path = path; request->path = path;
request->data = data; request->data = data;
request->is_ephemeral = create_mode == CreateMode::Ephemeral || create_mode == CreateMode::EphemeralSequential; request->is_ephemeral = create_mode == CreateMode::Ephemeral || create_mode == CreateMode::EphemeralSequential;
@ -867,26 +868,26 @@ RequestPtr makeCreateRequest(const std::string & path, const std::string & data,
return request; return request;
} }
RequestPtr makeRemoveRequest(const std::string & path, int version) Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version)
{ {
auto request = std::make_shared<RemoveRequest>(); auto request = std::make_shared<Coordination::RemoveRequest>();
request->path = path; request->path = path;
request->version = version; request->version = version;
return request; return request;
} }
RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version) Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version)
{ {
auto request = std::make_shared<SetRequest>(); auto request = std::make_shared<Coordination::SetRequest>();
request->path = path; request->path = path;
request->data = data; request->data = data;
request->version = version; request->version = version;
return request; return request;
} }
RequestPtr makeCheckRequest(const std::string & path, int version) Coordination::RequestPtr makeCheckRequest(const std::string & path, int version)
{ {
auto request = std::make_shared<CheckRequest>(); auto request = std::make_shared<Coordination::CheckRequest>();
request->path = path; request->path = path;
request->version = version; request->version = version;
return request; return request;

View File

@ -10,6 +10,7 @@
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <port/unistd.h> #include <port/unistd.h>
@ -109,20 +110,20 @@ public:
/// * The node has children. /// * The node has children.
int32_t tryRemove(const std::string & path, int32_t version = -1); int32_t tryRemove(const std::string & path, int32_t version = -1);
bool exists(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr); bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr);
bool existsWatch(const std::string & path, Stat * stat, WatchCallback watch_callback); bool existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
std::string get(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr); std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr);
std::string getWatch(const std::string & path, Stat * stat, WatchCallback watch_callback); std::string getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
/// Doesn't not throw in the following cases: /// Doesn't not throw in the following cases:
/// * The node doesn't exist. Returns false in this case. /// * The node doesn't exist. Returns false in this case.
bool tryGet(const std::string & path, std::string & res, Stat * stat = nullptr, const EventPtr & watch = nullptr, int * code = nullptr); bool tryGet(const std::string & path, std::string & res, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr, int * code = nullptr);
bool tryGetWatch(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback, int * code = nullptr); bool tryGetWatch(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback, int * code = nullptr);
void set(const std::string & path, const std::string & data, void set(const std::string & path, const std::string & data,
int32_t version = -1, Stat * stat = nullptr); int32_t version = -1, Coordination::Stat * stat = nullptr);
/// Creates the node if it doesn't exist. Updates its contents otherwise. /// Creates the node if it doesn't exist. Updates its contents otherwise.
void createOrUpdate(const std::string & path, const std::string & data, int32_t mode); void createOrUpdate(const std::string & path, const std::string & data, int32_t mode);
@ -131,34 +132,34 @@ public:
/// * The node doesn't exist. /// * The node doesn't exist.
/// * Versions do not match. /// * Versions do not match.
int32_t trySet(const std::string & path, const std::string & data, int32_t trySet(const std::string & path, const std::string & data,
int32_t version = -1, Stat * stat = nullptr); int32_t version = -1, Coordination::Stat * stat = nullptr);
Strings getChildren(const std::string & path, Strings getChildren(const std::string & path,
Stat * stat = nullptr, Coordination::Stat * stat = nullptr,
const EventPtr & watch = nullptr); const EventPtr & watch = nullptr);
Strings getChildrenWatch(const std::string & path, Strings getChildrenWatch(const std::string & path,
Stat * stat, Coordination::Stat * stat,
WatchCallback watch_callback); Coordination::WatchCallback watch_callback);
/// Doesn't not throw in the following cases: /// Doesn't not throw in the following cases:
/// * The node doesn't exist. /// * The node doesn't exist.
int32_t tryGetChildren(const std::string & path, Strings & res, int32_t tryGetChildren(const std::string & path, Strings & res,
Stat * stat = nullptr, Coordination::Stat * stat = nullptr,
const EventPtr & watch = nullptr); const EventPtr & watch = nullptr);
int32_t tryGetChildrenWatch(const std::string & path, Strings & res, int32_t tryGetChildrenWatch(const std::string & path, Strings & res,
Stat * stat, Coordination::Stat * stat,
WatchCallback watch_callback); Coordination::WatchCallback watch_callback);
/// Performs several operations in a transaction. /// Performs several operations in a transaction.
/// Throws on every error. /// Throws on every error.
Responses multi(const Requests & requests); Coordination::Responses multi(const Coordination::Requests & requests);
/// Throws only if some operation has returned an "unexpected" error /// Throws only if some operation has returned an "unexpected" error
/// - an error that would cause the corresponding try- method to throw. /// - an error that would cause the corresponding try- method to throw.
int32_t tryMulti(const Requests & requests, Responses & responses); int32_t tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses);
/// Throws nothing (even session expired errors) /// Throws nothing (even session expired errors)
int32_t tryMultiNoThrow(const Requests & requests, Responses & responses); int32_t tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses);
Int64 getClientID(); Int64 getClientID();
@ -190,24 +191,24 @@ public:
/// ///
/// Future should not be destroyed before the result is gotten. /// Future should not be destroyed before the result is gotten.
using FutureCreate = std::future<ZooKeeperImpl::ZooKeeper::CreateResponse>; using FutureCreate = std::future<Coordination::CreateResponse>;
FutureCreate asyncCreate(const std::string & path, const std::string & data, int32_t mode); FutureCreate asyncCreate(const std::string & path, const std::string & data, int32_t mode);
using FutureGet = std::future<ZooKeeperImpl::ZooKeeper::GetResponse>; using FutureGet = std::future<Coordination::GetResponse>;
FutureGet asyncGet(const std::string & path); FutureGet asyncGet(const std::string & path);
FutureGet asyncTryGet(const std::string & path); FutureGet asyncTryGet(const std::string & path);
using FutureExists = std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse>; using FutureExists = std::future<Coordination::ExistsResponse>;
FutureExists asyncExists(const std::string & path); FutureExists asyncExists(const std::string & path);
using FutureGetChildren = std::future<ZooKeeperImpl::ZooKeeper::ListResponse>; using FutureGetChildren = std::future<Coordination::ListResponse>;
FutureGetChildren asyncGetChildren(const std::string & path); FutureGetChildren asyncGetChildren(const std::string & path);
using FutureSet = std::future<ZooKeeperImpl::ZooKeeper::SetResponse>; using FutureSet = std::future<Coordination::SetResponse>;
FutureSet asyncSet(const std::string & path, const std::string & data, int32_t version = -1); FutureSet asyncSet(const std::string & path, const std::string & data, int32_t version = -1);
using FutureRemove = std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse>; using FutureRemove = std::future<Coordination::RemoveResponse>;
FutureRemove asyncRemove(const std::string & path, int32_t version = -1); FutureRemove asyncRemove(const std::string & path, int32_t version = -1);
/// Doesn't throw in the following cases: /// Doesn't throw in the following cases:
@ -216,11 +217,11 @@ public:
/// * The node has children /// * The node has children
FutureRemove asyncTryRemove(const std::string & path, int32_t version = -1); FutureRemove asyncTryRemove(const std::string & path, int32_t version = -1);
using FutureMulti = std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>; using FutureMulti = std::future<Coordination::MultiResponse>;
FutureMulti asyncMulti(const Requests & ops); FutureMulti asyncMulti(const Coordination::Requests & ops);
/// Like the previous one but don't throw any exceptions on future.get() /// Like the previous one but don't throw any exceptions on future.get()
FutureMulti tryAsyncMulti(const Requests & ops); FutureMulti tryAsyncMulti(const Coordination::Requests & ops);
static std::string error2string(int32_t code); static std::string error2string(int32_t code);
@ -235,13 +236,13 @@ private:
/// The following methods don't throw exceptions but return error codes. /// The following methods don't throw exceptions but return error codes.
int32_t createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); int32_t createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created);
int32_t removeImpl(const std::string & path, int32_t version); int32_t removeImpl(const std::string & path, int32_t version);
int32_t getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback); int32_t getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
int32_t setImpl(const std::string & path, const std::string & data, int32_t version, Stat * stat); int32_t setImpl(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat);
int32_t getChildrenImpl(const std::string & path, Strings & res, Stat * stat, WatchCallback watch_callback); int32_t getChildrenImpl(const std::string & path, Strings & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
int32_t multiImpl(const Requests & requests, Responses & responses); int32_t multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses);
int32_t existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback); int32_t existsImpl(const std::string & path, Coordination::Stat * stat_, Coordination::WatchCallback watch_callback);
std::unique_ptr<ZooKeeperImpl::ZooKeeper> impl; std::unique_ptr<Coordination::IKeeper> impl;
std::string hosts; std::string hosts;
std::string identity; std::string identity;

View File

@ -453,7 +453,6 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse
Coordination::read(state, in); Coordination::read(state, in);
Coordination::read(path, in); Coordination::read(path, in);
} }
void removeRootPath(const String & root_path) override;
}; };
struct ZooKeeperAuthRequest final : ZooKeeperRequest struct ZooKeeperAuthRequest final : ZooKeeperRequest
@ -463,7 +462,6 @@ struct ZooKeeperAuthRequest final : ZooKeeperRequest
String data; String data;
String getPath() const override { return {}; } String getPath() const override { return {}; }
RequestPtr clone() const override { return std::make_shared<ZooKeeperAuthRequest>(*this); }
ZooKeeper::OpNum getOpNum() const override { return 100; } ZooKeeper::OpNum getOpNum() const override { return 100; }
void writeImpl(WriteBuffer & out) const override void writeImpl(WriteBuffer & out) const override
{ {
@ -489,7 +487,7 @@ struct ZooKeeperCloseRequest final : ZooKeeperRequest
struct ZooKeeperCloseResponse final : ZooKeeperResponse struct ZooKeeperCloseResponse final : ZooKeeperResponse
{ {
void readImpl(ReadBuffer & in) override void readImpl(ReadBuffer &) override
{ {
throw Exception("Received response for close request", ZRUNTIMEINCONSISTENCY); throw Exception("Received response for close request", ZRUNTIMEINCONSISTENCY);
} }
@ -497,6 +495,9 @@ struct ZooKeeperCloseResponse final : ZooKeeperResponse
struct ZooKeeperCreateRequest final : CreateRequest, ZooKeeperRequest struct ZooKeeperCreateRequest final : CreateRequest, ZooKeeperRequest
{ {
ZooKeeperCreateRequest() {}
ZooKeeperCreateRequest(const CreateRequest & base) : CreateRequest(base) {}
ZooKeeper::OpNum getOpNum() const override { return 1; } ZooKeeper::OpNum getOpNum() const override { return 1; }
void writeImpl(WriteBuffer & out) const override void writeImpl(WriteBuffer & out) const override
{ {
@ -526,6 +527,9 @@ struct ZooKeeperCreateResponse final : CreateResponse, ZooKeeperResponse
struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest
{ {
ZooKeeperRemoveRequest() {}
ZooKeeperRemoveRequest(const RemoveRequest & base) : RemoveRequest(base) {}
ZooKeeper::OpNum getOpNum() const override { return 2; } ZooKeeper::OpNum getOpNum() const override { return 2; }
void writeImpl(WriteBuffer & out) const override void writeImpl(WriteBuffer & out) const override
{ {
@ -581,6 +585,9 @@ struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse
struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest
{ {
ZooKeeperSetRequest() {}
ZooKeeperSetRequest(const SetRequest & base) : SetRequest(base) {}
ZooKeeper::OpNum getOpNum() const override { return 5; } ZooKeeper::OpNum getOpNum() const override { return 5; }
void writeImpl(WriteBuffer & out) const override void writeImpl(WriteBuffer & out) const override
{ {
@ -621,6 +628,9 @@ struct ZooKeeperListResponse final : ListResponse, ZooKeeperResponse
struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest
{ {
ZooKeeperCheckRequest() {}
ZooKeeperCheckRequest(const CheckRequest & base) : CheckRequest(base) {}
ZooKeeper::OpNum getOpNum() const override { return 13; } ZooKeeper::OpNum getOpNum() const override { return 13; }
void writeImpl(WriteBuffer & out) const override void writeImpl(WriteBuffer & out) const override
{ {
@ -873,7 +883,7 @@ void ZooKeeper::connect(
Poco::Timespan connection_timeout) Poco::Timespan connection_timeout)
{ {
if (addresses.empty()) if (addresses.empty())
throw Exception("No addresses passed to ZooKeeperImpl constructor", ZBADARGUMENTS); throw Exception("No addresses passed to ZooKeeper constructor", ZBADARGUMENTS);
static constexpr size_t num_tries = 3; static constexpr size_t num_tries = 3;
bool connected = false; bool connected = false;

View File

@ -84,7 +84,7 @@ namespace Coordination
using namespace DB; using namespace DB;
class ZooKeeperRequest; struct ZooKeeperRequest;

View File

@ -41,18 +41,18 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
if (nonexistent_nodes.count(path)) if (nonexistent_nodes.count(path))
return std::nullopt; return std::nullopt;
auto watch_callback = [context=context](const ZooKeeperImpl::ZooKeeper::WatchResponse & response) auto watch_callback = [context=context](const Coordination::WatchResponse & response)
{ {
if (!(response.type != ZooKeeperImpl::ZooKeeper::SESSION || response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION)) if (!(response.type != Coordination::SESSION || response.state == Coordination::EXPIRED_SESSION))
return; return;
bool changed = false; bool changed = false;
{ {
std::lock_guard<std::mutex> lock(context->mutex); std::lock_guard<std::mutex> lock(context->mutex);
if (response.type != ZooKeeperImpl::ZooKeeper::SESSION) if (response.type != Coordination::SESSION)
changed = context->invalidated_paths.emplace(response.path).second; changed = context->invalidated_paths.emplace(response.path).second;
else if (response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION) else if (response.state == Coordination::EXPIRED_SESSION)
{ {
context->zookeeper = nullptr; context->zookeeper = nullptr;
context->invalidated_paths.clear(); context->invalidated_paths.clear();

View File

@ -33,7 +33,7 @@ TEST(zkutil, multi_nice_exception_msg)
{ {
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181"); auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
zkutil::Requests ops; Coordination::Requests ops;
ASSERT_NO_THROW( ASSERT_NO_THROW(
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi"); zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
@ -70,7 +70,7 @@ TEST(zkutil, multi_nice_exception_msg)
TEST(zkutil, multi_async) TEST(zkutil, multi_async)
{ {
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181"); auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
zkutil::Requests ops; Coordination::Requests ops;
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi"); zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
@ -88,13 +88,13 @@ TEST(zkutil, multi_async)
ops.clear(); ops.clear();
auto res = fut.get(); auto res = fut.get();
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZOK); ASSERT_TRUE(res.error == Coordination::ZOK);
ASSERT_EQ(res.responses.size(), 2); ASSERT_EQ(res.responses.size(), 2);
} }
EXPECT_ANY_THROW EXPECT_ANY_THROW
( (
std::vector<std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>> futures; std::vector<std::future<Coordination::MultiResponse>> futures;
for (size_t i = 0; i < 10000; ++i) for (size_t i = 0; i < 10000; ++i)
{ {
@ -124,7 +124,7 @@ TEST(zkutil, multi_async)
ops.clear(); ops.clear();
auto res = fut.get(); auto res = fut.get();
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS); ASSERT_TRUE(res.error == Coordination::ZNODEEXISTS);
ASSERT_EQ(res.responses.size(), 2); ASSERT_EQ(res.responses.size(), 2);
} }
} }
@ -176,11 +176,11 @@ TEST(zkutil, multi_create_sequential)
zookeeper->tryRemoveRecursive(base_path); zookeeper->tryRemoveRecursive(base_path);
zookeeper->createAncestors(base_path + "/"); zookeeper->createAncestors(base_path + "/");
zkutil::Requests ops; Coordination::Requests ops;
String sequential_node_prefix = base_path + "/queue-"; String sequential_node_prefix = base_path + "/queue-";
ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential)); ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential));
auto results = zookeeper->multi(ops); auto results = zookeeper->multi(ops);
const auto & sequential_node_result_op = typeid_cast<const zkutil::CreateResponse &>(*results.at(0)); const auto & sequential_node_result_op = dynamic_cast<const Coordination::CreateResponse &>(*results.at(0));
EXPECT_FALSE(sequential_node_result_op.path_created.empty()); EXPECT_FALSE(sequential_node_result_op.path_created.empty());
EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length()); EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length());

View File

@ -34,11 +34,11 @@ int main(int argc, char ** argv)
while (true) while (true)
{ {
{ {
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest("/test/zk_expiration_test", "hello", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest("/test/zk_expiration_test", "hello", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest("/test/zk_expiration_test", -1)); ops.emplace_back(zkutil::makeRemoveRequest("/test/zk_expiration_test", -1));
zkutil::Responses responses; Coordination::Responses responses;
int32_t code = zk.tryMultiNoThrow(ops, responses); int32_t code = zk.tryMultiNoThrow(ops, responses);
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(code) << std::endl; std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(code) << std::endl;
@ -57,7 +57,7 @@ int main(int argc, char ** argv)
sleep(1); sleep(1);
} }
} }
catch (zkutil::KeeperException & e) catch (Coordination::Exception & e)
{ {
std::cerr << "KeeperException: " << DB::getCurrentExceptionMessage(true) << std::endl; std::cerr << "KeeperException: " << DB::getCurrentExceptionMessage(true) << std::endl;
return 1; return 1;

View File

@ -23,7 +23,7 @@ try
{ {
while (true) while (true)
{ {
std::vector<std::future<zkutil::GetResponse>> futures; std::vector<std::future<Coordination::GetResponse>> futures;
for (auto & node : nodes) for (auto & node : nodes)
futures.push_back(zookeeper.asyncGet("/tmp/" + node)); futures.push_back(zookeeper.asyncGet("/tmp/" + node));

View File

@ -20,7 +20,7 @@ try
std::cout << "create path" << std::endl; std::cout << "create path" << std::endl;
zk.create("/test", "old", zkutil::CreateMode::Persistent); zk.create("/test", "old", zkutil::CreateMode::Persistent);
zkutil::Stat stat; Coordination::Stat stat;
zkutil::EventPtr watch = std::make_shared<Poco::Event>(); zkutil::EventPtr watch = std::make_shared<Poco::Event>();
std::cout << "get path" << std::endl; std::cout << "get path" << std::endl;
@ -38,13 +38,13 @@ try
zk.remove("/test"); zk.remove("/test");
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest("/test", "multi1", CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest("/test", "multi1", CreateMode::Persistent));
ops.emplace_back(zkutil::makeSetRequest("/test", "multi2", -1)); ops.emplace_back(zkutil::makeSetRequest("/test", "multi2", -1));
ops.emplace_back(zkutil::makeRemoveRequest("/test", -1)); ops.emplace_back(zkutil::makeRemoveRequest("/test", -1));
std::cout << "multi" << std::endl; std::cout << "multi" << std::endl;
zkutil::Responses res = zk.multi(ops); Coordination::Responses res = zk.multi(ops);
std::cout << "path created: " << typeid_cast<const CreateResponse &>(*res[0]).path_created << std::endl; std::cout << "path created: " << dynamic_cast<const Coordination::CreateResponse &>(*res[0]).path_created << std::endl;
return 0; return 0;
} }

View File

@ -7,7 +7,7 @@
#include <boost/algorithm/string.hpp> #include <boost/algorithm/string.hpp>
using namespace ZooKeeperImpl; using namespace Coordination;
int main(int argc, char ** argv) int main(int argc, char ** argv)
@ -38,10 +38,10 @@ try
std::cout << "create\n"; std::cout << "create\n";
zk.create("/test", "old", false, false, {}, zk.create("/test", "old", false, false, {},
[&](const ZooKeeper::CreateResponse & response) [&](const CreateResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Error (create) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Error (create) " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Created path: " << response.path_created << '\n'; std::cerr << "Created path: " << response.path_created << '\n';
@ -53,19 +53,19 @@ try
std::cout << "get\n"; std::cout << "get\n";
zk.get("/test", zk.get("/test",
[&](const ZooKeeper::GetResponse & response) [&](const GetResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Error (get) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Error (get) " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Value: " << response.data << '\n'; std::cerr << "Value: " << response.data << '\n';
//event.set(); //event.set();
}, },
[](const ZooKeeper::WatchResponse & response) [](const WatchResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Watch (get) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Watch (get) on /test, Error " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n'; std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n';
}); });
@ -75,10 +75,10 @@ try
std::cout << "set\n"; std::cout << "set\n";
zk.set("/test", "new", -1, zk.set("/test", "new", -1,
[&](const ZooKeeper::SetResponse & response) [&](const SetResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Error (set) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Error (set) " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Set\n"; std::cerr << "Set\n";
@ -90,10 +90,10 @@ try
std::cout << "list\n"; std::cout << "list\n";
zk.list("/", zk.list("/",
[&](const ZooKeeper::ListResponse & response) [&](const ListResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Error (list) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Error (list) " << response.error << ": " << errorMessage(response.error) << '\n';
else else
{ {
std::cerr << "Children:\n"; std::cerr << "Children:\n";
@ -103,10 +103,10 @@ try
//event.set(); //event.set();
}, },
[](const ZooKeeper::WatchResponse & response) [](const WatchResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Watch (list) on /, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Watch (list) on /, Error " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n'; std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n';
}); });
@ -116,19 +116,19 @@ try
std::cout << "exists\n"; std::cout << "exists\n";
zk.exists("/test", zk.exists("/test",
[&](const ZooKeeper::ExistsResponse & response) [&](const ExistsResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Error (exists) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Error (exists) " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Exists\n"; std::cerr << "Exists\n";
//event.set(); //event.set();
}, },
[](const ZooKeeper::WatchResponse & response) [](const WatchResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Watch (exists) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Watch (exists) on /test, Error " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n'; std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n';
}); });
@ -137,10 +137,10 @@ try
std::cout << "remove\n"; std::cout << "remove\n";
zk.remove("/test", -1, [&](const ZooKeeper::RemoveResponse & response) zk.remove("/test", -1, [&](const RemoveResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Error (remove) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; std::cerr << "Error (remove) " << response.error << ": " << errorMessage(response.error) << '\n';
else else
std::cerr << "Removed\n"; std::cerr << "Removed\n";
@ -151,43 +151,43 @@ try
std::cout << "multi\n"; std::cout << "multi\n";
ZooKeeper::Requests ops; Requests ops;
{ {
ZooKeeper::CreateRequest create_request; CreateRequest create_request;
create_request.path = "/test"; create_request.path = "/test";
create_request.data = "multi1"; create_request.data = "multi1";
ops.emplace_back(std::make_shared<ZooKeeper::CreateRequest>(std::move(create_request))); ops.emplace_back(std::make_shared<CreateRequest>(std::move(create_request)));
} }
{ {
ZooKeeper::SetRequest set_request; SetRequest set_request;
set_request.path = "/test"; set_request.path = "/test";
set_request.data = "multi2"; set_request.data = "multi2";
ops.emplace_back(std::make_shared<ZooKeeper::SetRequest>(std::move(set_request))); ops.emplace_back(std::make_shared<SetRequest>(std::move(set_request)));
} }
{ {
ZooKeeper::RemoveRequest remove_request; RemoveRequest remove_request;
remove_request.path = "/test"; remove_request.path = "/test";
ops.emplace_back(std::make_shared<ZooKeeper::RemoveRequest>(std::move(remove_request))); ops.emplace_back(std::make_shared<RemoveRequest>(std::move(remove_request)));
} }
zk.multi(ops, [&](const ZooKeeper::MultiResponse & response) zk.multi(ops, [&](const MultiResponse & response)
{
if (response.error)
std::cerr << "Error (multi) " << response.error << ": " << errorMessage(response.error) << '\n';
else
{ {
if (response.error) for (const auto & elem : response.responses)
std::cerr << "Error (multi) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; if (elem->error)
else std::cerr << "Error (elem) " << elem->error << ": " << errorMessage(elem->error) << '\n';
{
for (const auto & elem : response.responses)
if (elem->error)
std::cerr << "Error (elem) " << elem->error << ": " << ZooKeeper::errorMessage(elem->error) << '\n';
std::cerr << "Created path: " << typeid_cast<const ZooKeeper::CreateResponse &>(*response.responses[0]).path_created << '\n'; std::cerr << "Created path: " << dynamic_cast<const CreateResponse &>(*response.responses[0]).path_created << '\n';
} }
event.set(); event.set();
}); });
event.wait(); event.wait();
return 0; return 0;

View File

@ -5,12 +5,12 @@
int main() int main()
try try
{ {
ZooKeeperImpl::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000}); Coordination::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000});
zookeeper.create("/test", "hello", false, false, {}, [](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) zookeeper.create("/test", "hello", false, false, {}, [](const Coordination::CreateResponse & response)
{ {
if (response.error) if (response.error)
std::cerr << "Error " << response.error << ": " << ZooKeeperImpl::ZooKeeper::errorMessage(response.error) << "\n"; std::cerr << "Error " << response.error << ": " << Coordination::errorMessage(response.error) << "\n";
else else
std::cerr << "Path created: " << response.path_created << "\n"; std::cerr << "Path created: " << response.path_created << "\n";
}); });

View File

@ -145,9 +145,9 @@ void BackgroundSchedulePool::TaskInfo::scheduleImpl(std::lock_guard<std::mutex>
pool.queue.enqueueNotification(new TaskNotification(shared_from_this())); pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
} }
zkutil::WatchCallback BackgroundSchedulePool::TaskInfo::getWatchCallback() Coordination::WatchCallback BackgroundSchedulePool::TaskInfo::getWatchCallback()
{ {
return [t = shared_from_this()](const ZooKeeperImpl::ZooKeeper::WatchResponse &) return [t = shared_from_this()](const Coordination::WatchResponse &)
{ {
t->schedule(); t->schedule();
}; };

View File

@ -58,8 +58,8 @@ public:
/// Atomically activate task and schedule it for execution. /// Atomically activate task and schedule it for execution.
bool activateAndSchedule(); bool activateAndSchedule();
/// get zkutil::WatchCallback needed for notifications from ZooKeeper watches. /// get Coordination::WatchCallback needed for notifications from ZooKeeper watches.
zkutil::WatchCallback getWatchCallback(); Coordination::WatchCallback getWatchCallback();
private: private:
friend class TaskNotification; friend class TaskNotification;

View File

@ -545,19 +545,19 @@ void DDLWorker::processTask(DDLTask & task)
String finished_node_path = task.entry_path + "/finished/" + task.host_id_str; String finished_node_path = task.entry_path + "/finished/" + task.host_id_str;
auto code = zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy); auto code = zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy);
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
{ {
// Ok // Ok
} }
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) else if (code == Coordination::ZNONODE)
{ {
/// There is no parent /// There is no parent
createStatusDirs(task.entry_path); createStatusDirs(task.entry_path);
if (ZooKeeperImpl::ZooKeeper::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy)) if (Coordination::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy))
throw zkutil::KeeperException(code, active_node_path); throw Coordination::Exception(code, active_node_path);
} }
else else
throw zkutil::KeeperException(code, active_node_path); throw Coordination::Exception(code, active_node_path);
if (!task.was_executed) if (!task.was_executed)
{ {
@ -578,7 +578,7 @@ void DDLWorker::processTask(DDLTask & task)
tryExecuteQuery(rewritten_query, task, task.execution_status); tryExecuteQuery(rewritten_query, task, task.execution_status);
} }
} }
catch (const zkutil::KeeperException &) catch (const Coordination::Exception &)
{ {
throw; throw;
} }
@ -594,7 +594,7 @@ void DDLWorker::processTask(DDLTask & task)
/// FIXME: if server fails right here, the task will be executed twice. We need WAL here. /// FIXME: if server fails right here, the task will be executed twice. We need WAL here.
/// Delete active flag and create finish flag /// Delete active flag and create finish flag
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(active_node_path, -1)); ops.emplace_back(zkutil::makeRemoveRequest(active_node_path, -1));
ops.emplace_back(zkutil::makeCreateRequest(finished_node_path, task.execution_status.serializeText(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(finished_node_path, task.execution_status.serializeText(), zkutil::CreateMode::Persistent));
zookeeper->multi(ops); zookeeper->multi(ops);
@ -734,7 +734,7 @@ void DDLWorker::cleanupQueue()
String node_path = queue_dir + "/" + node_name; String node_path = queue_dir + "/" + node_name;
String lock_path = node_path + "/lock"; String lock_path = node_path + "/lock";
zkutil::Stat stat; Coordination::Stat stat;
String dummy; String dummy;
try try
@ -784,7 +784,7 @@ void DDLWorker::cleanupQueue()
} }
/// Remove the lock node and its parent atomically /// Remove the lock node and its parent atomically
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(lock_path, -1)); ops.emplace_back(zkutil::makeRemoveRequest(lock_path, -1));
ops.emplace_back(zkutil::makeRemoveRequest(node_path, -1)); ops.emplace_back(zkutil::makeRemoveRequest(node_path, -1));
zookeeper->multi(ops); zookeeper->multi(ops);
@ -803,21 +803,21 @@ void DDLWorker::cleanupQueue()
/// Try to create nonexisting "status" dirs for a node /// Try to create nonexisting "status" dirs for a node
void DDLWorker::createStatusDirs(const std::string & node_path) void DDLWorker::createStatusDirs(const std::string & node_path)
{ {
zkutil::Requests ops; Coordination::Requests ops;
{ {
zkutil::CreateRequest request; Coordination::CreateRequest request;
request.path = node_path + "/active"; request.path = node_path + "/active";
ops.emplace_back(std::make_shared<zkutil::CreateRequest>(std::move(request))); ops.emplace_back(std::make_shared<Coordination::CreateRequest>(std::move(request)));
} }
{ {
zkutil::CreateRequest request; Coordination::CreateRequest request;
request.path = node_path + "/finished"; request.path = node_path + "/finished";
ops.emplace_back(std::make_shared<zkutil::CreateRequest>(std::move(request))); ops.emplace_back(std::make_shared<Coordination::CreateRequest>(std::move(request)));
} }
zkutil::Responses responses; Coordination::Responses responses;
int code = zookeeper->tryMulti(ops, responses); int code = zookeeper->tryMulti(ops, responses);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code && code != Coordination::ZNODEEXISTS)
throw zkutil::KeeperException(code); throw Coordination::Exception(code);
} }
@ -862,9 +862,9 @@ void DDLWorker::run()
zookeeper->createAncestors(queue_dir + "/"); zookeeper->createAncestors(queue_dir + "/");
initialized = true; initialized = true;
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
if (!ZooKeeperImpl::ZooKeeper::isHardwareError(e.code)) if (!Coordination::isHardwareError(e.code))
throw; throw;
} }
} }
@ -890,9 +890,9 @@ void DDLWorker::run()
/// TODO: it might delay the execution, move it to separate thread. /// TODO: it might delay the execution, move it to separate thread.
cleanupQueue(); cleanupQueue();
} }
catch (zkutil::KeeperException & e) catch (Coordination::Exception & e)
{ {
if (ZooKeeperImpl::ZooKeeper::isHardwareError(e.code)) if (Coordination::isHardwareError(e.code))
{ {
LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false)); LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false));
@ -1066,8 +1066,8 @@ private:
{ {
Strings res; Strings res;
int code = zookeeper->tryGetChildren(node_path, res); int code = zookeeper->tryGetChildren(node_path, res);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE) if (code && code != Coordination::ZNONODE)
throw zkutil::KeeperException(code, node_path); throw Coordination::Exception(code, node_path);
return res; return res;
} }

View File

@ -12,7 +12,7 @@ namespace ErrorCodes
} }
EphemeralLockInZooKeeper::EphemeralLockInZooKeeper( EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(
const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Requests * precheck_ops) const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, Coordination::Requests * precheck_ops)
: zookeeper(&zookeeper_), path_prefix(path_prefix_) : zookeeper(&zookeeper_), path_prefix(path_prefix_)
{ {
/// The /abandonable_lock- name is for backward compatibility. /// The /abandonable_lock- name is for backward compatibility.
@ -26,8 +26,8 @@ EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(
else else
{ {
precheck_ops->emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential)); precheck_ops->emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential));
zkutil::Responses op_results = zookeeper->multi(*precheck_ops); Coordination::Responses op_results = zookeeper->multi(*precheck_ops);
holder_path = dynamic_cast<const zkutil::CreateResponse &>(*op_results.back()).path_created; holder_path = dynamic_cast<const Coordination::CreateResponse &>(*op_results.back()).path_created;
} }
/// Write the path to the secondary node in the main node. /// Write the path to the secondary node in the main node.
@ -39,13 +39,13 @@ EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(
void EphemeralLockInZooKeeper::unlock() void EphemeralLockInZooKeeper::unlock()
{ {
zkutil::Requests ops; Coordination::Requests ops;
getUnlockOps(ops); getUnlockOps(ops);
zookeeper->multi(ops); zookeeper->multi(ops);
holder_path = ""; holder_path = "";
} }
void EphemeralLockInZooKeeper::getUnlockOps(zkutil::Requests & ops) void EphemeralLockInZooKeeper::getUnlockOps(Coordination::Requests & ops)
{ {
checkCreated(); checkCreated();
ops.emplace_back(zkutil::makeRemoveRequest(path, -1)); ops.emplace_back(zkutil::makeRemoveRequest(path, -1));
@ -76,12 +76,12 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions(
std::vector<String> holders; std::vector<String> holders;
while (true) while (true)
{ {
zkutil::Stat partitions_stat; Coordination::Stat partitions_stat;
Strings partitions = zookeeper.getChildren(block_numbers_path, &partitions_stat); Strings partitions = zookeeper.getChildren(block_numbers_path, &partitions_stat);
if (holders.size() < partitions.size()) if (holders.size() < partitions.size())
{ {
std::vector<std::future<zkutil::CreateResponse>> holder_futures; std::vector<std::future<Coordination::CreateResponse>> holder_futures;
for (size_t i = 0; i < partitions.size() - holders.size(); ++i) for (size_t i = 0; i < partitions.size() - holders.size(); ++i)
{ {
String path = temp_path + "/abandonable_lock-"; String path = temp_path + "/abandonable_lock-";
@ -94,7 +94,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions(
} }
} }
zkutil::Requests lock_ops; Coordination::Requests lock_ops;
for (size_t i = 0; i < partitions.size(); ++i) for (size_t i = 0; i < partitions.size(); ++i)
{ {
String partition_path_prefix = block_numbers_path + "/" + partitions[i] + "/" + path_prefix; String partition_path_prefix = block_numbers_path + "/" + partitions[i] + "/" + path_prefix;
@ -103,21 +103,21 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions(
} }
lock_ops.push_back(zkutil::makeCheckRequest(block_numbers_path, partitions_stat.version)); lock_ops.push_back(zkutil::makeCheckRequest(block_numbers_path, partitions_stat.version));
zkutil::Responses lock_responses; Coordination::Responses lock_responses;
int rc = zookeeper.tryMulti(lock_ops, lock_responses); int rc = zookeeper.tryMulti(lock_ops, lock_responses);
if (rc == ZooKeeperImpl::ZooKeeper::ZBADVERSION) if (rc == Coordination::ZBADVERSION)
{ {
LOG_TRACE(&Logger::get("EphemeralLocksInAllPartitions"), LOG_TRACE(&Logger::get("EphemeralLocksInAllPartitions"),
"Someone has inserted a block in a new partition while we were creating locks. Retry."); "Someone has inserted a block in a new partition while we were creating locks. Retry.");
continue; continue;
} }
else if (rc != ZooKeeperImpl::ZooKeeper::ZOK) else if (rc != Coordination::ZOK)
throw zkutil::KeeperException(rc); throw Coordination::Exception(rc);
for (size_t i = 0; i < partitions.size(); ++i) for (size_t i = 0; i < partitions.size(); ++i)
{ {
size_t prefix_size = block_numbers_path.size() + 1 + partitions[i].size() + 1 + path_prefix.size(); size_t prefix_size = block_numbers_path.size() + 1 + partitions[i].size() + 1 + path_prefix.size();
const String & path = static_cast<const zkutil::CreateResponse &>(*lock_responses[i]).path_created; const String & path = dynamic_cast<const Coordination::CreateResponse &>(*lock_responses[i]).path_created;
if (path.size() <= prefix_size) if (path.size() <= prefix_size)
throw Exception("Logical error: name of the sequential node is shorter than prefix.", throw Exception("Logical error: name of the sequential node is shorter than prefix.",
ErrorCodes::LOGICAL_ERROR); ErrorCodes::LOGICAL_ERROR);
@ -135,7 +135,7 @@ void EphemeralLocksInAllPartitions::unlock()
std::vector<zkutil::ZooKeeper::FutureMulti> futures; std::vector<zkutil::ZooKeeper::FutureMulti> futures;
for (const auto & lock : locks) for (const auto & lock : locks)
{ {
zkutil::Requests unlock_ops; Coordination::Requests unlock_ops;
unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.path, -1)); unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.path, -1));
unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.holder_path, -1)); unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.holder_path, -1));
futures.push_back(zookeeper.asyncMulti(unlock_ops)); futures.push_back(zookeeper.asyncMulti(unlock_ops));

View File

@ -21,7 +21,7 @@ class EphemeralLockInZooKeeper : public boost::noncopyable
{ {
public: public:
EphemeralLockInZooKeeper( EphemeralLockInZooKeeper(
const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Requests * precheck_ops = nullptr); const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, Coordination::Requests * precheck_ops = nullptr);
EphemeralLockInZooKeeper() = default; EphemeralLockInZooKeeper() = default;
@ -61,7 +61,7 @@ public:
void unlock(); void unlock();
/// Adds actions equivalent to `unlock()` to the list. /// Adds actions equivalent to `unlock()` to the list.
void getUnlockOps(zkutil::Requests & ops); void getUnlockOps(Coordination::Requests & ops);
/// Do not delete nodes in destructor. You may call this method after 'getUnlockOps' and successful execution of these ops, /// Do not delete nodes in destructor. You may call this method after 'getUnlockOps' and successful execution of these ops,
/// because the nodes will be already deleted. /// because the nodes will be already deleted.

View File

@ -53,7 +53,7 @@ void ReplicatedMergeTreeAlterThread::run()
auto zookeeper = storage.getZooKeeper(); auto zookeeper = storage.getZooKeeper();
zkutil::Stat stat; Coordination::Stat stat;
const String columns_str = zookeeper->getWatch(storage.zookeeper_path + "/columns", &stat, task->getWatchCallback()); const String columns_str = zookeeper->getWatch(storage.zookeeper_path + "/columns", &stat, task->getWatchCallback());
auto columns_in_zk = ColumnsDescription::parse(columns_str); auto columns_in_zk = ColumnsDescription::parse(columns_str);
@ -139,7 +139,7 @@ void ReplicatedMergeTreeAlterThread::run()
++changed_parts; ++changed_parts;
/// Update part metadata in ZooKeeper. /// Update part metadata in ZooKeeper.
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeSetRequest( ops.emplace_back(zkutil::makeSetRequest(
storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1)); storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
ops.emplace_back(zkutil::makeSetRequest( ops.emplace_back(zkutil::makeSetRequest(
@ -151,10 +151,10 @@ void ReplicatedMergeTreeAlterThread::run()
{ {
zookeeper->multi(ops); zookeeper->multi(ops);
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
/// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally. /// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally.
if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (e.code == Coordination::ZNONODE)
storage.enqueuePartForCheck(part->name); storage.enqueuePartForCheck(part->name);
throw; throw;
@ -184,11 +184,11 @@ void ReplicatedMergeTreeAlterThread::run()
/// It's important that parts and merge_blocker are destroyed before the wait. /// It's important that parts and merge_blocker are destroyed before the wait.
} }
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == Coordination::ZSESSIONEXPIRED)
return; return;
force_recheck_parts = true; force_recheck_parts = true;

View File

@ -62,13 +62,13 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe
{ {
quorum_info.status_path = storage.zookeeper_path + "/quorum/status"; quorum_info.status_path = storage.zookeeper_path + "/quorum/status";
std::future<zkutil::GetResponse> quorum_status_future = zookeeper->asyncTryGet(quorum_info.status_path); std::future<Coordination::GetResponse> quorum_status_future = zookeeper->asyncTryGet(quorum_info.status_path);
std::future<zkutil::GetResponse> is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active"); std::future<Coordination::GetResponse> is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active");
std::future<zkutil::GetResponse> host_future = zookeeper->asyncTryGet(storage.replica_path + "/host"); std::future<Coordination::GetResponse> host_future = zookeeper->asyncTryGet(storage.replica_path + "/host");
/// List of live replicas. All of them register an ephemeral node for leader_election. /// List of live replicas. All of them register an ephemeral node for leader_election.
zkutil::Stat leader_election_stat; Coordination::Stat leader_election_stat;
zookeeper->get(storage.zookeeper_path + "/leader_election", &leader_election_stat); zookeeper->get(storage.zookeeper_path + "/leader_election", &leader_election_stat);
if (leader_election_stat.numChildren < static_cast<int32_t>(quorum)) if (leader_election_stat.numChildren < static_cast<int32_t>(quorum))
@ -85,7 +85,7 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe
*/ */
auto quorum_status = quorum_status_future.get(); auto quorum_status = quorum_status_future.get();
if (quorum_status.error != ZooKeeperImpl::ZooKeeper::ZNONODE) if (quorum_status.error != Coordination::ZNONODE)
throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.data, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.data, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE);
/// Both checks are implicitly made also later (otherwise there would be a race condition). /// Both checks are implicitly made also later (otherwise there would be a race condition).
@ -93,7 +93,7 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe
auto is_active = is_active_future.get(); auto is_active = is_active_future.get();
auto host = host_future.get(); auto host = host_future.get();
if (is_active.error == ZooKeeperImpl::ZooKeeper::ZNONODE || host.error == ZooKeeperImpl::ZooKeeper::ZNONODE) if (is_active.error == Coordination::ZNONODE || host.error == Coordination::ZNONODE)
throw Exception("Replica is not active right now", ErrorCodes::READONLY); throw Exception("Replica is not active right now", ErrorCodes::READONLY);
quorum_info.is_active_node_value = is_active.data; quorum_info.is_active_node_value = is_active.data;
@ -243,7 +243,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. /// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock.
/// Information about the part. /// Information about the part.
zkutil::Requests ops; Coordination::Requests ops;
storage.getCommitPartOps(ops, part, block_id_path); storage.getCommitPartOps(ops, part, block_id_path);
@ -298,10 +298,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
MergeTreeData::Transaction transaction; /// If you can not add a part to ZK, we'll remove it back from the working set. MergeTreeData::Transaction transaction; /// If you can not add a part to ZK, we'll remove it back from the working set.
storage.data.renameTempPartAndAdd(part, nullptr, &transaction); storage.data.renameTempPartAndAdd(part, nullptr, &transaction);
zkutil::Responses responses; Coordination::Responses responses;
int32_t multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT int32_t multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT
if (multi_code == ZooKeeperImpl::ZooKeeper::ZOK) if (multi_code == Coordination::ZOK)
{ {
transaction.commit(); transaction.commit();
storage.merge_selecting_task->schedule(); storage.merge_selecting_task->schedule();
@ -309,9 +309,9 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// Lock nodes have been already deleted, do not delete them in destructor /// Lock nodes have been already deleted, do not delete them in destructor
block_number_lock->assumeUnlocked(); block_number_lock->assumeUnlocked();
} }
else if (multi_code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS else if (multi_code == Coordination::ZCONNECTIONLOSS
|| multi_code == ZooKeeperImpl::ZooKeeper::ZOPERATIONTIMEOUT || multi_code == Coordination::ZOPERATIONTIMEOUT
|| multi_code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) || multi_code == Coordination::ZSESSIONEXPIRED)
{ {
/** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part /** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part
* if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again.
@ -322,14 +322,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER);
/// We do not know whether or not data has been inserted. /// We do not know whether or not data has been inserted.
throw Exception("Unknown status, client must retry. Reason: " + String(ZooKeeperImpl::ZooKeeper::errorMessage(multi_code)), throw Exception("Unknown status, client must retry. Reason: " + String(Coordination::errorMessage(multi_code)),
ErrorCodes::UNKNOWN_STATUS_OF_INSERT); ErrorCodes::UNKNOWN_STATUS_OF_INSERT);
} }
else if (ZooKeeperImpl::ZooKeeper::isUserError(multi_code)) else if (Coordination::isUserError(multi_code))
{ {
String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp(); String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp();
if (multi_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) if (multi_code == Coordination::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path)
{ {
/// Block with the same id have just appeared in table (or other replica), rollback thee insertion. /// Block with the same id have just appeared in table (or other replica), rollback thee insertion.
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")"); LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")");
@ -339,7 +339,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
last_block_is_duplicate = true; last_block_is_duplicate = true;
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
} }
else if (multi_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && failed_op_path == quorum_info.status_path) else if (multi_code == Coordination::ZNODEEXISTS && failed_op_path == quorum_info.status_path)
{ {
transaction.rollback(); transaction.rollback();
@ -354,7 +354,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
} }
} }
else if (ZooKeeperImpl::ZooKeeper::isHardwareError(multi_code)) else if (Coordination::isHardwareError(multi_code))
{ {
transaction.rollback(); transaction.rollback();
throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': "

View File

@ -32,11 +32,11 @@ void ReplicatedMergeTreeCleanupThread::run()
{ {
iterate(); iterate();
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == Coordination::ZSESSIONEXPIRED)
return; return;
} }
catch (...) catch (...)
@ -68,7 +68,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs()
{ {
auto zookeeper = storage.getZooKeeper(); auto zookeeper = storage.getZooKeeper();
zkutil::Stat stat; Coordination::Stat stat;
if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat)) if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat))
throw Exception(storage.zookeeper_path + "/log doesn't exist", ErrorCodes::NOT_FOUND_NODE); throw Exception(storage.zookeeper_path + "/log doesn't exist", ErrorCodes::NOT_FOUND_NODE);
@ -99,7 +99,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs()
if (entries.empty()) if (entries.empty())
return; return;
zkutil::Requests ops; Coordination::Requests ops;
for (size_t i = 0; i < entries.size(); ++i) for (size_t i = 0; i < entries.size(); ++i)
{ {
ops.emplace_back(zkutil::makeRemoveRequest(storage.zookeeper_path + "/log/" + entries[i], -1)); ops.emplace_back(zkutil::makeRemoveRequest(storage.zookeeper_path + "/log/" + entries[i], -1));
@ -152,7 +152,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime); auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime);
auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold); auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold);
zkutil::AsyncResponses<zkutil::RemoveResponse> try_remove_futures; zkutil::AsyncResponses<Coordination::RemoveResponse> try_remove_futures;
for (auto it = first_outdated_block; it != timed_blocks.end(); ++it) for (auto it = first_outdated_block; it != timed_blocks.end(); ++it)
{ {
String path = storage.zookeeper_path + "/blocks/" + it->node; String path = storage.zookeeper_path + "/blocks/" + it->node;
@ -163,7 +163,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
{ {
const String & path = pair.first; const String & path = pair.first;
int32_t rc = pair.second.get().error; int32_t rc = pair.second.get().error;
if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) if (rc == Coordination::ZNOTEMPTY)
{ {
/// Can happen if there are leftover block nodes with children created by previous server versions. /// Can happen if there are leftover block nodes with children created by previous server versions.
zookeeper->removeRecursive(path); zookeeper->removeRecursive(path);
@ -184,7 +184,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
timed_blocks.clear(); timed_blocks.clear();
Strings blocks; Strings blocks;
zkutil::Stat stat; Coordination::Stat stat;
if (zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat)) if (zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat))
throw Exception(storage.zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); throw Exception(storage.zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);
@ -207,7 +207,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
<< " to clear old ones from ZooKeeper."); << " to clear old ones from ZooKeeper.");
} }
zkutil::AsyncResponses<zkutil::ExistsResponse> exists_futures; zkutil::AsyncResponses<Coordination::ExistsResponse> exists_futures;
for (const String & block : blocks) for (const String & block : blocks)
{ {
auto it = cached_block_stats.find(block); auto it = cached_block_stats.find(block);
@ -227,7 +227,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
for (auto & elem : exists_futures) for (auto & elem : exists_futures)
{ {
auto status = elem.second.get(); auto status = elem.second.get();
if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE) if (status.error != Coordination::ZNONODE)
{ {
cached_block_stats.emplace(elem.first, status.stat.ctime); cached_block_stats.emplace(elem.first, status.stat.ctime);
timed_blocks.emplace_back(elem.first, status.stat.ctime); timed_blocks.emplace_back(elem.first, status.stat.ctime);
@ -252,7 +252,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations()
auto zookeeper = storage.getZooKeeper(); auto zookeeper = storage.getZooKeeper();
zkutil::Stat replicas_stat; Coordination::Stat replicas_stat;
Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas", &replicas_stat); Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas", &replicas_stat);
UInt64 min_pointer = std::numeric_limits<UInt64>::max(); UInt64 min_pointer = std::numeric_limits<UInt64>::max();
@ -278,7 +278,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations()
if (entries.empty()) if (entries.empty())
return; return;
zkutil::Requests ops; Coordination::Requests ops;
size_t batch_start_i = 0; size_t batch_start_i = 0;
for (size_t i = 0; i < entries.size(); ++i) for (size_t i = 0; i < entries.size(); ++i)
{ {

View File

@ -197,7 +197,7 @@ String ReplicatedMergeTreeLogEntryData::toString() const
return out.str(); return out.str();
} }
ReplicatedMergeTreeLogEntry::Ptr ReplicatedMergeTreeLogEntry::parse(const String & s, const zkutil::Stat & stat) ReplicatedMergeTreeLogEntry::Ptr ReplicatedMergeTreeLogEntry::parse(const String & s, const Coordination::Stat & stat)
{ {
ReadBufferFromString in(s); ReadBufferFromString in(s);
Ptr res = std::make_shared<ReplicatedMergeTreeLogEntry>(); Ptr res = std::make_shared<ReplicatedMergeTreeLogEntry>();

View File

@ -158,7 +158,7 @@ struct ReplicatedMergeTreeLogEntry : public ReplicatedMergeTreeLogEntryData, std
std::condition_variable execution_complete; /// Awake when currently_executing becomes false. std::condition_variable execution_complete; /// Awake when currently_executing becomes false.
static Ptr parse(const String & s, const zkutil::Stat & stat); static Ptr parse(const String & s, const Coordination::Stat & stat);
}; };
using ReplicatedMergeTreeLogEntryPtr = std::shared_ptr<ReplicatedMergeTreeLogEntry>; using ReplicatedMergeTreeLogEntryPtr = std::shared_ptr<ReplicatedMergeTreeLogEntry>;

View File

@ -341,11 +341,11 @@ void ReplicatedMergeTreePartCheckThread::run()
task->schedule(); task->schedule();
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == Coordination::ZSESSIONEXPIRED)
return; return;
task->scheduleAfter(PART_CHECK_ERROR_SLEEP_MS); task->scheduleAfter(PART_CHECK_ERROR_SLEEP_MS);

View File

@ -56,7 +56,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
LOG_DEBUG(log, "Having " << children.size() << " queue entries to load."); LOG_DEBUG(log, "Having " << children.size() << " queue entries to load.");
std::sort(children.begin(), children.end()); std::sort(children.begin(), children.end());
zkutil::AsyncResponses<zkutil::GetResponse> futures; zkutil::AsyncResponses<Coordination::GetResponse> futures;
futures.reserve(children.size()); futures.reserve(children.size());
for (const String & child : children) for (const String & child : children)
@ -64,7 +64,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
for (auto & future : futures) for (auto & future : futures)
{ {
zkutil::GetResponse res = future.second.get(); Coordination::GetResponse res = future.second.get();
LogEntryPtr entry = LogEntry::parse(res.data, res.stat); LogEntryPtr entry = LogEntry::parse(res.data, res.stat);
entry->znode_name = future.first; entry->znode_name = future.first;
@ -237,7 +237,7 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper(
/// because we update times in ZooKeeper with unlocked mutex, while these times may change. /// because we update times in ZooKeeper with unlocked mutex, while these times may change.
/// Consider it unimportant (for a short time, ZK will have a slightly different time value). /// Consider it unimportant (for a short time, ZK will have a slightly different time value).
zkutil::Requests ops; Coordination::Requests ops;
if (min_unprocessed_insert_time_changed) if (min_unprocessed_insert_time_changed)
ops.emplace_back(zkutil::makeSetRequest( ops.emplace_back(zkutil::makeSetRequest(
@ -249,7 +249,7 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper(
if (!ops.empty()) if (!ops.empty())
{ {
zkutil::Responses responses; Coordination::Responses responses;
auto code = zookeeper->tryMulti(ops, responses); auto code = zookeeper->tryMulti(ops, responses);
if (code) if (code)
@ -356,7 +356,7 @@ bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo &
} }
void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, zkutil::WatchCallback watch_callback) void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback)
{ {
std::lock_guard lock(pull_logs_to_queue_mutex); std::lock_guard lock(pull_logs_to_queue_mutex);
@ -418,7 +418,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
LOG_DEBUG(log, "Pulling " << (end - begin) << " entries to queue: " << *begin << " - " << *last); LOG_DEBUG(log, "Pulling " << (end - begin) << " entries to queue: " << *begin << " - " << *last);
zkutil::AsyncResponses<zkutil::GetResponse> futures; zkutil::AsyncResponses<Coordination::GetResponse> futures;
futures.reserve(end - begin); futures.reserve(end - begin);
for (auto it = begin; it != end; ++it) for (auto it = begin; it != end; ++it)
@ -426,7 +426,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
/// Simultaneously add all new entries to the queue and move the pointer to the log. /// Simultaneously add all new entries to the queue and move the pointer to the log.
zkutil::Requests ops; Coordination::Requests ops;
std::vector<LogEntryPtr> copied_entries; std::vector<LogEntryPtr> copied_entries;
copied_entries.reserve(end - begin); copied_entries.reserve(end - begin);
@ -434,7 +434,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
for (auto & future : futures) for (auto & future : futures)
{ {
zkutil::GetResponse res = future.second.get(); Coordination::GetResponse res = future.second.get();
copied_entries.emplace_back(LogEntry::parse(res.data, res.stat)); copied_entries.emplace_back(LogEntry::parse(res.data, res.stat));
@ -472,7 +472,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
for (size_t i = 0, size = copied_entries.size(); i < size; ++i) for (size_t i = 0, size = copied_entries.size(); i < size; ++i)
{ {
String path_created = dynamic_cast<const zkutil::CreateResponse &>(*responses[i]).path_created; String path_created = dynamic_cast<const Coordination::CreateResponse &>(*responses[i]).path_created;
copied_entries[i]->znode_name = path_created.substr(path_created.find_last_of('/') + 1); copied_entries[i]->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
std::optional<time_t> unused = false; std::optional<time_t> unused = false;
@ -523,7 +523,7 @@ static size_t countPartsToMutate(
} }
void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, zkutil::WatchCallback watch_callback) void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback)
{ {
std::lock_guard lock(update_mutations_mutex); std::lock_guard lock(update_mutations_mutex);
@ -567,7 +567,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, z
LOG_INFO(log, "Loading " + toString(entries_to_load.size()) + " mutation entries: " LOG_INFO(log, "Loading " + toString(entries_to_load.size()) + " mutation entries: "
+ entries_to_load.front() + " - " + entries_to_load.back()); + entries_to_load.front() + " - " + entries_to_load.back());
std::vector<std::future<zkutil::GetResponse>> futures; std::vector<std::future<Coordination::GetResponse>> futures;
for (const String & entry : entries_to_load) for (const String & entry : entries_to_load)
futures.emplace_back(zookeeper->asyncGet(zookeeper_path + "/mutations/" + entry)); futures.emplace_back(zookeeper->asyncGet(zookeeper_path + "/mutations/" + entry));
@ -1340,7 +1340,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
if (!lock_holder_paths.empty()) if (!lock_holder_paths.empty())
{ {
Strings partitions = zookeeper->getChildren(queue.zookeeper_path + "/block_numbers"); Strings partitions = zookeeper->getChildren(queue.zookeeper_path + "/block_numbers");
std::vector<std::future<zkutil::ListResponse>> lock_futures; std::vector<std::future<Coordination::ListResponse>> lock_futures;
for (const String & partition : partitions) for (const String & partition : partitions)
lock_futures.push_back(zookeeper->asyncGetChildren(queue.zookeeper_path + "/block_numbers/" + partition)); lock_futures.push_back(zookeeper->asyncGetChildren(queue.zookeeper_path + "/block_numbers/" + partition));
@ -1349,7 +1349,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
String partition; String partition;
Int64 number; Int64 number;
String zk_path; String zk_path;
std::future<zkutil::GetResponse> contents_future; std::future<Coordination::GetResponse> contents_future;
}; };
std::vector<BlockInfo> block_infos; std::vector<BlockInfo> block_infos;
@ -1372,7 +1372,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
for (BlockInfo & block : block_infos) for (BlockInfo & block : block_infos)
{ {
zkutil::GetResponse resp = block.contents_future.get(); Coordination::GetResponse resp = block.contents_future.get();
if (!resp.error && lock_holder_paths.count(resp.data)) if (!resp.error && lock_holder_paths.count(resp.data))
committing_blocks[block.partition].insert(block.number); committing_blocks[block.partition].insert(block.number);
} }
@ -1380,11 +1380,11 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
queue_.pullLogsToQueue(zookeeper); queue_.pullLogsToQueue(zookeeper);
zkutil::GetResponse quorum_last_part_response = quorum_last_part_future.get(); Coordination::GetResponse quorum_last_part_response = quorum_last_part_future.get();
if (!quorum_last_part_response.error) if (!quorum_last_part_response.error)
last_quorum_part = quorum_last_part_response.data; last_quorum_part = quorum_last_part_response.data;
zkutil::GetResponse quorum_status_response = quorum_status_future.get(); Coordination::GetResponse quorum_status_response = quorum_status_future.get();
if (!quorum_status_response.error) if (!quorum_status_response.error)
{ {
ReplicatedMergeTreeQuorumEntry quorum_status; ReplicatedMergeTreeQuorumEntry quorum_status;

View File

@ -242,11 +242,11 @@ public:
* If there were new entries, notifies storage.queue_task_handle. * If there were new entries, notifies storage.queue_task_handle.
* Additionally loads mutations (so that the set of mutations is always more recent than the queue). * Additionally loads mutations (so that the set of mutations is always more recent than the queue).
*/ */
void pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, zkutil::WatchCallback watch_callback = {}); void pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {});
/// Load new mutation entries. If something new is loaded, schedule storage.merge_selecting_task. /// Load new mutation entries. If something new is loaded, schedule storage.merge_selecting_task.
/// If watch_callback is not empty, will call it when new mutations appear in ZK. /// If watch_callback is not empty, will call it when new mutations appear in ZK.
void updateMutations(zkutil::ZooKeeperPtr zookeeper, zkutil::WatchCallback watch_callback = {}); void updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {});
/** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM). /** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM).
* And also wait for the completion of their execution, if they are now being executed. * And also wait for the completion of their execution, if they are now being executed.

View File

@ -86,7 +86,7 @@ void ReplicatedMergeTreeRestartingThread::run()
{ {
storage.setZooKeeper(storage.context.getZooKeeper()); storage.setZooKeeper(storage.context.getZooKeeper());
} }
catch (const zkutil::KeeperException &) catch (const Coordination::Exception &)
{ {
/// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again. /// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again.
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
@ -196,7 +196,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
{ {
throw; throw;
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString()); LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString());
return false; return false;
@ -218,7 +218,7 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
auto zookeeper = storage.getZooKeeper(); auto zookeeper = storage.getZooKeeper();
Strings failed_parts; Strings failed_parts;
if (zookeeper->tryGetChildren(storage.zookeeper_path + "/quorum/failed_parts", failed_parts) != ZooKeeperImpl::ZooKeeper::ZOK) if (zookeeper->tryGetChildren(storage.zookeeper_path + "/quorum/failed_parts", failed_parts) != Coordination::ZOK)
return; return;
/// Firstly, remove parts from ZooKeeper /// Firstly, remove parts from ZooKeeper
@ -273,23 +273,23 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
* This is possible only when session in ZooKeeper expires. * This is possible only when session in ZooKeeper expires.
*/ */
String data; String data;
zkutil::Stat stat; Coordination::Stat stat;
bool has_is_active = zookeeper->tryGet(is_active_path, data, &stat); bool has_is_active = zookeeper->tryGet(is_active_path, data, &stat);
if (has_is_active && data == active_node_identifier) if (has_is_active && data == active_node_identifier)
{ {
auto code = zookeeper->tryRemove(is_active_path, stat.version); auto code = zookeeper->tryRemove(is_active_path, stat.version);
if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) if (code == Coordination::ZBADVERSION)
throw Exception("Another instance of replica " + storage.replica_path + " was created just now." throw Exception("Another instance of replica " + storage.replica_path + " was created just now."
" You shouldn't run multiple instances of same replica. You need to check configuration files.", " You shouldn't run multiple instances of same replica. You need to check configuration files.",
ErrorCodes::REPLICA_IS_ALREADY_ACTIVE); ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE) if (code && code != Coordination::ZNONODE)
throw zkutil::KeeperException(code, is_active_path); throw Coordination::Exception(code, is_active_path);
} }
/// Simultaneously declare that this replica is active, and update the host. /// Simultaneously declare that this replica is active, and update the host.
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral));
ops.emplace_back(zkutil::makeSetRequest(storage.replica_path + "/host", address.toString(), -1)); ops.emplace_back(zkutil::makeSetRequest(storage.replica_path + "/host", address.toString(), -1));
@ -297,9 +297,9 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
{ {
zookeeper->multi(ops); zookeeper->multi(ops);
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (e.code == Coordination::ZNODEEXISTS)
throw Exception("Replica " + storage.replica_path + " appears to be already active. If you're sure it's not, " throw Exception("Replica " + storage.replica_path + " appears to be already active. If you're sure it's not, "
"try again in a minute or remove znode " + storage.replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE); "try again in a minute or remove znode " + storage.replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);

View File

@ -498,7 +498,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
/// We write metadata of table so that the replicas can check table parameters with them. /// We write metadata of table so that the replicas can check table parameters with them.
String metadata = TableMetadata(data).toString(); String metadata = TableMetadata(data).toString();
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "",
zkutil::CreateMode::Persistent)); zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata, ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata,
@ -520,10 +520,10 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "", ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "",
zkutil::CreateMode::Persistent)); zkutil::CreateMode::Persistent));
zkutil::Responses responses; Coordination::Responses responses;
auto code = zookeeper->tryMulti(ops, responses); auto code = zookeeper->tryMulti(ops, responses);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code && code != Coordination::ZNODEEXISTS)
throw zkutil::KeeperException(code); throw Coordination::Exception(code);
} }
@ -537,7 +537,7 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo
String metadata_str = zookeeper->get(zookeeper_path + "/metadata"); String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
TableMetadata(data).check(metadata_str); TableMetadata(data).check(metadata_str);
zkutil::Stat stat; Coordination::Stat stat;
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &stat)); auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
columns_version = stat.version; columns_version = stat.version;
@ -575,7 +575,7 @@ static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const Strin
time_t res = 0; time_t res = 0;
/// We get creation time of part, if it still exists (was not merged, for example). /// We get creation time of part, if it still exists (was not merged, for example).
zkutil::Stat stat; Coordination::Stat stat;
String unused; String unused;
if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat)) if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat))
res = stat.ctime / 1000; res = stat.ctime / 1000;
@ -591,7 +591,7 @@ void StorageReplicatedMergeTree::createReplica()
LOG_DEBUG(log, "Creating replica " << replica_path); LOG_DEBUG(log, "Creating replica " << replica_path);
/// Create an empty replica. We'll create `columns` node at the end - we'll use it as a sign that replica creation is complete. /// Create an empty replica. We'll create `columns` node at the end - we'll use it as a sign that replica creation is complete.
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/host", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/host", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_pointer", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_pointer", "", zkutil::CreateMode::Persistent));
@ -603,9 +603,9 @@ void StorageReplicatedMergeTree::createReplica()
{ {
zookeeper->multi(ops); zookeeper->multi(ops);
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (e.code == Coordination::ZNODEEXISTS)
throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST); throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST);
throw; throw;
@ -623,7 +623,7 @@ void StorageReplicatedMergeTree::createReplica()
*/ */
String source_replica; String source_replica;
zkutil::Stat stat; Coordination::Stat stat;
zookeeper->exists(replica_path, &stat); zookeeper->exists(replica_path, &stat);
auto my_create_time = stat.czxid; auto my_create_time = stat.czxid;
@ -856,7 +856,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
{ {
LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name); LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name);
zkutil::Requests ops; Coordination::Requests ops;
checkPartChecksumsAndAddCommitOps(zookeeper, part, ops); checkPartChecksumsAndAddCommitOps(zookeeper, part, ops);
zookeeper->multi(ops); zookeeper->multi(ops);
} }
@ -881,7 +881,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name); log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);
/// We assume that this occurs before the queue is loaded (queue.initialize). /// We assume that this occurs before the queue is loaded (queue.initialize).
zkutil::Requests ops; Coordination::Requests ops;
removePartFromZooKeeper(name, ops); removePartFromZooKeeper(name, ops);
ops.emplace_back(zkutil::makeCreateRequest( ops.emplace_back(zkutil::makeCreateRequest(
replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential));
@ -898,7 +898,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper,
const MergeTreeData::DataPartPtr & part, zkutil::Requests & ops, String part_name, NameSet * absent_replicas_paths) const MergeTreeData::DataPartPtr & part, Coordination::Requests & ops, String part_name, NameSet * absent_replicas_paths)
{ {
if (part_name.empty()) if (part_name.empty())
part_name = part->name; part_name = part->name;
@ -913,7 +913,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
for (const String & replica : replicas) for (const String & replica : replicas)
{ {
zkutil::Stat stat_before, stat_after; Coordination::Stat stat_before, stat_after;
String current_part_path = zookeeper_path + "/replicas/" + replica + "/parts/" + part_name; String current_part_path = zookeeper_path + "/replicas/" + replica + "/parts/" + part_name;
String columns_str; String columns_str;
@ -986,7 +986,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
while (true) while (true)
{ {
zkutil::Requests ops; Coordination::Requests ops;
NameSet absent_part_paths_on_replicas; NameSet absent_part_paths_on_replicas;
/// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`. /// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`.
@ -999,7 +999,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
/// Will check that the part did not suddenly appear on skipped replicas /// Will check that the part did not suddenly appear on skipped replicas
if (!absent_part_paths_on_replicas.empty()) if (!absent_part_paths_on_replicas.empty())
{ {
zkutil::Requests new_ops; Coordination::Requests new_ops;
for (const String & part_path : absent_part_paths_on_replicas) for (const String & part_path : absent_part_paths_on_replicas)
{ {
new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent)); new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent));
@ -1021,7 +1021,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
size_t num_check_ops = 2 * absent_part_paths_on_replicas.size(); size_t num_check_ops = 2 * absent_part_paths_on_replicas.size();
size_t failed_op_index = e.failed_op_index; size_t failed_op_index = e.failed_op_index;
if (failed_op_index < num_check_ops && e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (failed_op_index < num_check_ops && e.code == Coordination::ZNODEEXISTS)
{ {
LOG_INFO(log, "The part " << e.getPathForFirstFailedOp() << " on a replica suddenly appeared, will recheck checksums"); LOG_INFO(log, "The part " << e.getPathForFirstFailedOp() << " on a replica suddenly appeared, will recheck checksums");
} }
@ -1488,11 +1488,11 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
zkutil::Requests ops; Coordination::Requests ops;
for (size_t i = 0, size = replicas.size(); i < size; ++i) for (size_t i = 0, size = replicas.size(); i < size; ++i)
{ {
zkutil::Stat stat; Coordination::Stat stat;
String path = zookeeper_path + "/replicas/" + replicas[i] + "/host"; String path = zookeeper_path + "/replicas/" + replicas[i] + "/host";
zookeeper->get(path, &stat); zookeeper->get(path, &stat);
ops.emplace_back(zkutil::makeCheckRequest(path, stat.version)); ops.emplace_back(zkutil::makeCheckRequest(path, stat.version));
@ -1506,7 +1506,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
if (replica.empty()) if (replica.empty())
{ {
zkutil::Stat quorum_stat; Coordination::Stat quorum_stat;
String quorum_path = zookeeper_path + "/quorum/status"; String quorum_path = zookeeper_path + "/quorum/status";
String quorum_str = zookeeper->get(quorum_path, &quorum_stat); String quorum_str = zookeeper->get(quorum_path, &quorum_stat);
ReplicatedMergeTreeQuorumEntry quorum_entry; ReplicatedMergeTreeQuorumEntry quorum_entry;
@ -1531,22 +1531,22 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id)) if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id))
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1)); ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1));
zkutil::Responses responses; Coordination::Responses responses;
auto code = zookeeper->tryMulti(ops, responses); auto code = zookeeper->tryMulti(ops, responses);
if (code == ZooKeeperImpl::ZooKeeper::ZOK) if (code == Coordination::ZOK)
{ {
LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed."); LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed.");
queue.removeFromVirtualParts(part_info); queue.removeFromVirtualParts(part_info);
return true; return true;
} }
else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) else if (code == Coordination::ZBADVERSION || code == Coordination::ZNONODE || code == Coordination::ZNODEEXISTS)
{ {
LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part " LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
<< entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code)); << entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code));
} }
else else
throw zkutil::KeeperException(code); throw Coordination::Exception(code);
} }
else else
{ {
@ -1700,7 +1700,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
continue; continue;
/// Update part metadata in ZooKeeper. /// Update part metadata in ZooKeeper.
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeSetRequest( ops.emplace_back(zkutil::makeSetRequest(
replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1)); replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
ops.emplace_back(zkutil::makeSetRequest( ops.emplace_back(zkutil::makeSetRequest(
@ -2003,7 +2003,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
auto zookeeper = getZooKeeper(); auto zookeeper = getZooKeeper();
MergeTreeData::Transaction transaction; MergeTreeData::Transaction transaction;
zkutil::Requests ops; Coordination::Requests ops;
for (PartDescriptionPtr & part_desc : final_parts) for (PartDescriptionPtr & part_desc : final_parts)
{ {
data.renameTempPartAndReplace(part_desc->res_part, nullptr, &transaction); data.renameTempPartAndReplace(part_desc->res_part, nullptr, &transaction);
@ -2057,11 +2057,11 @@ void StorageReplicatedMergeTree::queueUpdatingTask()
last_queue_update_finish_time.store(time(nullptr)); last_queue_update_finish_time.store(time(nullptr));
queue_update_in_progress = false; queue_update_in_progress = false;
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == Coordination::ZSESSIONEXPIRED)
{ {
restarting_thread.wakeup(); restarting_thread.wakeup();
return; return;
@ -2083,11 +2083,11 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask()
{ {
queue.updateMutations(getZooKeeper(), mutations_updating_task->getWatchCallback()); queue.updateMutations(getZooKeeper(), mutations_updating_task->getWatchCallback());
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == Coordination::ZSESSIONEXPIRED)
return; return;
mutations_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); mutations_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
@ -2282,7 +2282,7 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
bool deduplicate, bool deduplicate,
ReplicatedMergeTreeLogEntryData * out_log_entry) ReplicatedMergeTreeLogEntryData * out_log_entry)
{ {
std::vector<std::future<zkutil::ExistsResponse>> exists_futures; std::vector<std::future<Coordination::ExistsResponse>> exists_futures;
exists_futures.reserve(parts.size()); exists_futures.reserve(parts.size());
for (const auto & part : parts) for (const auto & part : parts)
exists_futures.emplace_back(zookeeper->asyncExists(replica_path + "/parts/" + part->name)); exists_futures.emplace_back(zookeeper->asyncExists(replica_path + "/parts/" + part->name));
@ -2291,7 +2291,7 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
for (size_t i = 0; i < parts.size(); ++i) for (size_t i = 0; i < parts.size(); ++i)
{ {
/// If there is no information about part in ZK, we will not merge it. /// If there is no information about part in ZK, we will not merge it.
if (exists_futures[i].get().error == ZooKeeperImpl::ZooKeeper::ZNONODE) if (exists_futures[i].get().error == Coordination::ZNONODE)
{ {
all_in_zk = false; all_in_zk = false;
@ -2366,7 +2366,7 @@ bool StorageReplicatedMergeTree::createLogEntryToMutatePart(const MergeTreeDataP
} }
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops) void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops)
{ {
String part_path = replica_path + "/parts/" + part_name; String part_path = replica_path + "/parts/" + part_name;
@ -2388,7 +2388,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
log_entry->source_replica = ""; log_entry->source_replica = "";
log_entry->new_part_name = part_name; log_entry->new_part_name = part_name;
zkutil::Requests ops; Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest( ops.emplace_back(zkutil::makeCreateRequest(
replica_path + "/queue/queue-", log_entry->toString(), replica_path + "/queue/queue-", log_entry->toString(),
zkutil::CreateMode::PersistentSequential)); zkutil::CreateMode::PersistentSequential));
@ -2397,7 +2397,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
auto results = zookeeper->multi(ops); auto results = zookeeper->multi(ops);
String path_created = dynamic_cast<const zkutil::CreateResponse &>(*results[0]).path_created; String path_created = dynamic_cast<const Coordination::CreateResponse &>(*results[0]).path_created;
log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1); log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
queue.insert(zookeeper, log_entry); queue.insert(zookeeper, log_entry);
} }
@ -2592,7 +2592,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; const String quorum_last_part_path = zookeeper_path + "/quorum/last_part";
String value; String value;
zkutil::Stat stat; Coordination::Stat stat;
/// If there is no node, then all quorum INSERTs have already reached the quorum, and nothing is needed. /// If there is no node, then all quorum INSERTs have already reached the quorum, and nothing is needed.
while (zookeeper->tryGet(quorum_status_path, value, &stat)) while (zookeeper->tryGet(quorum_status_path, value, &stat))
@ -2612,50 +2612,50 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
{ {
/// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum. /// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum.
zkutil::Requests ops; Coordination::Requests ops;
zkutil::Responses responses; Coordination::Responses responses;
ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version));
ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1)); ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1));
auto code = zookeeper->tryMulti(ops, responses); auto code = zookeeper->tryMulti(ops, responses);
if (code == ZooKeeperImpl::ZooKeeper::ZOK) if (code == Coordination::ZOK)
{ {
break; break;
} }
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) else if (code == Coordination::ZNONODE)
{ {
/// The quorum has already been achieved. /// The quorum has already been achieved.
break; break;
} }
else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) else if (code == Coordination::ZBADVERSION)
{ {
/// Node was updated meanwhile. We must re-read it and repeat all the actions. /// Node was updated meanwhile. We must re-read it and repeat all the actions.
continue; continue;
} }
else else
throw zkutil::KeeperException(code, quorum_status_path); throw Coordination::Exception(code, quorum_status_path);
} }
else else
{ {
/// We update the node, registering there one more replica. /// We update the node, registering there one more replica.
auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version); auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);
if (code == ZooKeeperImpl::ZooKeeper::ZOK) if (code == Coordination::ZOK)
{ {
break; break;
} }
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) else if (code == Coordination::ZNONODE)
{ {
/// The quorum has already been achieved. /// The quorum has already been achieved.
break; break;
} }
else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) else if (code == Coordination::ZBADVERSION)
{ {
/// Node was updated meanwhile. We must re-read it and repeat all the actions. /// Node was updated meanwhile. We must re-read it and repeat all the actions.
continue; continue;
} }
else else
throw zkutil::KeeperException(code, quorum_status_path); throw Coordination::Exception(code, quorum_status_path);
} }
} }
} }
@ -2985,7 +2985,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
int new_columns_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck. int new_columns_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck.
String new_columns_str; String new_columns_str;
zkutil::Stat stat; Coordination::Stat stat;
{ {
/// Just to read current structure. Alter will be done in separate thread. /// Just to read current structure. Alter will be done in separate thread.
@ -3398,7 +3398,7 @@ void StorageReplicatedMergeTree::drop()
/// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line. /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
Strings replicas; Strings replicas;
if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZooKeeperImpl::ZooKeeper::ZOK && replicas.empty()) if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == Coordination::ZOK && replicas.empty())
{ {
LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)"); LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
zookeeper->tryRemoveRecursive(zookeeper_path); zookeeper->tryRemoveRecursive(zookeeper_path);
@ -3452,7 +3452,7 @@ StorageReplicatedMergeTree::allocateBlockNumber(
const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path) const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path)
{ {
/// Lets check for duplicates in advance, to avoid superflous block numbers allocation /// Lets check for duplicates in advance, to avoid superflous block numbers allocation
zkutil::Requests deduplication_check_ops; Coordination::Requests deduplication_check_ops;
if (!zookeeper_block_id_path.empty()) if (!zookeeper_block_id_path.empty())
{ {
deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(zookeeper_block_id_path, "", zkutil::CreateMode::Persistent)); deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(zookeeper_block_id_path, "", zkutil::CreateMode::Persistent));
@ -3464,16 +3464,16 @@ StorageReplicatedMergeTree::allocateBlockNumber(
if (!existsNodeCached(partition_path)) if (!existsNodeCached(partition_path))
{ {
zkutil::Requests ops; Coordination::Requests ops;
ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent)); ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent));
/// We increment data version of the block_numbers node so that it becomes possible /// We increment data version of the block_numbers node so that it becomes possible
/// to check in a ZK transaction that the set of partitions didn't change /// to check in a ZK transaction that the set of partitions didn't change
/// (unfortunately there is no CheckChildren op). /// (unfortunately there is no CheckChildren op).
ops.push_back(zkutil::makeSetRequest(block_numbers_path, "", -1)); ops.push_back(zkutil::makeSetRequest(block_numbers_path, "", -1));
zkutil::Responses responses; Coordination::Responses responses;
int code = zookeeper->tryMulti(ops, responses); int code = zookeeper->tryMulti(ops, responses);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (code && code != Coordination::ZNODEEXISTS)
zkutil::KeeperMultiException::check(code, ops, responses); zkutil::KeeperMultiException::check(code, ops, responses);
} }
@ -3486,12 +3486,12 @@ StorageReplicatedMergeTree::allocateBlockNumber(
} }
catch (const zkutil::KeeperMultiException & e) catch (const zkutil::KeeperMultiException & e)
{ {
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && e.getPathForFirstFailedOp() == zookeeper_block_id_path) if (e.code == Coordination::ZNODEEXISTS && e.getPathForFirstFailedOp() == zookeeper_block_id_path)
return {}; return {};
throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION); throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION); throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
} }
@ -3913,7 +3913,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const
String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer"); String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str); Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
zkutil::Stat stat; Coordination::Stat stat;
zookeeper->get(current_replica_path + "/queue", &stat); zookeeper->get(current_replica_path + "/queue", &stat);
size_t queue_size = stat.numChildren; size_t queue_size = stat.numChildren;
@ -4085,7 +4085,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const
{ {
auto zookeeper = getZooKeeper(); auto zookeeper = getZooKeeper();
zkutil::Stat mutations_stat; Coordination::Stat mutations_stat;
zookeeper->get(mutations_path, &mutations_stat); zookeeper->get(mutations_path, &mutations_stat);
EphemeralLocksInAllPartitions block_number_locks( EphemeralLocksInAllPartitions block_number_locks(
@ -4096,29 +4096,29 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const
entry.create_time = time(nullptr); entry.create_time = time(nullptr);
zkutil::Requests requests; Coordination::Requests requests;
requests.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); requests.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version));
requests.emplace_back(zkutil::makeCreateRequest( requests.emplace_back(zkutil::makeCreateRequest(
mutations_path + "/", entry.toString(), zkutil::CreateMode::PersistentSequential)); mutations_path + "/", entry.toString(), zkutil::CreateMode::PersistentSequential));
zkutil::Responses responses; Coordination::Responses responses;
int32_t rc = zookeeper->tryMulti(requests, responses); int32_t rc = zookeeper->tryMulti(requests, responses);
if (rc == ZooKeeperImpl::ZooKeeper::ZOK) if (rc == Coordination::ZOK)
{ {
const String & path_created = const String & path_created =
static_cast<const zkutil::CreateResponse *>(responses[1].get())->path_created; dynamic_cast<const Coordination::CreateResponse *>(responses[1].get())->path_created;
entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1); entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1);
LOG_TRACE(log, "Created mutation with ID " << entry.znode_name); LOG_TRACE(log, "Created mutation with ID " << entry.znode_name);
break; break;
} }
else if (rc == ZooKeeperImpl::ZooKeeper::ZBADVERSION) else if (rc == Coordination::ZBADVERSION)
{ {
LOG_TRACE(log, "Version conflict when trying to create a mutation node, retrying..."); LOG_TRACE(log, "Version conflict when trying to create a mutation node, retrying...");
continue; continue;
} }
else else
throw zkutil::KeeperException("Unable to create a mutation znode", rc); throw Coordination::Exception("Unable to create a mutation znode", rc);
} }
} }
@ -4235,7 +4235,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(MergeTre
bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries) bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries)
{ {
using MultiFuture = std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>; using MultiFuture = std::future<Coordination::MultiResponse>;
size_t num_tries = 0; size_t num_tries = 0;
bool sucess = false; bool sucess = false;
@ -4254,7 +4254,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St
for (const String & part_name : part_names) for (const String & part_name : part_names)
{ {
zkutil::Requests ops; Coordination::Requests ops;
removePartFromZooKeeper(part_name, ops); removePartFromZooKeeper(part_name, ops);
futures.emplace_back(zookeeper->tryAsyncMulti(ops)); futures.emplace_back(zookeeper->tryAsyncMulti(ops));
@ -4264,23 +4264,23 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St
{ {
auto response = future.get(); auto response = future.get();
if (response.error == 0 || response.error == ZooKeeperImpl::ZooKeeper::ZNONODE) if (response.error == 0 || response.error == Coordination::ZNONODE)
continue; continue;
if (ZooKeeperImpl::ZooKeeper::isHardwareError(response.error)) if (Coordination::isHardwareError(response.error))
{ {
sucess = false; sucess = false;
continue; continue;
} }
throw ZooKeeperImpl::Exception(response.error); throw Coordination::Exception(response.error);
} }
} }
catch (ZooKeeperImpl::Exception & e) catch (Coordination::Exception & e)
{ {
sucess = false; sucess = false;
if (ZooKeeperImpl::ZooKeeper::isHardwareError(e.code)) if (Coordination::isHardwareError(e.code))
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
else else
throw; throw;
@ -4297,7 +4297,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St
void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
NameSet * parts_should_be_retried) NameSet * parts_should_be_retried)
{ {
zkutil::Requests ops; Coordination::Requests ops;
auto it_first_node_in_batch = part_names.cbegin(); auto it_first_node_in_batch = part_names.cbegin();
for (auto it = part_names.cbegin(); it != part_names.cend(); ++it) for (auto it = part_names.cbegin(); it != part_names.cend(); ++it)
@ -4307,26 +4307,26 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr &
auto it_next = std::next(it); auto it_next = std::next(it);
if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend()) if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend())
{ {
zkutil::Responses unused_responses; Coordination::Responses unused_responses;
auto code = zookeeper->tryMultiNoThrow(ops, unused_responses); auto code = zookeeper->tryMultiNoThrow(ops, unused_responses);
ops.clear(); ops.clear();
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (code == Coordination::ZNONODE)
{ {
/// Fallback /// Fallback
LOG_DEBUG(log, "There are no some part nodes in ZooKeeper, will remove part nodes sequentially"); LOG_DEBUG(log, "There are no some part nodes in ZooKeeper, will remove part nodes sequentially");
for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch) for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
{ {
zkutil::Requests cur_ops; Coordination::Requests cur_ops;
removePartFromZooKeeper(*it_in_batch, cur_ops); removePartFromZooKeeper(*it_in_batch, cur_ops);
auto cur_code = zookeeper->tryMultiNoThrow(cur_ops, unused_responses); auto cur_code = zookeeper->tryMultiNoThrow(cur_ops, unused_responses);
if (cur_code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (cur_code == Coordination::ZNONODE)
{ {
LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem"); LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem");
} }
else if (parts_should_be_retried && ZooKeeperImpl::ZooKeeper::isHardwareError(cur_code)) else if (parts_should_be_retried && Coordination::isHardwareError(cur_code))
{ {
parts_should_be_retried->emplace(*it_in_batch); parts_should_be_retried->emplace(*it_in_batch);
} }
@ -4336,7 +4336,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr &
} }
} }
} }
else if (parts_should_be_retried && ZooKeeperImpl::ZooKeeper::isHardwareError(code)) else if (parts_should_be_retried && Coordination::isHardwareError(code))
{ {
for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch) for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
parts_should_be_retried->emplace(*it_in_batch); parts_should_be_retried->emplace(*it_in_batch);
@ -4361,7 +4361,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);
String partition_prefix = partition_id + "_"; String partition_prefix = partition_id + "_";
zkutil::AsyncResponses<zkutil::GetResponse> get_futures; zkutil::AsyncResponses<Coordination::GetResponse> get_futures;
for (const String & block_id : blocks) for (const String & block_id : blocks)
{ {
if (startsWith(block_id, partition_prefix)) if (startsWith(block_id, partition_prefix))
@ -4371,13 +4371,13 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
} }
} }
zkutil::AsyncResponses<zkutil::RemoveResponse> to_delete_futures; zkutil::AsyncResponses<Coordination::RemoveResponse> to_delete_futures;
for (auto & pair : get_futures) for (auto & pair : get_futures)
{ {
const String & path = pair.first; const String & path = pair.first;
auto result = pair.second.get(); auto result = pair.second.get();
if (result.error == ZooKeeperImpl::ZooKeeper::ZNONODE) if (result.error == Coordination::ZNONODE)
continue; continue;
ReadBufferFromString buf(result.data); ReadBufferFromString buf(result.data);
@ -4391,7 +4391,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
{ {
const String & path = pair.first; const String & path = pair.first;
int32_t rc = pair.second.get().error; int32_t rc = pair.second.get().error;
if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) if (rc == Coordination::ZNOTEMPTY)
{ {
/// Can happen if there are leftover block nodes with children created by previous server versions. /// Can happen if there are leftover block nodes with children created by previous server versions.
zookeeper.removeRecursive(path); zookeeper.removeRecursive(path);
@ -4508,11 +4508,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block);
MergeTreeData::DataPartsVector parts_to_remove; MergeTreeData::DataPartsVector parts_to_remove;
zkutil::Responses op_results; Coordination::Responses op_results;
try try
{ {
zkutil::Requests ops; Coordination::Requests ops;
for (size_t i = 0; i < dst_parts.size(); ++i) for (size_t i = 0; i < dst_parts.size(); ++i)
{ {
getCommitPartOps(ops, dst_parts[i], block_id_paths[i]); getCommitPartOps(ops, dst_parts[i], block_id_paths[i]);
@ -4554,7 +4554,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
throw; throw;
} }
String log_znode_path = dynamic_cast<const zkutil::CreateResponse &>(*op_results.back()).path_created; String log_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*op_results.back()).path_created;
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
for (auto & lock : ephemeral_locks) for (auto & lock : ephemeral_locks)
@ -4573,7 +4573,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
} }
void StorageReplicatedMergeTree::getCommitPartOps( void StorageReplicatedMergeTree::getCommitPartOps(
zkutil::Requests & ops, Coordination::Requests & ops,
MergeTreeData::MutableDataPartPtr & part, MergeTreeData::MutableDataPartPtr & part,
const String & block_id_path) const const String & block_id_path) const
{ {

View File

@ -340,7 +340,7 @@ private:
* Call under TableStructureLock. * Call under TableStructureLock.
*/ */
void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const MergeTreeData::DataPartPtr & part, void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const MergeTreeData::DataPartPtr & part,
zkutil::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr); Coordination::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr);
String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const; String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const;
@ -349,12 +349,12 @@ private:
const MergeTreeData::DataPartPtr & part); const MergeTreeData::DataPartPtr & part);
void getCommitPartOps( void getCommitPartOps(
zkutil::Requests & ops, Coordination::Requests & ops,
MergeTreeData::MutableDataPartPtr & part, MergeTreeData::MutableDataPartPtr & part,
const String & block_id_path = "") const; const String & block_id_path = "") const;
/// Adds actions to `ops` that remove a part from ZooKeeper. /// Adds actions to `ops` that remove a part from ZooKeeper.
void removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops); void removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops);
/// Quickly removes big set of parts from ZooKeeper (using async multi queries) /// Quickly removes big set of parts from ZooKeeper (using async multi queries)
void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,

View File

@ -118,7 +118,7 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Contex
if (path_part == "/") if (path_part == "/")
path_part.clear(); path_part.clear();
std::vector<std::future<zkutil::GetResponse>> futures; std::vector<std::future<Coordination::GetResponse>> futures;
futures.reserve(nodes.size()); futures.reserve(nodes.size());
for (const String & node : nodes) for (const String & node : nodes)
futures.push_back(zookeeper->asyncTryGet(path_part + '/' + node)); futures.push_back(zookeeper->asyncTryGet(path_part + '/' + node));
@ -126,10 +126,10 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Contex
for (size_t i = 0, size = nodes.size(); i < size; ++i) for (size_t i = 0, size = nodes.size(); i < size; ++i)
{ {
auto res = futures[i].get(); auto res = futures[i].get();
if (res.error == ZooKeeperImpl::ZooKeeper::ZNONODE) if (res.error == Coordination::ZNONODE)
continue; /// Node was deleted meanwhile. continue; /// Node was deleted meanwhile.
const zkutil::Stat & stat = res.stat; const Coordination::Stat & stat = res.stat;
size_t col_num = 0; size_t col_num = 0;
res_columns[col_num++]->insert(nodes[i]); res_columns[col_num++]->insert(nodes[i]);

View File

@ -54,7 +54,7 @@ try
<< " partitions, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; << " partitions, elapsed: " << stage.elapsedSeconds() << "s." << std::endl;
stage.restart(); stage.restart();
std::vector<std::future<zkutil::ListResponse>> lock_futures; std::vector<std::future<Coordination::ListResponse>> lock_futures;
for (const String & partition : partitions) for (const String & partition : partitions)
lock_futures.push_back(zookeeper->asyncGetChildren(zookeeper_path + "/block_numbers/" + partition)); lock_futures.push_back(zookeeper->asyncGetChildren(zookeeper_path + "/block_numbers/" + partition));
@ -63,7 +63,7 @@ try
String partition; String partition;
Int64 number; Int64 number;
String zk_path; String zk_path;
std::future<zkutil::GetResponse> contents_future; std::future<Coordination::GetResponse> contents_future;
}; };
std::vector<BlockInfo> block_infos; std::vector<BlockInfo> block_infos;
@ -85,7 +85,7 @@ try
size_t total_count = 0; size_t total_count = 0;
for (BlockInfo & block : block_infos) for (BlockInfo & block : block_infos)
{ {
zkutil::GetResponse resp = block.contents_future.get(); Coordination::GetResponse resp = block.contents_future.get();
if (!resp.error && lock_holder_paths.count(resp.data)) if (!resp.error && lock_holder_paths.count(resp.data))
{ {
++total_count; ++total_count;

View File

@ -9,7 +9,7 @@
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
void printStat(const zkutil::Stat & s) void printStat(const Coordination::Stat & s)
{ {
std::cout << "Stat:\n"; std::cout << "Stat:\n";
std::cout << " czxid: " << s.czxid << '\n'; std::cout << " czxid: " << s.czxid << '\n';
@ -148,7 +148,7 @@ int main(int argc, char ** argv)
ss >> w; ss >> w;
bool watch = w == "w"; bool watch = w == "w";
zkutil::EventPtr event = watch ? std::make_shared<Poco::Event>() : nullptr; zkutil::EventPtr event = watch ? std::make_shared<Poco::Event>() : nullptr;
zkutil::Stat stat; Coordination::Stat stat;
bool e = zk.exists(path, &stat, event); bool e = zk.exists(path, &stat, event);
if (e) if (e)
printStat(stat); printStat(stat);
@ -163,7 +163,7 @@ int main(int argc, char ** argv)
ss >> w; ss >> w;
bool watch = w == "w"; bool watch = w == "w";
zkutil::EventPtr event = watch ? std::make_shared<Poco::Event>() : nullptr; zkutil::EventPtr event = watch ? std::make_shared<Poco::Event>() : nullptr;
zkutil::Stat stat; Coordination::Stat stat;
std::string data = zk.get(path, &stat, event); std::string data = zk.get(path, &stat, event);
std::cout << "Data: " << data << std::endl; std::cout << "Data: " << data << std::endl;
printStat(stat); printStat(stat);
@ -187,7 +187,7 @@ int main(int argc, char ** argv)
if (!in.eof()) if (!in.eof())
DB::readText(version, in); DB::readText(version, in);
zkutil::Stat stat; Coordination::Stat stat;
zk.set(path, data, version, &stat); zk.set(path, data, version, &stat);
printStat(stat); printStat(stat);
} }
@ -206,13 +206,13 @@ int main(int argc, char ** argv)
} }
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
std::cerr << "KeeperException: " << e.displayText() << std::endl; std::cerr << "KeeperException: " << e.displayText() << std::endl;
} }
} }
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
std::cerr << "KeeperException: " << e.displayText() << std::endl; std::cerr << "KeeperException: " << e.displayText() << std::endl;
return 1; return 1;

View File

@ -33,19 +33,19 @@ try
std::string initial_path = options.at("path").as<std::string>(); std::string initial_path = options.at("path").as<std::string>();
std::list<std::pair<std::string, std::future<zkutil::ListResponse>>> list_futures; std::list<std::pair<std::string, std::future<Coordination::ListResponse>>> list_futures;
list_futures.emplace_back(initial_path, zookeeper.asyncGetChildren(initial_path)); list_futures.emplace_back(initial_path, zookeeper.asyncGetChildren(initial_path));
for (auto it = list_futures.begin(); it != list_futures.end(); ++it) for (auto it = list_futures.begin(); it != list_futures.end(); ++it)
{ {
zkutil::ListResponse response; Coordination::ListResponse response;
try try
{ {
response = it->second.get(); response = it->second.get();
} }
catch (const zkutil::KeeperException & e) catch (const Coordination::Exception & e)
{ {
if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (e.code == Coordination::ZNONODE)
continue; continue;
throw; throw;
} }

View File

@ -29,7 +29,7 @@ try
zkutil::ZooKeeper zookeeper(options.at("address").as<std::string>()); zkutil::ZooKeeper zookeeper(options.at("address").as<std::string>());
DB::ReadBufferFromFileDescriptor in(STDIN_FILENO); DB::ReadBufferFromFileDescriptor in(STDIN_FILENO);
std::list<std::future<zkutil::RemoveResponse>> futures; std::list<std::future<Coordination::RemoveResponse>> futures;
std::cerr << "Requested: "; std::cerr << "Requested: ";
while (!in.eof()) while (!in.eof())