mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #2950 from yandex/generalize-zookeeper
Generalize zookeeper library
This commit is contained in:
commit
3be0b6ca6d
@ -363,8 +363,8 @@ struct TaskCluster
|
||||
struct MultiTransactionInfo
|
||||
{
|
||||
int32_t code;
|
||||
zkutil::Requests requests;
|
||||
zkutil::Responses responses;
|
||||
Coordination::Requests requests;
|
||||
Coordination::Responses responses;
|
||||
};
|
||||
|
||||
|
||||
@ -373,7 +373,7 @@ struct MultiTransactionInfo
|
||||
static MultiTransactionInfo checkNoNodeAndCommit(
|
||||
const zkutil::ZooKeeperPtr & zookeeper,
|
||||
const String & checking_node_path,
|
||||
zkutil::RequestPtr && op)
|
||||
Coordination::RequestPtr && op)
|
||||
{
|
||||
MultiTransactionInfo info;
|
||||
info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent));
|
||||
@ -742,7 +742,7 @@ public:
|
||||
{
|
||||
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;
|
||||
LOG_DEBUG(log, "Task description should be updated, local version " << version);
|
||||
@ -902,7 +902,7 @@ public:
|
||||
task_description_watch_zookeeper = zookeeper;
|
||||
|
||||
String task_config_str;
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
int code;
|
||||
|
||||
zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code);
|
||||
@ -1052,7 +1052,7 @@ protected:
|
||||
{
|
||||
updateConfigIfNeeded();
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
zookeeper->get(workers_version_path, &stat);
|
||||
auto version = stat.version;
|
||||
zookeeper->get(workers_path, &stat);
|
||||
@ -1070,16 +1070,16 @@ protected:
|
||||
}
|
||||
else
|
||||
{
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral));
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses 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);
|
||||
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
|
||||
if (code == Coordination::ZBADVERSION)
|
||||
{
|
||||
++num_bad_version_errors;
|
||||
|
||||
@ -1093,7 +1093,7 @@ protected:
|
||||
}
|
||||
}
|
||||
else
|
||||
throw zkutil::KeeperException(code);
|
||||
throw Coordination::Exception(code);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1157,7 +1157,7 @@ protected:
|
||||
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
|
||||
<< ". Will recheck the partition. Error: " << e.displayText());
|
||||
@ -1242,9 +1242,9 @@ protected:
|
||||
{
|
||||
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");
|
||||
std::this_thread::sleep_for(default_sleep_time);
|
||||
@ -1254,7 +1254,7 @@ protected:
|
||||
throw;
|
||||
}
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
if (zookeeper->exists(current_partition_active_workers_dir, &stat))
|
||||
{
|
||||
if (stat.numChildren != 0)
|
||||
@ -1291,7 +1291,7 @@ protected:
|
||||
}
|
||||
|
||||
/// Remove the locking node
|
||||
zkutil::Requests requests;
|
||||
Coordination::Requests requests;
|
||||
requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1));
|
||||
requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1));
|
||||
zookeeper->multi(requests);
|
||||
@ -1503,8 +1503,8 @@ protected:
|
||||
auto create_is_dirty_node = [&] ()
|
||||
{
|
||||
auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent);
|
||||
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
|
||||
throw zkutil::KeeperException(code, is_dirty_flag_path);
|
||||
if (code && code != Coordination::ZNODEEXISTS)
|
||||
throw Coordination::Exception(code, is_dirty_flag_path);
|
||||
};
|
||||
|
||||
/// 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);
|
||||
}
|
||||
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);
|
||||
return PartitionTaskStatus::Active;
|
||||
@ -1605,7 +1605,7 @@ protected:
|
||||
|
||||
if (count != 0)
|
||||
{
|
||||
zkutil::Stat stat_shards;
|
||||
Coordination::Stat stat_shards;
|
||||
zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards);
|
||||
|
||||
if (stat_shards.numChildren == 0)
|
||||
@ -1698,7 +1698,7 @@ protected:
|
||||
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);
|
||||
constexpr size_t check_period_milliseconds = 500;
|
||||
@ -1716,9 +1716,9 @@ protected:
|
||||
/// Otherwise, the insertion will slow a little bit
|
||||
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);
|
||||
}
|
||||
|
||||
@ -2023,13 +2023,13 @@ private:
|
||||
/// Auto update config stuff
|
||||
UInt64 task_descprtion_current_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
|
||||
zkutil::ZooKeeperPtr task_description_watch_zookeeper;
|
||||
|
||||
ConfigurationPtr task_cluster_initial_config;
|
||||
ConfigurationPtr task_cluster_current_config;
|
||||
zkutil::Stat task_descprtion_current_stat;
|
||||
Coordination::Stat task_descprtion_current_stat;
|
||||
|
||||
std::unique_ptr<TaskCluster> task_cluster;
|
||||
|
||||
|
@ -540,7 +540,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
|
||||
if (!fallback_to_preprocessed)
|
||||
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)
|
||||
throw;
|
||||
|
||||
|
169
dbms/src/Common/ZooKeeper/IKeeper.cpp
Normal file
169
dbms/src/Common/ZooKeeper/IKeeper.cpp
Normal file
@ -0,0 +1,169 @@
|
||||
#include <string.h>
|
||||
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int KEEPER_EXCEPTION;
|
||||
}
|
||||
}
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ZooKeeperUserExceptions;
|
||||
extern const Event ZooKeeperHardwareExceptions;
|
||||
extern const Event ZooKeeperOtherExceptions;
|
||||
}
|
||||
|
||||
|
||||
namespace Coordination
|
||||
{
|
||||
|
||||
Exception::Exception(const std::string & msg, const int32_t code, int)
|
||||
: DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code)
|
||||
{
|
||||
if (Coordination::isUserError(code))
|
||||
ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions);
|
||||
else if (Coordination::isHardwareError(code))
|
||||
ProfileEvents::increment(ProfileEvents::ZooKeeperHardwareExceptions);
|
||||
else
|
||||
ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions);
|
||||
}
|
||||
|
||||
Exception::Exception(const std::string & msg, const int32_t code)
|
||||
: Exception(msg + " (" + errorMessage(code) + ")", code, 0)
|
||||
{
|
||||
}
|
||||
|
||||
Exception::Exception(const int32_t code)
|
||||
: Exception(errorMessage(code), code, 0)
|
||||
{
|
||||
}
|
||||
|
||||
Exception::Exception(const int32_t code, const std::string & path)
|
||||
: Exception(std::string{errorMessage(code)} + ", path: " + path, code, 0)
|
||||
{
|
||||
}
|
||||
|
||||
Exception::Exception(const Exception & exc)
|
||||
: DB::Exception(exc), code(exc.code)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
using namespace DB;
|
||||
|
||||
|
||||
void addRootPath(String & path, const String & root_path)
|
||||
{
|
||||
if (path.empty())
|
||||
throw Exception("Path cannot be empty", ZBADARGUMENTS);
|
||||
|
||||
if (path[0] != '/')
|
||||
throw Exception("Path must begin with /", ZBADARGUMENTS);
|
||||
|
||||
if (root_path.empty())
|
||||
return;
|
||||
|
||||
if (path.size() == 1) /// "/"
|
||||
path = root_path;
|
||||
else
|
||||
path = root_path + path;
|
||||
}
|
||||
|
||||
void removeRootPath(String & path, const String & root_path)
|
||||
{
|
||||
if (root_path.empty())
|
||||
return;
|
||||
|
||||
if (path.size() <= root_path.size())
|
||||
throw Exception("Received path is not longer than root_path", ZDATAINCONSISTENCY);
|
||||
|
||||
path = path.substr(root_path.size());
|
||||
}
|
||||
|
||||
|
||||
const char * errorMessage(int32_t code)
|
||||
{
|
||||
switch (code)
|
||||
{
|
||||
case ZOK: return "Ok";
|
||||
case ZSYSTEMERROR: return "System error";
|
||||
case ZRUNTIMEINCONSISTENCY: return "Run time inconsistency";
|
||||
case ZDATAINCONSISTENCY: return "Data inconsistency";
|
||||
case ZCONNECTIONLOSS: return "Connection loss";
|
||||
case ZMARSHALLINGERROR: return "Marshalling error";
|
||||
case ZUNIMPLEMENTED: return "Unimplemented";
|
||||
case ZOPERATIONTIMEOUT: return "Operation timeout";
|
||||
case ZBADARGUMENTS: return "Bad arguments";
|
||||
case ZINVALIDSTATE: return "Invalid zhandle state";
|
||||
case ZAPIERROR: return "API error";
|
||||
case ZNONODE: return "No node";
|
||||
case ZNOAUTH: return "Not authenticated";
|
||||
case ZBADVERSION: return "Bad version";
|
||||
case ZNOCHILDRENFOREPHEMERALS: return "No children for ephemerals";
|
||||
case ZNODEEXISTS: return "Node exists";
|
||||
case ZNOTEMPTY: return "Not empty";
|
||||
case ZSESSIONEXPIRED: return "Session expired";
|
||||
case ZINVALIDCALLBACK: return "Invalid callback";
|
||||
case ZINVALIDACL: return "Invalid ACL";
|
||||
case ZAUTHFAILED: return "Authentication failed";
|
||||
case ZCLOSING: return "ZooKeeper is closing";
|
||||
case ZNOTHING: return "(not error) no server responses to process";
|
||||
case ZSESSIONMOVED: return "Session moved to another server, so operation is ignored";
|
||||
}
|
||||
if (code > 0)
|
||||
return strerror(code);
|
||||
|
||||
return "unknown error";
|
||||
}
|
||||
|
||||
bool isHardwareError(int32_t zk_return_code)
|
||||
{
|
||||
return zk_return_code == ZINVALIDSTATE
|
||||
|| zk_return_code == ZSESSIONEXPIRED
|
||||
|| zk_return_code == ZSESSIONMOVED
|
||||
|| zk_return_code == ZCONNECTIONLOSS
|
||||
|| zk_return_code == ZMARSHALLINGERROR
|
||||
|| zk_return_code == ZOPERATIONTIMEOUT;
|
||||
}
|
||||
|
||||
bool isUserError(int32_t zk_return_code)
|
||||
{
|
||||
return zk_return_code == ZNONODE
|
||||
|| zk_return_code == ZBADVERSION
|
||||
|| zk_return_code == ZNOCHILDRENFOREPHEMERALS
|
||||
|| zk_return_code == ZNODEEXISTS
|
||||
|| zk_return_code == ZNOTEMPTY;
|
||||
}
|
||||
|
||||
|
||||
void CreateRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||
void RemoveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||
void ExistsRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||
void GetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||
void SetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||
void ListRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||
void CheckRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||
|
||||
void MultiRequest::addRootPath(const String & root_path)
|
||||
{
|
||||
for (auto & request : requests)
|
||||
request->addRootPath(root_path);
|
||||
}
|
||||
|
||||
void CreateResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path_created, root_path); }
|
||||
void WatchResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path, root_path); }
|
||||
|
||||
void MultiResponse::removeRootPath(const String & root_path)
|
||||
{
|
||||
for (auto & response : responses)
|
||||
response->removeRootPath(root_path);
|
||||
}
|
||||
|
||||
}
|
||||
|
389
dbms/src/Common/ZooKeeper/IKeeper.h
Normal file
389
dbms/src/Common/ZooKeeper/IKeeper.h
Normal file
@ -0,0 +1,389 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
|
||||
/** Generic interface for ZooKeeper-like services.
|
||||
* Possible examples are:
|
||||
* - ZooKeeper client itself;
|
||||
* - fake ZooKeeper client for testing;
|
||||
* - ZooKeeper emulation layer on top of Etcd, FoundationDB, whatever.
|
||||
*/
|
||||
|
||||
|
||||
namespace Coordination
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
|
||||
struct ACL
|
||||
{
|
||||
static constexpr int32_t Read = 1;
|
||||
static constexpr int32_t Write = 2;
|
||||
static constexpr int32_t Create = 4;
|
||||
static constexpr int32_t Delete = 8;
|
||||
static constexpr int32_t Admin = 16;
|
||||
static constexpr int32_t All = 0x1F;
|
||||
|
||||
int32_t permissions;
|
||||
String scheme;
|
||||
String id;
|
||||
};
|
||||
|
||||
using ACLs = std::vector<ACL>;
|
||||
|
||||
struct Stat
|
||||
{
|
||||
int64_t czxid;
|
||||
int64_t mzxid;
|
||||
int64_t ctime;
|
||||
int64_t mtime;
|
||||
int32_t version;
|
||||
int32_t cversion;
|
||||
int32_t aversion;
|
||||
int64_t ephemeralOwner;
|
||||
int32_t dataLength;
|
||||
int32_t numChildren;
|
||||
int64_t pzxid;
|
||||
};
|
||||
|
||||
struct Request;
|
||||
using RequestPtr = std::shared_ptr<Request>;
|
||||
using Requests = std::vector<RequestPtr>;
|
||||
|
||||
struct Request
|
||||
{
|
||||
virtual ~Request() {}
|
||||
virtual String getPath() const = 0;
|
||||
virtual void addRootPath(const String & /* root_path */) {}
|
||||
};
|
||||
|
||||
struct Response;
|
||||
using ResponsePtr = std::shared_ptr<Response>;
|
||||
using Responses = std::vector<ResponsePtr>;
|
||||
using ResponseCallback = std::function<void(const Response &)>;
|
||||
|
||||
struct Response
|
||||
{
|
||||
int32_t error = 0;
|
||||
virtual ~Response() {}
|
||||
virtual void removeRootPath(const String & /* root_path */) {}
|
||||
};
|
||||
|
||||
struct WatchResponse : virtual Response
|
||||
{
|
||||
int32_t type = 0;
|
||||
int32_t state = 0;
|
||||
String path;
|
||||
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
using WatchCallback = std::function<void(const WatchResponse &)>;
|
||||
|
||||
struct CreateRequest : virtual Request
|
||||
{
|
||||
String path;
|
||||
String data;
|
||||
bool is_ephemeral = false;
|
||||
bool is_sequential = false;
|
||||
ACLs acls;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct CreateResponse : virtual Response
|
||||
{
|
||||
String path_created;
|
||||
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
struct RemoveRequest : virtual Request
|
||||
{
|
||||
String path;
|
||||
int32_t version = -1;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct RemoveResponse : virtual Response
|
||||
{
|
||||
};
|
||||
|
||||
struct ExistsRequest : virtual Request
|
||||
{
|
||||
String path;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct ExistsResponse : virtual Response
|
||||
{
|
||||
Stat stat;
|
||||
};
|
||||
|
||||
struct GetRequest : virtual Request
|
||||
{
|
||||
String path;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct GetResponse : virtual Response
|
||||
{
|
||||
String data;
|
||||
Stat stat;
|
||||
};
|
||||
|
||||
struct SetRequest : virtual Request
|
||||
{
|
||||
String path;
|
||||
String data;
|
||||
int32_t version = -1;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct SetResponse : virtual Response
|
||||
{
|
||||
Stat stat;
|
||||
};
|
||||
|
||||
struct ListRequest : virtual Request
|
||||
{
|
||||
String path;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct ListResponse : virtual Response
|
||||
{
|
||||
std::vector<String> names;
|
||||
Stat stat;
|
||||
};
|
||||
|
||||
struct CheckRequest : virtual Request
|
||||
{
|
||||
String path;
|
||||
int32_t version = -1;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct CheckResponse : virtual Response
|
||||
{
|
||||
};
|
||||
|
||||
struct MultiRequest : virtual Request
|
||||
{
|
||||
Requests requests;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct MultiResponse : virtual Response
|
||||
{
|
||||
Responses responses;
|
||||
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
/// This response may be received only as an element of responses in MultiResponse.
|
||||
struct ErrorResponse : virtual Response
|
||||
{
|
||||
};
|
||||
|
||||
|
||||
using CreateCallback = std::function<void(const CreateResponse &)>;
|
||||
using RemoveCallback = std::function<void(const RemoveResponse &)>;
|
||||
using ExistsCallback = std::function<void(const ExistsResponse &)>;
|
||||
using GetCallback = std::function<void(const GetResponse &)>;
|
||||
using SetCallback = std::function<void(const SetResponse &)>;
|
||||
using ListCallback = std::function<void(const ListResponse &)>;
|
||||
using CheckCallback = std::function<void(const CheckResponse &)>;
|
||||
using MultiCallback = std::function<void(const MultiResponse &)>;
|
||||
|
||||
|
||||
enum Error
|
||||
{
|
||||
ZOK = 0,
|
||||
|
||||
/** System and server-side errors.
|
||||
* This is never thrown by the server, it shouldn't be used other than
|
||||
* to indicate a range. Specifically error codes greater than this
|
||||
* value, but lesser than ZAPIERROR, are system errors.
|
||||
*/
|
||||
ZSYSTEMERROR = -1,
|
||||
|
||||
ZRUNTIMEINCONSISTENCY = -2, /// A runtime inconsistency was found
|
||||
ZDATAINCONSISTENCY = -3, /// A data inconsistency was found
|
||||
ZCONNECTIONLOSS = -4, /// Connection to the server has been lost
|
||||
ZMARSHALLINGERROR = -5, /// Error while marshalling or unmarshalling data
|
||||
ZUNIMPLEMENTED = -6, /// Operation is unimplemented
|
||||
ZOPERATIONTIMEOUT = -7, /// Operation timeout
|
||||
ZBADARGUMENTS = -8, /// Invalid arguments
|
||||
ZINVALIDSTATE = -9, /// Invliad zhandle state
|
||||
|
||||
/** API errors.
|
||||
* This is never thrown by the server, it shouldn't be used other than
|
||||
* to indicate a range. Specifically error codes greater than this
|
||||
* value are API errors.
|
||||
*/
|
||||
ZAPIERROR = -100,
|
||||
|
||||
ZNONODE = -101, /// Node does not exist
|
||||
ZNOAUTH = -102, /// Not authenticated
|
||||
ZBADVERSION = -103, /// Version conflict
|
||||
ZNOCHILDRENFOREPHEMERALS = -108, /// Ephemeral nodes may not have children
|
||||
ZNODEEXISTS = -110, /// The node already exists
|
||||
ZNOTEMPTY = -111, /// The node has children
|
||||
ZSESSIONEXPIRED = -112, /// The session has been expired by the server
|
||||
ZINVALIDCALLBACK = -113, /// Invalid callback specified
|
||||
ZINVALIDACL = -114, /// Invalid ACL specified
|
||||
ZAUTHFAILED = -115, /// Client authentication failed
|
||||
ZCLOSING = -116, /// ZooKeeper is closing
|
||||
ZNOTHING = -117, /// (not error) no server responses to process
|
||||
ZSESSIONMOVED = -118 /// Session moved to another server, so operation is ignored
|
||||
};
|
||||
|
||||
/// Network errors and similar. You should reinitialize ZooKeeper session in case of these errors
|
||||
bool isHardwareError(int32_t code);
|
||||
|
||||
/// Valid errors sent from the server about database state (like "no node"). Logical and authentication errors (like "bad arguments") are not here.
|
||||
bool isUserError(int32_t code);
|
||||
|
||||
const char * errorMessage(int32_t code);
|
||||
|
||||
/// For watches.
|
||||
enum State
|
||||
{
|
||||
EXPIRED_SESSION = -112,
|
||||
AUTH_FAILED = -113,
|
||||
CONNECTING = 1,
|
||||
ASSOCIATING = 2,
|
||||
CONNECTED = 3,
|
||||
NOTCONNECTED = 999
|
||||
};
|
||||
|
||||
enum Event
|
||||
{
|
||||
CREATED = 1,
|
||||
DELETED = 2,
|
||||
CHANGED = 3,
|
||||
CHILD = 4,
|
||||
SESSION = -1,
|
||||
NOTWATCHING = -2
|
||||
};
|
||||
|
||||
|
||||
class Exception : public DB::Exception
|
||||
{
|
||||
private:
|
||||
/// Delegate constructor, used to minimize repetition; last parameter used for overload resolution.
|
||||
Exception(const std::string & msg, const int32_t code, int);
|
||||
|
||||
public:
|
||||
explicit Exception(const int32_t code);
|
||||
Exception(const std::string & msg, const int32_t code);
|
||||
Exception(const int32_t code, const std::string & path);
|
||||
Exception(const Exception & exc);
|
||||
|
||||
const char * name() const throw() override { return "Coordination::Exception"; }
|
||||
const char * className() const throw() override { return "Coordination::Exception"; }
|
||||
Exception * clone() const override { return new Exception(*this); }
|
||||
|
||||
const int32_t code;
|
||||
};
|
||||
|
||||
|
||||
/** Usage scenario:
|
||||
* - create an object and issue commands;
|
||||
* - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap:
|
||||
* for example, just signal a condvar / fulfull a promise.
|
||||
* - you also may provide callbacks for watches; they are also invoked in internal thread and must be cheap.
|
||||
* - whenever you receive exception with ZSESSIONEXPIRED code or method isExpired returns true,
|
||||
* the ZooKeeper instance is no longer usable - you may only destroy it and probably create another.
|
||||
* - whenever session is expired or ZooKeeper instance is destroying, all callbacks are notified with special event.
|
||||
* - data for callbacks must be alive when ZooKeeper instance is alive.
|
||||
*/
|
||||
class IKeeper
|
||||
{
|
||||
public:
|
||||
virtual ~IKeeper() {}
|
||||
|
||||
/// If expired, you can only destroy the object. All other methods will throw exception.
|
||||
virtual bool isExpired() const = 0;
|
||||
|
||||
/// Useful to check owner of ephemeral node.
|
||||
virtual int64_t getSessionID() const = 0;
|
||||
|
||||
/// If the method will throw an exception, callbacks won't be called.
|
||||
///
|
||||
/// After the method is executed successfully, you must wait for callbacks
|
||||
/// (don't destroy callback data before it will be called).
|
||||
///
|
||||
/// All callbacks are executed sequentially (the execution of callbacks is serialized).
|
||||
///
|
||||
/// If an exception is thrown inside the callback, the session will expire,
|
||||
/// and all other callbacks will be called with "Session expired" error.
|
||||
|
||||
virtual void create(
|
||||
const String & path,
|
||||
const String & data,
|
||||
bool is_ephemeral,
|
||||
bool is_sequential,
|
||||
const ACLs & acls,
|
||||
CreateCallback callback) = 0;
|
||||
|
||||
virtual void remove(
|
||||
const String & path,
|
||||
int32_t version,
|
||||
RemoveCallback callback) = 0;
|
||||
|
||||
virtual void exists(
|
||||
const String & path,
|
||||
ExistsCallback callback,
|
||||
WatchCallback watch) = 0;
|
||||
|
||||
virtual void get(
|
||||
const String & path,
|
||||
GetCallback callback,
|
||||
WatchCallback watch) = 0;
|
||||
|
||||
virtual void set(
|
||||
const String & path,
|
||||
const String & data,
|
||||
int32_t version,
|
||||
SetCallback callback) = 0;
|
||||
|
||||
virtual void list(
|
||||
const String & path,
|
||||
ListCallback callback,
|
||||
WatchCallback watch) = 0;
|
||||
|
||||
virtual void check(
|
||||
const String & path,
|
||||
int32_t version,
|
||||
CheckCallback callback) = 0;
|
||||
|
||||
virtual void multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback) = 0;
|
||||
};
|
||||
|
||||
}
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
size_t result = 0;
|
||||
std::string result_str;
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
|
||||
bool success = false;
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
@ -29,11 +29,11 @@ public:
|
||||
if (zookeeper->tryGet(path, result_str, &stat))
|
||||
{
|
||||
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
|
||||
{
|
||||
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);
|
||||
|
@ -1,32 +1,32 @@
|
||||
#pragma once
|
||||
|
||||
#include "Types.h"
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
|
||||
|
||||
using KeeperException = ZooKeeperImpl::Exception;
|
||||
using KeeperException = Coordination::Exception;
|
||||
|
||||
|
||||
class KeeperMultiException : public KeeperException
|
||||
{
|
||||
public:
|
||||
Requests requests;
|
||||
Responses responses;
|
||||
Coordination::Requests requests;
|
||||
Coordination::Responses responses;
|
||||
size_t failed_op_index = 0;
|
||||
|
||||
std::string getPathForFirstFailedOp() const;
|
||||
|
||||
/// If it is user error throws KeeperMultiException else throws ordinary KeeperException
|
||||
/// 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:
|
||||
static size_t getFailedOpIndex(int32_t code, const Responses & responses);
|
||||
static size_t getFailedOpIndex(int32_t code, const Coordination::Responses & responses);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -122,7 +122,7 @@ private:
|
||||
{
|
||||
DB::tryLogCurrentException(log);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
if (e.code == Coordination::ZSESSIONEXPIRED)
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
|
@ -18,17 +18,17 @@ bool Lock::tryLock()
|
||||
std::string 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);
|
||||
}
|
||||
else if (code == ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
else if (code == Coordination::ZOK)
|
||||
{
|
||||
locked.reset(new ZooKeeperHandler(zookeeper));
|
||||
}
|
||||
else
|
||||
{
|
||||
throw zkutil::KeeperException(code);
|
||||
throw Coordination::Exception(code);
|
||||
}
|
||||
}
|
||||
return bool(locked);
|
||||
@ -50,7 +50,7 @@ Lock::Status Lock::tryCheck() const
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
|
||||
Status lock_status;
|
||||
Stat stat;
|
||||
Coordination::Stat stat;
|
||||
std::string dummy;
|
||||
bool result = zookeeper->tryGet(lock_path, dummy, &stat);
|
||||
if (!result)
|
||||
|
@ -1,16 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/Types.h>
|
||||
#include <future>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <Common/ZooKeeper/ZooKeeperImpl.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Poco/Event.h>
|
||||
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
|
||||
using Stat = ZooKeeperImpl::ZooKeeper::Stat;
|
||||
using Strings = std::vector<std::string>;
|
||||
|
||||
|
||||
@ -24,44 +24,14 @@ namespace CreateMode
|
||||
|
||||
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
|
||||
/// Each pair, the first is path, the second is response eg. CreateResponse, RemoveResponse
|
||||
template <typename 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);
|
||||
RequestPtr makeRemoveRequest(const std::string & path, int version);
|
||||
RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
|
||||
RequestPtr makeCheckRequest(const std::string & path, int version);
|
||||
Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode);
|
||||
Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version);
|
||||
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
|
||||
Coordination::RequestPtr makeCheckRequest(const std::string & path, int version);
|
||||
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "ZooKeeper.h"
|
||||
#include "ZooKeeperImpl.h"
|
||||
#include "KeeperException.h"
|
||||
|
||||
#include <random>
|
||||
@ -50,16 +51,16 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
|
||||
chroot = chroot_;
|
||||
|
||||
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;
|
||||
boost::split(addresses_strings, hosts, boost::is_any_of(","));
|
||||
ZooKeeperImpl::ZooKeeper::Addresses addresses;
|
||||
Coordination::ZooKeeper::Addresses addresses;
|
||||
addresses.reserve(addresses_strings.size());
|
||||
for (const auto & address_string : addresses_strings)
|
||||
addresses.emplace_back(address_string);
|
||||
|
||||
impl = std::make_unique<ZooKeeperImpl::ZooKeeper>(
|
||||
impl = std::make_unique<Coordination::ZooKeeper>(
|
||||
addresses,
|
||||
chroot,
|
||||
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));
|
||||
|
||||
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,
|
||||
@ -112,7 +113,7 @@ struct ZooKeeperArgs
|
||||
chroot = config.getString(config_name + "." + key);
|
||||
}
|
||||
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.
|
||||
@ -129,7 +130,7 @@ struct ZooKeeperArgs
|
||||
if (!chroot.empty())
|
||||
{
|
||||
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() == '/')
|
||||
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)
|
||||
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,
|
||||
Stat * stat,
|
||||
WatchCallback watch_callback)
|
||||
Coordination::Stat * stat,
|
||||
Coordination::WatchCallback watch_callback)
|
||||
{
|
||||
int32_t code = 0;
|
||||
Poco::Event event;
|
||||
|
||||
auto callback = [&](const ZooKeeperImpl::ZooKeeper::ListResponse & response)
|
||||
auto callback = [&](const Coordination::ListResponse & response)
|
||||
{
|
||||
code = response.error;
|
||||
if (!code)
|
||||
@ -181,7 +182,7 @@ int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res,
|
||||
}
|
||||
|
||||
Strings ZooKeeper::getChildren(
|
||||
const std::string & path, Stat * stat, const EventPtr & watch)
|
||||
const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
|
||||
{
|
||||
Strings res;
|
||||
check(tryGetChildren(path, res, stat, watch), path);
|
||||
@ -189,7 +190,7 @@ Strings ZooKeeper::getChildren(
|
||||
}
|
||||
|
||||
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;
|
||||
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,
|
||||
Stat * stat, const EventPtr & watch)
|
||||
Coordination::Stat * stat, const EventPtr & 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);
|
||||
|
||||
return code;
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
|
||||
if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
|
||||
throw KeeperException(code, path);
|
||||
|
||||
return code;
|
||||
@ -223,7 +224,7 @@ int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data
|
||||
int32_t code = 0;
|
||||
Poco::Event event;
|
||||
|
||||
auto callback = [&](const ZooKeeperImpl::ZooKeeper::CreateResponse & response)
|
||||
auto callback = [&](const Coordination::CreateResponse & response)
|
||||
{
|
||||
code = response.error;
|
||||
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);
|
||||
|
||||
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZNONODE ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS))
|
||||
if (!(code == Coordination::ZOK ||
|
||||
code == Coordination::ZNONODE ||
|
||||
code == Coordination::ZNODEEXISTS ||
|
||||
code == Coordination::ZNOCHILDRENFOREPHEMERALS))
|
||||
throw KeeperException(code, path);
|
||||
|
||||
return code;
|
||||
@ -267,7 +268,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string &
|
||||
std::string 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;
|
||||
else
|
||||
throw KeeperException(code, path);
|
||||
@ -291,7 +292,7 @@ int32_t ZooKeeper::removeImpl(const std::string & path, int32_t version)
|
||||
int32_t code = 0;
|
||||
Poco::Event event;
|
||||
|
||||
auto callback = [&](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response)
|
||||
auto callback = [&](const Coordination::RemoveResponse & response)
|
||||
{
|
||||
if (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 code = removeImpl(path, version);
|
||||
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZNONODE ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZBADVERSION ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY))
|
||||
if (!(code == Coordination::ZOK ||
|
||||
code == Coordination::ZNONODE ||
|
||||
code == Coordination::ZBADVERSION ||
|
||||
code == Coordination::ZNOTEMPTY))
|
||||
throw KeeperException(code, path);
|
||||
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;
|
||||
Poco::Event event;
|
||||
|
||||
auto callback = [&](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response)
|
||||
auto callback = [&](const Coordination::ExistsResponse & response)
|
||||
{
|
||||
code = response.error;
|
||||
if (!code && stat)
|
||||
@ -337,28 +338,28 @@ int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallba
|
||||
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));
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
|
||||
if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
|
||||
throw KeeperException(code, path);
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (code == Coordination::ZNONODE)
|
||||
return false;
|
||||
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;
|
||||
Poco::Event event;
|
||||
|
||||
auto callback = [&](const ZooKeeperImpl::ZooKeeper::GetResponse & response)
|
||||
auto callback = [&](const Coordination::GetResponse & response)
|
||||
{
|
||||
code = response.error;
|
||||
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;
|
||||
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);
|
||||
}
|
||||
|
||||
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;
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
|
||||
if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
|
||||
throw KeeperException(code, path);
|
||||
|
||||
if (return_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 version, Stat * stat)
|
||||
int32_t version, Coordination::Stat * stat)
|
||||
{
|
||||
int32_t code = 0;
|
||||
Poco::Event event;
|
||||
|
||||
auto callback = [&](const ZooKeeperImpl::ZooKeeper::SetResponse & response)
|
||||
auto callback = [&](const Coordination::SetResponse & response)
|
||||
{
|
||||
code = response.error;
|
||||
if (!code && stat)
|
||||
@ -433,7 +434,7 @@ int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data,
|
||||
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);
|
||||
}
|
||||
@ -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)
|
||||
{
|
||||
int32_t code = trySet(path, data, -1);
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (code == Coordination::ZNONODE)
|
||||
{
|
||||
create(path, data, mode);
|
||||
}
|
||||
else if (code != ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
else if (code != Coordination::ZOK)
|
||||
throw KeeperException(code, path);
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZNONODE ||
|
||||
code == ZooKeeperImpl::ZooKeeper::ZBADVERSION))
|
||||
if (!(code == Coordination::ZOK ||
|
||||
code == Coordination::ZNONODE ||
|
||||
code == Coordination::ZBADVERSION))
|
||||
throw KeeperException(code, path);
|
||||
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())
|
||||
return ZooKeeperImpl::ZooKeeper::ZOK;
|
||||
return Coordination::ZOK;
|
||||
|
||||
int32_t code = 0;
|
||||
Poco::Event event;
|
||||
|
||||
auto callback = [&](const ZooKeeperImpl::ZooKeeper::MultiResponse & response)
|
||||
auto callback = [&](const Coordination::MultiResponse & response)
|
||||
{
|
||||
code = response.error;
|
||||
responses = response.responses;
|
||||
@ -482,18 +483,18 @@ int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses)
|
||||
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);
|
||||
KeeperMultiException::check(code, requests, 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);
|
||||
if (code && !ZooKeeperImpl::ZooKeeper::isUserError(code))
|
||||
if (code && !Coordination::isUserError(code))
|
||||
throw KeeperException(code);
|
||||
return code;
|
||||
}
|
||||
@ -504,7 +505,7 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path)
|
||||
Strings children = getChildren(path);
|
||||
while (!children.empty())
|
||||
{
|
||||
Requests ops;
|
||||
Coordination::Requests ops;
|
||||
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
|
||||
{
|
||||
removeChildrenRecursive(path + "/" + children.back());
|
||||
@ -518,11 +519,11 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path)
|
||||
void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path)
|
||||
{
|
||||
Strings children;
|
||||
if (tryGetChildren(path, children) != ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
if (tryGetChildren(path, children) != Coordination::ZOK)
|
||||
return;
|
||||
while (!children.empty())
|
||||
{
|
||||
Requests ops;
|
||||
Coordination::Requests ops;
|
||||
Strings batch;
|
||||
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();
|
||||
tryRemoveChildrenRecursive(batch.back());
|
||||
|
||||
ZooKeeperImpl::ZooKeeper::RemoveRequest request;
|
||||
Coordination::RemoveRequest request;
|
||||
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,
|
||||
/// this means someone is concurrently removing these children and we will have
|
||||
/// to remove them one by one.
|
||||
Responses responses;
|
||||
if (tryMulti(ops, responses) != ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
Coordination::Responses responses;
|
||||
if (tryMulti(ops, responses) != Coordination::ZOK)
|
||||
for (const std::string & child : batch)
|
||||
tryRemove(child);
|
||||
}
|
||||
@ -576,14 +577,14 @@ void ZooKeeper::waitForDisappear(const std::string & path)
|
||||
|
||||
while (true)
|
||||
{
|
||||
auto callback = [state](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response)
|
||||
auto callback = [state](const Coordination::ExistsResponse & response)
|
||||
{
|
||||
state->code = response.error;
|
||||
if (state->code)
|
||||
state->event.set();
|
||||
};
|
||||
|
||||
auto watch = [state](const ZooKeeperImpl::ZooKeeper::WatchResponse & response)
|
||||
auto watch = [state](const Coordination::WatchResponse & response)
|
||||
{
|
||||
if (!state->code)
|
||||
{
|
||||
@ -599,13 +600,13 @@ void ZooKeeper::waitForDisappear(const std::string & path)
|
||||
impl->exists(path, callback, watch);
|
||||
state->event.wait();
|
||||
|
||||
if (state->code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (state->code == Coordination::ZNONODE)
|
||||
return;
|
||||
|
||||
if (state->code)
|
||||
throw KeeperException(state->code, path);
|
||||
|
||||
if (state->event_type == ZooKeeperImpl::ZooKeeper::DELETED)
|
||||
if (state->event_type == Coordination::DELETED)
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -618,7 +619,7 @@ ZooKeeperPtr ZooKeeper::startNewSession() const
|
||||
|
||||
std::string ZooKeeper::error2string(int32_t code)
|
||||
{
|
||||
return ZooKeeperImpl::ZooKeeper::errorMessage(code);
|
||||
return Coordination::errorMessage(code);
|
||||
}
|
||||
|
||||
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
|
||||
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 callback = [promise, path](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) mutable
|
||||
auto callback = [promise, path](const Coordination::CreateResponse & response) mutable
|
||||
{
|
||||
if (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 callback = [promise, path](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable
|
||||
auto callback = [promise, path](const Coordination::GetResponse & response) mutable
|
||||
{
|
||||
if (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 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)));
|
||||
else
|
||||
promise->set_value(response);
|
||||
@ -686,14 +687,14 @@ std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncTryGet(const
|
||||
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 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)));
|
||||
else
|
||||
promise->set_value(response);
|
||||
@ -703,12 +704,12 @@ std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> ZooKeeper::asyncExists(con
|
||||
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 callback = [promise, path](const ZooKeeperImpl::ZooKeeper::SetResponse & response) mutable
|
||||
auto callback = [promise, path](const Coordination::SetResponse & response) mutable
|
||||
{
|
||||
if (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;
|
||||
}
|
||||
|
||||
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 callback = [promise, path](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable
|
||||
auto callback = [promise, path](const Coordination::ListResponse & response) mutable
|
||||
{
|
||||
if (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;
|
||||
}
|
||||
|
||||
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 callback = [promise, path](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable
|
||||
auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable
|
||||
{
|
||||
if (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;
|
||||
}
|
||||
|
||||
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 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)));
|
||||
else
|
||||
promise->set_value(response);
|
||||
@ -771,12 +772,12 @@ std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncTryRemove(
|
||||
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 callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable
|
||||
auto callback = [promise](const Coordination::MultiResponse & response) mutable
|
||||
{
|
||||
promise->set_value(response);
|
||||
};
|
||||
@ -785,12 +786,12 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::tryAsyncMulti(co
|
||||
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 callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable
|
||||
auto callback = [promise](const Coordination::MultiResponse & response) mutable
|
||||
{
|
||||
if (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;
|
||||
}
|
||||
|
||||
int32_t ZooKeeper::tryMultiNoThrow(const Requests & requests, Responses & responses)
|
||||
int32_t ZooKeeper::tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses)
|
||||
{
|
||||
try
|
||||
{
|
||||
return multiImpl(requests, responses);
|
||||
}
|
||||
catch (const ZooKeeperImpl::Exception & e)
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
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())
|
||||
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)
|
||||
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",
|
||||
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),
|
||||
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();
|
||||
}
|
||||
|
||||
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)
|
||||
return;
|
||||
|
||||
if (ZooKeeperImpl::ZooKeeper::isUserError(code))
|
||||
if (Coordination::isUserError(code))
|
||||
throw KeeperMultiException(code, requests, responses);
|
||||
else
|
||||
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->data = data;
|
||||
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;
|
||||
}
|
||||
|
||||
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->version = version;
|
||||
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->data = data;
|
||||
request->version = version;
|
||||
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->version = version;
|
||||
return request;
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <port/unistd.h>
|
||||
|
||||
|
||||
@ -109,20 +110,20 @@ public:
|
||||
/// * The node has children.
|
||||
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 existsWatch(const std::string & path, Stat * stat, WatchCallback watch_callback);
|
||||
bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr);
|
||||
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 getWatch(const std::string & path, Stat * stat, WatchCallback watch_callback);
|
||||
std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr);
|
||||
std::string getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
|
||||
|
||||
/// Doesn't not throw in the following cases:
|
||||
/// * 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,
|
||||
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.
|
||||
void createOrUpdate(const std::string & path, const std::string & data, int32_t mode);
|
||||
@ -131,34 +132,34 @@ public:
|
||||
/// * The node doesn't exist.
|
||||
/// * Versions do not match.
|
||||
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,
|
||||
Stat * stat = nullptr,
|
||||
Coordination::Stat * stat = nullptr,
|
||||
const EventPtr & watch = nullptr);
|
||||
|
||||
Strings getChildrenWatch(const std::string & path,
|
||||
Stat * stat,
|
||||
WatchCallback watch_callback);
|
||||
Coordination::Stat * stat,
|
||||
Coordination::WatchCallback watch_callback);
|
||||
|
||||
/// Doesn't not throw in the following cases:
|
||||
/// * The node doesn't exist.
|
||||
int32_t tryGetChildren(const std::string & path, Strings & res,
|
||||
Stat * stat = nullptr,
|
||||
Coordination::Stat * stat = nullptr,
|
||||
const EventPtr & watch = nullptr);
|
||||
|
||||
int32_t tryGetChildrenWatch(const std::string & path, Strings & res,
|
||||
Stat * stat,
|
||||
WatchCallback watch_callback);
|
||||
Coordination::Stat * stat,
|
||||
Coordination::WatchCallback watch_callback);
|
||||
|
||||
/// Performs several operations in a transaction.
|
||||
/// 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
|
||||
/// - 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)
|
||||
int32_t tryMultiNoThrow(const Requests & requests, Responses & responses);
|
||||
int32_t tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses);
|
||||
|
||||
Int64 getClientID();
|
||||
|
||||
@ -190,24 +191,24 @@ public:
|
||||
///
|
||||
/// 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);
|
||||
|
||||
using FutureGet = std::future<ZooKeeperImpl::ZooKeeper::GetResponse>;
|
||||
using FutureGet = std::future<Coordination::GetResponse>;
|
||||
FutureGet asyncGet(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);
|
||||
|
||||
using FutureGetChildren = std::future<ZooKeeperImpl::ZooKeeper::ListResponse>;
|
||||
using FutureGetChildren = std::future<Coordination::ListResponse>;
|
||||
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);
|
||||
|
||||
using FutureRemove = std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse>;
|
||||
using FutureRemove = std::future<Coordination::RemoveResponse>;
|
||||
FutureRemove asyncRemove(const std::string & path, int32_t version = -1);
|
||||
|
||||
/// Doesn't throw in the following cases:
|
||||
@ -216,11 +217,11 @@ public:
|
||||
/// * The node has children
|
||||
FutureRemove asyncTryRemove(const std::string & path, int32_t version = -1);
|
||||
|
||||
using FutureMulti = std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>;
|
||||
FutureMulti asyncMulti(const Requests & ops);
|
||||
using FutureMulti = std::future<Coordination::MultiResponse>;
|
||||
FutureMulti asyncMulti(const Coordination::Requests & ops);
|
||||
|
||||
/// 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);
|
||||
|
||||
@ -235,13 +236,13 @@ private:
|
||||
/// 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 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 setImpl(const std::string & path, const std::string & data, int32_t version, Stat * stat);
|
||||
int32_t getChildrenImpl(const std::string & path, Strings & res, Stat * stat, WatchCallback watch_callback);
|
||||
int32_t multiImpl(const Requests & requests, Responses & responses);
|
||||
int32_t existsImpl(const std::string & path, 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, Coordination::Stat * stat);
|
||||
int32_t getChildrenImpl(const std::string & path, Strings & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
|
||||
int32_t multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses);
|
||||
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 identity;
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -3,6 +3,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
@ -78,352 +79,25 @@ namespace CurrentMetrics
|
||||
}
|
||||
|
||||
|
||||
namespace ZooKeeperImpl
|
||||
namespace Coordination
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
|
||||
class Exception : public DB::Exception
|
||||
{
|
||||
private:
|
||||
/// Delegate constructor, used to minimize repetition; last parameter used for overload resolution.
|
||||
Exception(const std::string & msg, const int32_t code, int);
|
||||
|
||||
public:
|
||||
explicit Exception(const int32_t code);
|
||||
Exception(const std::string & msg, const int32_t code);
|
||||
Exception(const int32_t code, const std::string & path);
|
||||
Exception(const Exception & exc);
|
||||
|
||||
const char * name() const throw() override { return "ZooKeeperImpl::Exception"; }
|
||||
const char * className() const throw() override { return "ZooKeeperImpl::Exception"; }
|
||||
Exception * clone() const override { return new Exception(*this); }
|
||||
|
||||
const int32_t code;
|
||||
};
|
||||
struct ZooKeeperRequest;
|
||||
|
||||
|
||||
/** Usage scenario:
|
||||
* - create an object and issue commands;
|
||||
* - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap:
|
||||
* for example, just signal a condvar / fulfull a promise.
|
||||
* - you also may provide callbacks for watches; they are also invoked in internal thread and must be cheap.
|
||||
* - whenever you receive exception with ZSESSIONEXPIRED code or method isExpired returns true,
|
||||
* the ZooKeeper instance is no longer usable - you may only destroy it and probably create another.
|
||||
* - whenever session is expired or ZooKeeper instance is destroying, all callbacks are notified with special event.
|
||||
* - data for callbacks must be alive when ZooKeeper instance is alive.
|
||||
|
||||
/** Usage scenario: look at the documentation for IKeeper class.
|
||||
*/
|
||||
class ZooKeeper
|
||||
class ZooKeeper : public IKeeper
|
||||
{
|
||||
public:
|
||||
using Addresses = std::vector<Poco::Net::SocketAddress>;
|
||||
|
||||
struct ACL
|
||||
{
|
||||
static constexpr int32_t Read = 1;
|
||||
static constexpr int32_t Write = 2;
|
||||
static constexpr int32_t Create = 4;
|
||||
static constexpr int32_t Delete = 8;
|
||||
static constexpr int32_t Admin = 16;
|
||||
static constexpr int32_t All = 0x1F;
|
||||
|
||||
int32_t permissions;
|
||||
String scheme;
|
||||
String id;
|
||||
|
||||
void write(WriteBuffer & out) const;
|
||||
};
|
||||
using ACLs = std::vector<ACL>;
|
||||
|
||||
struct Stat
|
||||
{
|
||||
int64_t czxid;
|
||||
int64_t mzxid;
|
||||
int64_t ctime;
|
||||
int64_t mtime;
|
||||
int32_t version;
|
||||
int32_t cversion;
|
||||
int32_t aversion;
|
||||
int64_t ephemeralOwner;
|
||||
int32_t dataLength;
|
||||
int32_t numChildren;
|
||||
int64_t pzxid;
|
||||
|
||||
void read(ReadBuffer & in);
|
||||
};
|
||||
|
||||
using XID = int32_t;
|
||||
using OpNum = int32_t;
|
||||
|
||||
struct Response;
|
||||
using ResponsePtr = std::shared_ptr<Response>;
|
||||
using Responses = std::vector<ResponsePtr>;
|
||||
using ResponseCallback = std::function<void(const Response &)>;
|
||||
|
||||
struct Response
|
||||
{
|
||||
int32_t error = 0;
|
||||
virtual ~Response() {}
|
||||
virtual void readImpl(ReadBuffer &) = 0;
|
||||
|
||||
virtual void removeRootPath(const String & /* root_path */) {}
|
||||
};
|
||||
|
||||
struct Request;
|
||||
using RequestPtr = std::shared_ptr<Request>;
|
||||
using Requests = std::vector<RequestPtr>;
|
||||
|
||||
struct Request
|
||||
{
|
||||
XID xid = 0;
|
||||
bool has_watch = false;
|
||||
/// If the request was not send and the error happens, we definitely sure, that is has not been processed by the server.
|
||||
/// If the request was sent and we didn't get the response and the error happens, then we cannot be sure was it processed or not.
|
||||
bool probably_sent = false;
|
||||
|
||||
virtual ~Request() {}
|
||||
virtual RequestPtr clone() const = 0;
|
||||
|
||||
virtual OpNum getOpNum() const = 0;
|
||||
|
||||
/// Writes length, xid, op_num, then the rest.
|
||||
void write(WriteBuffer & out) const;
|
||||
virtual void writeImpl(WriteBuffer &) const = 0;
|
||||
|
||||
virtual ResponsePtr makeResponse() const = 0;
|
||||
|
||||
virtual void addRootPath(const String & /* root_path */) {}
|
||||
virtual String getPath() const = 0;
|
||||
};
|
||||
|
||||
struct HeartbeatRequest final : Request
|
||||
{
|
||||
RequestPtr clone() const override { return std::make_shared<HeartbeatRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 11; }
|
||||
void writeImpl(WriteBuffer &) const override {}
|
||||
ResponsePtr makeResponse() const override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct HeartbeatResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
};
|
||||
|
||||
struct WatchResponse final : Response
|
||||
{
|
||||
int32_t type = 0;
|
||||
int32_t state = 0;
|
||||
String path;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
using WatchCallback = std::function<void(const WatchResponse &)>;
|
||||
|
||||
struct AuthRequest final : Request
|
||||
{
|
||||
int32_t type = 0; /// ignored by the server
|
||||
String scheme;
|
||||
String data;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<AuthRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 100; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct AuthResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
};
|
||||
|
||||
struct CloseRequest final : Request
|
||||
{
|
||||
RequestPtr clone() const override { return std::make_shared<CloseRequest>(*this); }
|
||||
OpNum getOpNum() const override { return -11; }
|
||||
void writeImpl(WriteBuffer &) const override {}
|
||||
ResponsePtr makeResponse() const override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct CloseResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct CreateRequest final : Request
|
||||
{
|
||||
String path;
|
||||
String data;
|
||||
bool is_ephemeral = false;
|
||||
bool is_sequential = false;
|
||||
ACLs acls;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<CreateRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 1; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct CreateResponse final : Response
|
||||
{
|
||||
String path_created;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
struct RemoveRequest final : Request
|
||||
{
|
||||
String path;
|
||||
int32_t version = -1;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<RemoveRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 2; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct RemoveResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
};
|
||||
|
||||
struct ExistsRequest final : Request
|
||||
{
|
||||
String path;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<ExistsRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 3; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct ExistsResponse final : Response
|
||||
{
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct GetRequest final : Request
|
||||
{
|
||||
String path;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<GetRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 4; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct GetResponse final : Response
|
||||
{
|
||||
String data;
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct SetRequest final : Request
|
||||
{
|
||||
String path;
|
||||
String data;
|
||||
int32_t version = -1;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<SetRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 5; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct SetResponse final : Response
|
||||
{
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct ListRequest final : Request
|
||||
{
|
||||
String path;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<ListRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 12; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct ListResponse final : Response
|
||||
{
|
||||
std::vector<String> names;
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct CheckRequest final : Request
|
||||
{
|
||||
String path;
|
||||
int32_t version = -1;
|
||||
|
||||
RequestPtr clone() const override { return std::make_shared<CheckRequest>(*this); }
|
||||
OpNum getOpNum() const override { return 13; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct CheckResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
};
|
||||
|
||||
struct MultiRequest final : Request
|
||||
{
|
||||
Requests requests;
|
||||
|
||||
RequestPtr clone() const override;
|
||||
OpNum getOpNum() const override { return 14; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct MultiResponse final : Response
|
||||
{
|
||||
Responses responses;
|
||||
|
||||
MultiResponse(const Requests & requests);
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
/// This response may be received only as an element of responses in MultiResponse.
|
||||
struct ErrorResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
|
||||
/** Connection to addresses is performed in order. If you want, shuffle them manually.
|
||||
* Operation timeout couldn't be greater than session timeout.
|
||||
* Operation timeout applies independently for network read, network write, waiting for events and synchronization.
|
||||
@ -441,30 +115,13 @@ public:
|
||||
|
||||
|
||||
/// If expired, you can only destroy the object. All other methods will throw exception.
|
||||
bool isExpired() const { return expired; }
|
||||
bool isExpired() const override { return expired; }
|
||||
|
||||
/// Useful to check owner of ephemeral node.
|
||||
int64_t getSessionID() const { return session_id; }
|
||||
int64_t getSessionID() const override { return session_id; }
|
||||
|
||||
|
||||
using CreateCallback = std::function<void(const CreateResponse &)>;
|
||||
using RemoveCallback = std::function<void(const RemoveResponse &)>;
|
||||
using ExistsCallback = std::function<void(const ExistsResponse &)>;
|
||||
using GetCallback = std::function<void(const GetResponse &)>;
|
||||
using SetCallback = std::function<void(const SetResponse &)>;
|
||||
using ListCallback = std::function<void(const ListResponse &)>;
|
||||
using CheckCallback = std::function<void(const CheckResponse &)>;
|
||||
using MultiCallback = std::function<void(const MultiResponse &)>;
|
||||
|
||||
/// If the method will throw an exception, callbacks won't be called.
|
||||
///
|
||||
/// After the method is executed successfully, you must wait for callbacks
|
||||
/// (don't destroy callback data before it will be called).
|
||||
///
|
||||
/// All callbacks are executed sequentially (the execution of callbacks is serialized).
|
||||
///
|
||||
/// If an exception is thrown inside the callback, the session will expire,
|
||||
/// and all other callbacks will be called with "Session expired" error.
|
||||
/// See the documentation about semantics of these methods in IKeeper class.
|
||||
|
||||
void create(
|
||||
const String & path,
|
||||
@ -472,114 +129,42 @@ public:
|
||||
bool is_ephemeral,
|
||||
bool is_sequential,
|
||||
const ACLs & acls,
|
||||
CreateCallback callback);
|
||||
CreateCallback callback) override;
|
||||
|
||||
void remove(
|
||||
const String & path,
|
||||
int32_t version,
|
||||
RemoveCallback callback);
|
||||
RemoveCallback callback) override;
|
||||
|
||||
void exists(
|
||||
const String & path,
|
||||
ExistsCallback callback,
|
||||
WatchCallback watch);
|
||||
WatchCallback watch) override;
|
||||
|
||||
void get(
|
||||
const String & path,
|
||||
GetCallback callback,
|
||||
WatchCallback watch);
|
||||
WatchCallback watch) override;
|
||||
|
||||
void set(
|
||||
const String & path,
|
||||
const String & data,
|
||||
int32_t version,
|
||||
SetCallback callback);
|
||||
SetCallback callback) override;
|
||||
|
||||
void list(
|
||||
const String & path,
|
||||
ListCallback callback,
|
||||
WatchCallback watch);
|
||||
WatchCallback watch) override;
|
||||
|
||||
void check(
|
||||
const String & path,
|
||||
int32_t version,
|
||||
CheckCallback callback);
|
||||
CheckCallback callback) override;
|
||||
|
||||
void multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback);
|
||||
|
||||
|
||||
enum Error
|
||||
{
|
||||
ZOK = 0,
|
||||
|
||||
/** System and server-side errors.
|
||||
* This is never thrown by the server, it shouldn't be used other than
|
||||
* to indicate a range. Specifically error codes greater than this
|
||||
* value, but lesser than ZAPIERROR, are system errors.
|
||||
*/
|
||||
ZSYSTEMERROR = -1,
|
||||
|
||||
ZRUNTIMEINCONSISTENCY = -2, /// A runtime inconsistency was found
|
||||
ZDATAINCONSISTENCY = -3, /// A data inconsistency was found
|
||||
ZCONNECTIONLOSS = -4, /// Connection to the server has been lost
|
||||
ZMARSHALLINGERROR = -5, /// Error while marshalling or unmarshalling data
|
||||
ZUNIMPLEMENTED = -6, /// Operation is unimplemented
|
||||
ZOPERATIONTIMEOUT = -7, /// Operation timeout
|
||||
ZBADARGUMENTS = -8, /// Invalid arguments
|
||||
ZINVALIDSTATE = -9, /// Invliad zhandle state
|
||||
|
||||
/** API errors.
|
||||
* This is never thrown by the server, it shouldn't be used other than
|
||||
* to indicate a range. Specifically error codes greater than this
|
||||
* value are API errors.
|
||||
*/
|
||||
ZAPIERROR = -100,
|
||||
|
||||
ZNONODE = -101, /// Node does not exist
|
||||
ZNOAUTH = -102, /// Not authenticated
|
||||
ZBADVERSION = -103, /// Version conflict
|
||||
ZNOCHILDRENFOREPHEMERALS = -108, /// Ephemeral nodes may not have children
|
||||
ZNODEEXISTS = -110, /// The node already exists
|
||||
ZNOTEMPTY = -111, /// The node has children
|
||||
ZSESSIONEXPIRED = -112, /// The session has been expired by the server
|
||||
ZINVALIDCALLBACK = -113, /// Invalid callback specified
|
||||
ZINVALIDACL = -114, /// Invalid ACL specified
|
||||
ZAUTHFAILED = -115, /// Client authentication failed
|
||||
ZCLOSING = -116, /// ZooKeeper is closing
|
||||
ZNOTHING = -117, /// (not error) no server responses to process
|
||||
ZSESSIONMOVED = -118 /// Session moved to another server, so operation is ignored
|
||||
};
|
||||
|
||||
/// Network errors and similar. You should reinitialize ZooKeeper session in case of these errors
|
||||
static bool isHardwareError(int32_t code);
|
||||
|
||||
/// Valid errors sent from the server about database state (like "no node"). Logical and authentication errors (like "bad arguments") are not here.
|
||||
static bool isUserError(int32_t code);
|
||||
|
||||
static const char * errorMessage(int32_t code);
|
||||
|
||||
/// For watches.
|
||||
enum State
|
||||
{
|
||||
EXPIRED_SESSION = -112,
|
||||
AUTH_FAILED = -113,
|
||||
CONNECTING = 1,
|
||||
ASSOCIATING = 2,
|
||||
CONNECTED = 3,
|
||||
NOTCONNECTED = 999
|
||||
};
|
||||
|
||||
enum Event
|
||||
{
|
||||
CREATED = 1,
|
||||
DELETED = 2,
|
||||
CHANGED = 3,
|
||||
CHILD = 4,
|
||||
SESSION = -1,
|
||||
NOTWATCHING = -2
|
||||
};
|
||||
MultiCallback callback) override;
|
||||
|
||||
private:
|
||||
String root_path;
|
||||
@ -602,7 +187,7 @@ private:
|
||||
|
||||
struct RequestInfo
|
||||
{
|
||||
RequestPtr request;
|
||||
std::shared_ptr<ZooKeeperRequest> request;
|
||||
ResponseCallback callback;
|
||||
WatchCallback watch;
|
||||
clock::time_point time;
|
||||
|
@ -41,18 +41,18 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
|
||||
if (nonexistent_nodes.count(path))
|
||||
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;
|
||||
|
||||
bool changed = false;
|
||||
{
|
||||
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;
|
||||
else if (response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION)
|
||||
else if (response.state == Coordination::EXPIRED_SESSION)
|
||||
{
|
||||
context->zookeeper = nullptr;
|
||||
context->invalidated_paths.clear();
|
||||
|
@ -33,7 +33,7 @@ TEST(zkutil, multi_nice_exception_msg)
|
||||
{
|
||||
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
|
||||
ASSERT_NO_THROW(
|
||||
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
|
||||
@ -70,7 +70,7 @@ TEST(zkutil, multi_nice_exception_msg)
|
||||
TEST(zkutil, multi_async)
|
||||
{
|
||||
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
|
||||
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
|
||||
|
||||
@ -88,13 +88,13 @@ TEST(zkutil, multi_async)
|
||||
ops.clear();
|
||||
|
||||
auto res = fut.get();
|
||||
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZOK);
|
||||
ASSERT_TRUE(res.error == Coordination::ZOK);
|
||||
ASSERT_EQ(res.responses.size(), 2);
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
@ -124,7 +124,7 @@ TEST(zkutil, multi_async)
|
||||
ops.clear();
|
||||
|
||||
auto res = fut.get();
|
||||
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS);
|
||||
ASSERT_TRUE(res.error == Coordination::ZNODEEXISTS);
|
||||
ASSERT_EQ(res.responses.size(), 2);
|
||||
}
|
||||
}
|
||||
@ -176,11 +176,11 @@ TEST(zkutil, multi_create_sequential)
|
||||
zookeeper->tryRemoveRecursive(base_path);
|
||||
zookeeper->createAncestors(base_path + "/");
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
String sequential_node_prefix = base_path + "/queue-";
|
||||
ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential));
|
||||
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_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length());
|
||||
|
@ -34,11 +34,11 @@ int main(int argc, char ** argv)
|
||||
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::makeRemoveRequest("/test/zk_expiration_test", -1));
|
||||
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses responses;
|
||||
int32_t code = zk.tryMultiNoThrow(ops, responses);
|
||||
|
||||
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(code) << std::endl;
|
||||
@ -57,7 +57,7 @@ int main(int argc, char ** argv)
|
||||
sleep(1);
|
||||
}
|
||||
}
|
||||
catch (zkutil::KeeperException & e)
|
||||
catch (Coordination::Exception & e)
|
||||
{
|
||||
std::cerr << "KeeperException: " << DB::getCurrentExceptionMessage(true) << std::endl;
|
||||
return 1;
|
||||
|
@ -23,7 +23,7 @@ try
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
std::vector<std::future<zkutil::GetResponse>> futures;
|
||||
std::vector<std::future<Coordination::GetResponse>> futures;
|
||||
for (auto & node : nodes)
|
||||
futures.push_back(zookeeper.asyncGet("/tmp/" + node));
|
||||
|
||||
|
@ -20,7 +20,7 @@ try
|
||||
|
||||
std::cout << "create path" << std::endl;
|
||||
zk.create("/test", "old", zkutil::CreateMode::Persistent);
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
zkutil::EventPtr watch = std::make_shared<Poco::Event>();
|
||||
|
||||
std::cout << "get path" << std::endl;
|
||||
@ -38,13 +38,13 @@ try
|
||||
|
||||
zk.remove("/test");
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/test", "multi1", CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeSetRequest("/test", "multi2", -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest("/test", -1));
|
||||
std::cout << "multi" << std::endl;
|
||||
zkutil::Responses res = zk.multi(ops);
|
||||
std::cout << "path created: " << typeid_cast<const CreateResponse &>(*res[0]).path_created << std::endl;
|
||||
Coordination::Responses res = zk.multi(ops);
|
||||
std::cout << "path created: " << dynamic_cast<const Coordination::CreateResponse &>(*res[0]).path_created << std::endl;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
using namespace ZooKeeperImpl;
|
||||
using namespace Coordination;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
@ -38,10 +38,10 @@ try
|
||||
std::cout << "create\n";
|
||||
|
||||
zk.create("/test", "old", false, false, {},
|
||||
[&](const ZooKeeper::CreateResponse & response)
|
||||
[&](const CreateResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Created path: " << response.path_created << '\n';
|
||||
|
||||
@ -53,19 +53,19 @@ try
|
||||
std::cout << "get\n";
|
||||
|
||||
zk.get("/test",
|
||||
[&](const ZooKeeper::GetResponse & response)
|
||||
[&](const GetResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Value: " << response.data << '\n';
|
||||
|
||||
//event.set();
|
||||
},
|
||||
[](const ZooKeeper::WatchResponse & response)
|
||||
[](const WatchResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n';
|
||||
});
|
||||
@ -75,10 +75,10 @@ try
|
||||
std::cout << "set\n";
|
||||
|
||||
zk.set("/test", "new", -1,
|
||||
[&](const ZooKeeper::SetResponse & response)
|
||||
[&](const SetResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Set\n";
|
||||
|
||||
@ -90,10 +90,10 @@ try
|
||||
std::cout << "list\n";
|
||||
|
||||
zk.list("/",
|
||||
[&](const ZooKeeper::ListResponse & response)
|
||||
[&](const ListResponse & response)
|
||||
{
|
||||
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
|
||||
{
|
||||
std::cerr << "Children:\n";
|
||||
@ -103,10 +103,10 @@ try
|
||||
|
||||
//event.set();
|
||||
},
|
||||
[](const ZooKeeper::WatchResponse & response)
|
||||
[](const WatchResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n';
|
||||
});
|
||||
@ -116,19 +116,19 @@ try
|
||||
std::cout << "exists\n";
|
||||
|
||||
zk.exists("/test",
|
||||
[&](const ZooKeeper::ExistsResponse & response)
|
||||
[&](const ExistsResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Exists\n";
|
||||
|
||||
//event.set();
|
||||
},
|
||||
[](const ZooKeeper::WatchResponse & response)
|
||||
[](const WatchResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n';
|
||||
});
|
||||
@ -137,10 +137,10 @@ try
|
||||
|
||||
std::cout << "remove\n";
|
||||
|
||||
zk.remove("/test", -1, [&](const ZooKeeper::RemoveResponse & response)
|
||||
zk.remove("/test", -1, [&](const RemoveResponse & response)
|
||||
{
|
||||
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
|
||||
std::cerr << "Removed\n";
|
||||
|
||||
@ -151,43 +151,43 @@ try
|
||||
|
||||
std::cout << "multi\n";
|
||||
|
||||
ZooKeeper::Requests ops;
|
||||
Requests ops;
|
||||
|
||||
{
|
||||
ZooKeeper::CreateRequest create_request;
|
||||
CreateRequest create_request;
|
||||
create_request.path = "/test";
|
||||
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.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";
|
||||
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)
|
||||
std::cerr << "Error (multi) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
{
|
||||
for (const auto & elem : response.responses)
|
||||
if (elem->error)
|
||||
std::cerr << "Error (elem) " << elem->error << ": " << ZooKeeper::errorMessage(elem->error) << '\n';
|
||||
for (const auto & elem : response.responses)
|
||||
if (elem->error)
|
||||
std::cerr << "Error (elem) " << elem->error << ": " << 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();
|
||||
return 0;
|
||||
|
@ -5,12 +5,12 @@
|
||||
int main()
|
||||
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)
|
||||
std::cerr << "Error " << response.error << ": " << ZooKeeperImpl::ZooKeeper::errorMessage(response.error) << "\n";
|
||||
std::cerr << "Error " << response.error << ": " << Coordination::errorMessage(response.error) << "\n";
|
||||
else
|
||||
std::cerr << "Path created: " << response.path_created << "\n";
|
||||
});
|
||||
|
@ -145,9 +145,9 @@ void BackgroundSchedulePool::TaskInfo::scheduleImpl(std::lock_guard<std::mutex>
|
||||
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();
|
||||
};
|
||||
|
@ -58,8 +58,8 @@ public:
|
||||
/// Atomically activate task and schedule it for execution.
|
||||
bool activateAndSchedule();
|
||||
|
||||
/// get zkutil::WatchCallback needed for notifications from ZooKeeper watches.
|
||||
zkutil::WatchCallback getWatchCallback();
|
||||
/// get Coordination::WatchCallback needed for notifications from ZooKeeper watches.
|
||||
Coordination::WatchCallback getWatchCallback();
|
||||
|
||||
private:
|
||||
friend class TaskNotification;
|
||||
|
@ -545,19 +545,19 @@ void DDLWorker::processTask(DDLTask & task)
|
||||
String finished_node_path = task.entry_path + "/finished/" + task.host_id_str;
|
||||
|
||||
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
|
||||
}
|
||||
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
else if (code == Coordination::ZNONODE)
|
||||
{
|
||||
/// There is no parent
|
||||
createStatusDirs(task.entry_path);
|
||||
if (ZooKeeperImpl::ZooKeeper::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy))
|
||||
throw zkutil::KeeperException(code, active_node_path);
|
||||
if (Coordination::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy))
|
||||
throw Coordination::Exception(code, active_node_path);
|
||||
}
|
||||
else
|
||||
throw zkutil::KeeperException(code, active_node_path);
|
||||
throw Coordination::Exception(code, active_node_path);
|
||||
|
||||
if (!task.was_executed)
|
||||
{
|
||||
@ -578,7 +578,7 @@ void DDLWorker::processTask(DDLTask & task)
|
||||
tryExecuteQuery(rewritten_query, task, task.execution_status);
|
||||
}
|
||||
}
|
||||
catch (const zkutil::KeeperException &)
|
||||
catch (const Coordination::Exception &)
|
||||
{
|
||||
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.
|
||||
|
||||
/// 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::makeCreateRequest(finished_node_path, task.execution_status.serializeText(), zkutil::CreateMode::Persistent));
|
||||
zookeeper->multi(ops);
|
||||
@ -734,7 +734,7 @@ void DDLWorker::cleanupQueue()
|
||||
String node_path = queue_dir + "/" + node_name;
|
||||
String lock_path = node_path + "/lock";
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
String dummy;
|
||||
|
||||
try
|
||||
@ -784,7 +784,7 @@ void DDLWorker::cleanupQueue()
|
||||
}
|
||||
|
||||
/// 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(node_path, -1));
|
||||
zookeeper->multi(ops);
|
||||
@ -803,21 +803,21 @@ void DDLWorker::cleanupQueue()
|
||||
/// Try to create nonexisting "status" dirs for a node
|
||||
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";
|
||||
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";
|
||||
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);
|
||||
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
|
||||
throw zkutil::KeeperException(code);
|
||||
if (code && code != Coordination::ZNODEEXISTS)
|
||||
throw Coordination::Exception(code);
|
||||
}
|
||||
|
||||
|
||||
@ -862,9 +862,9 @@ void DDLWorker::run()
|
||||
zookeeper->createAncestors(queue_dir + "/");
|
||||
initialized = true;
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
if (!ZooKeeperImpl::ZooKeeper::isHardwareError(e.code))
|
||||
if (!Coordination::isHardwareError(e.code))
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -890,9 +890,9 @@ void DDLWorker::run()
|
||||
/// TODO: it might delay the execution, move it to separate thread.
|
||||
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));
|
||||
|
||||
@ -1066,8 +1066,8 @@ private:
|
||||
{
|
||||
Strings res;
|
||||
int code = zookeeper->tryGetChildren(node_path, res);
|
||||
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
throw zkutil::KeeperException(code, node_path);
|
||||
if (code && code != Coordination::ZNONODE)
|
||||
throw Coordination::Exception(code, node_path);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -12,7 +12,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
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_)
|
||||
{
|
||||
/// The /abandonable_lock- name is for backward compatibility.
|
||||
@ -26,8 +26,8 @@ EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(
|
||||
else
|
||||
{
|
||||
precheck_ops->emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential));
|
||||
zkutil::Responses op_results = zookeeper->multi(*precheck_ops);
|
||||
holder_path = dynamic_cast<const zkutil::CreateResponse &>(*op_results.back()).path_created;
|
||||
Coordination::Responses op_results = zookeeper->multi(*precheck_ops);
|
||||
holder_path = dynamic_cast<const Coordination::CreateResponse &>(*op_results.back()).path_created;
|
||||
}
|
||||
|
||||
/// Write the path to the secondary node in the main node.
|
||||
@ -39,13 +39,13 @@ EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(
|
||||
|
||||
void EphemeralLockInZooKeeper::unlock()
|
||||
{
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
getUnlockOps(ops);
|
||||
zookeeper->multi(ops);
|
||||
holder_path = "";
|
||||
}
|
||||
|
||||
void EphemeralLockInZooKeeper::getUnlockOps(zkutil::Requests & ops)
|
||||
void EphemeralLockInZooKeeper::getUnlockOps(Coordination::Requests & ops)
|
||||
{
|
||||
checkCreated();
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(path, -1));
|
||||
@ -76,12 +76,12 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions(
|
||||
std::vector<String> holders;
|
||||
while (true)
|
||||
{
|
||||
zkutil::Stat partitions_stat;
|
||||
Coordination::Stat partitions_stat;
|
||||
Strings partitions = zookeeper.getChildren(block_numbers_path, &partitions_stat);
|
||||
|
||||
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)
|
||||
{
|
||||
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)
|
||||
{
|
||||
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));
|
||||
|
||||
zkutil::Responses lock_responses;
|
||||
Coordination::Responses lock_responses;
|
||||
int rc = zookeeper.tryMulti(lock_ops, lock_responses);
|
||||
if (rc == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
|
||||
if (rc == Coordination::ZBADVERSION)
|
||||
{
|
||||
LOG_TRACE(&Logger::get("EphemeralLocksInAllPartitions"),
|
||||
"Someone has inserted a block in a new partition while we were creating locks. Retry.");
|
||||
continue;
|
||||
}
|
||||
else if (rc != ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
throw zkutil::KeeperException(rc);
|
||||
else if (rc != Coordination::ZOK)
|
||||
throw Coordination::Exception(rc);
|
||||
|
||||
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();
|
||||
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)
|
||||
throw Exception("Logical error: name of the sequential node is shorter than prefix.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
@ -135,7 +135,7 @@ void EphemeralLocksInAllPartitions::unlock()
|
||||
std::vector<zkutil::ZooKeeper::FutureMulti> futures;
|
||||
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.holder_path, -1));
|
||||
futures.push_back(zookeeper.asyncMulti(unlock_ops));
|
||||
|
@ -21,7 +21,7 @@ class EphemeralLockInZooKeeper : public boost::noncopyable
|
||||
{
|
||||
public:
|
||||
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;
|
||||
|
||||
@ -61,7 +61,7 @@ public:
|
||||
void unlock();
|
||||
|
||||
/// 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,
|
||||
/// because the nodes will be already deleted.
|
||||
|
@ -53,7 +53,7 @@ void ReplicatedMergeTreeAlterThread::run()
|
||||
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
const String columns_str = zookeeper->getWatch(storage.zookeeper_path + "/columns", &stat, task->getWatchCallback());
|
||||
auto columns_in_zk = ColumnsDescription::parse(columns_str);
|
||||
|
||||
@ -139,7 +139,7 @@ void ReplicatedMergeTreeAlterThread::run()
|
||||
++changed_parts;
|
||||
|
||||
/// Update part metadata in ZooKeeper.
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
@ -151,10 +151,10 @@ void ReplicatedMergeTreeAlterThread::run()
|
||||
{
|
||||
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.
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (e.code == Coordination::ZNONODE)
|
||||
storage.enqueuePartForCheck(part->name);
|
||||
|
||||
throw;
|
||||
@ -184,11 +184,11 @@ void ReplicatedMergeTreeAlterThread::run()
|
||||
/// 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__);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
if (e.code == Coordination::ZSESSIONEXPIRED)
|
||||
return;
|
||||
|
||||
force_recheck_parts = true;
|
||||
|
@ -62,13 +62,13 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe
|
||||
{
|
||||
quorum_info.status_path = storage.zookeeper_path + "/quorum/status";
|
||||
|
||||
std::future<zkutil::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<zkutil::GetResponse> host_future = zookeeper->asyncTryGet(storage.replica_path + "/host");
|
||||
std::future<Coordination::GetResponse> quorum_status_future = zookeeper->asyncTryGet(quorum_info.status_path);
|
||||
std::future<Coordination::GetResponse> is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active");
|
||||
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.
|
||||
|
||||
zkutil::Stat leader_election_stat;
|
||||
Coordination::Stat leader_election_stat;
|
||||
zookeeper->get(storage.zookeeper_path + "/leader_election", &leader_election_stat);
|
||||
|
||||
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();
|
||||
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);
|
||||
|
||||
/// 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 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);
|
||||
|
||||
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.
|
||||
|
||||
/// Information about the part.
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
|
||||
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.
|
||||
storage.data.renameTempPartAndAdd(part, nullptr, &transaction);
|
||||
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses responses;
|
||||
int32_t multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT
|
||||
|
||||
if (multi_code == ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
if (multi_code == Coordination::ZOK)
|
||||
{
|
||||
transaction.commit();
|
||||
storage.merge_selecting_task->schedule();
|
||||
@ -309,8 +309,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
/// Lock nodes have been already deleted, do not delete them in destructor
|
||||
block_number_lock->assumeUnlocked();
|
||||
}
|
||||
else if (multi_code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS
|
||||
|| multi_code == ZooKeeperImpl::ZooKeeper::ZOPERATIONTIMEOUT)
|
||||
else if (multi_code == Coordination::ZCONNECTIONLOSS
|
||||
|| multi_code == Coordination::ZOPERATIONTIMEOUT)
|
||||
{
|
||||
/** 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.
|
||||
@ -319,14 +319,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
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.
|
||||
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);
|
||||
}
|
||||
else if (ZooKeeperImpl::ZooKeeper::isUserError(multi_code))
|
||||
else if (Coordination::isUserError(multi_code))
|
||||
{
|
||||
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.
|
||||
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")");
|
||||
@ -336,7 +336,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
last_block_is_duplicate = true;
|
||||
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();
|
||||
|
||||
@ -351,7 +351,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
|
||||
}
|
||||
}
|
||||
else if (ZooKeeperImpl::ZooKeeper::isHardwareError(multi_code))
|
||||
else if (Coordination::isHardwareError(multi_code))
|
||||
{
|
||||
transaction.rollback();
|
||||
throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': "
|
||||
|
@ -32,11 +32,11 @@ void ReplicatedMergeTreeCleanupThread::run()
|
||||
{
|
||||
iterate();
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
if (e.code == Coordination::ZSESSIONEXPIRED)
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
@ -68,7 +68,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs()
|
||||
{
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat))
|
||||
throw Exception(storage.zookeeper_path + "/log doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
|
||||
@ -99,7 +99,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs()
|
||||
if (entries.empty())
|
||||
return;
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
for (size_t i = 0; i < entries.size(); ++i)
|
||||
{
|
||||
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 = 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)
|
||||
{
|
||||
String path = storage.zookeeper_path + "/blocks/" + it->node;
|
||||
@ -163,7 +163,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
|
||||
{
|
||||
const String & path = pair.first;
|
||||
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.
|
||||
zookeeper->removeRecursive(path);
|
||||
@ -184,7 +184,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
|
||||
timed_blocks.clear();
|
||||
|
||||
Strings blocks;
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
if (zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat))
|
||||
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.");
|
||||
}
|
||||
|
||||
zkutil::AsyncResponses<zkutil::ExistsResponse> exists_futures;
|
||||
zkutil::AsyncResponses<Coordination::ExistsResponse> exists_futures;
|
||||
for (const String & block : blocks)
|
||||
{
|
||||
auto it = cached_block_stats.find(block);
|
||||
@ -227,7 +227,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
|
||||
for (auto & elem : exists_futures)
|
||||
{
|
||||
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);
|
||||
timed_blocks.emplace_back(elem.first, status.stat.ctime);
|
||||
@ -252,7 +252,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations()
|
||||
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
zkutil::Stat replicas_stat;
|
||||
Coordination::Stat replicas_stat;
|
||||
Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas", &replicas_stat);
|
||||
|
||||
UInt64 min_pointer = std::numeric_limits<UInt64>::max();
|
||||
@ -278,7 +278,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations()
|
||||
if (entries.empty())
|
||||
return;
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
size_t batch_start_i = 0;
|
||||
for (size_t i = 0; i < entries.size(); ++i)
|
||||
{
|
||||
|
@ -197,7 +197,7 @@ String ReplicatedMergeTreeLogEntryData::toString() const
|
||||
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);
|
||||
Ptr res = std::make_shared<ReplicatedMergeTreeLogEntry>();
|
||||
|
@ -158,7 +158,7 @@ struct ReplicatedMergeTreeLogEntry : public ReplicatedMergeTreeLogEntryData, std
|
||||
|
||||
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>;
|
||||
|
@ -341,11 +341,11 @@ void ReplicatedMergeTreePartCheckThread::run()
|
||||
|
||||
task->schedule();
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
if (e.code == Coordination::ZSESSIONEXPIRED)
|
||||
return;
|
||||
|
||||
task->scheduleAfter(PART_CHECK_ERROR_SLEEP_MS);
|
||||
|
@ -56,7 +56,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
|
||||
LOG_DEBUG(log, "Having " << children.size() << " queue entries to load.");
|
||||
std::sort(children.begin(), children.end());
|
||||
|
||||
zkutil::AsyncResponses<zkutil::GetResponse> futures;
|
||||
zkutil::AsyncResponses<Coordination::GetResponse> futures;
|
||||
futures.reserve(children.size());
|
||||
|
||||
for (const String & child : children)
|
||||
@ -64,7 +64,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
|
||||
|
||||
for (auto & future : futures)
|
||||
{
|
||||
zkutil::GetResponse res = future.second.get();
|
||||
Coordination::GetResponse res = future.second.get();
|
||||
LogEntryPtr entry = LogEntry::parse(res.data, res.stat);
|
||||
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.
|
||||
/// 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)
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
@ -249,7 +249,7 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper(
|
||||
|
||||
if (!ops.empty())
|
||||
{
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses responses;
|
||||
auto code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
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);
|
||||
|
||||
@ -418,7 +418,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
|
||||
|
||||
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);
|
||||
|
||||
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.
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
std::vector<LogEntryPtr> copied_entries;
|
||||
copied_entries.reserve(end - begin);
|
||||
|
||||
@ -434,7 +434,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
|
||||
|
||||
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));
|
||||
|
||||
@ -472,7 +472,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
|
||||
|
||||
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);
|
||||
|
||||
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);
|
||||
|
||||
@ -567,7 +567,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, z
|
||||
LOG_INFO(log, "Loading " + toString(entries_to_load.size()) + " mutation entries: "
|
||||
+ 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)
|
||||
futures.emplace_back(zookeeper->asyncGet(zookeeper_path + "/mutations/" + entry));
|
||||
|
||||
@ -1340,7 +1340,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
|
||||
if (!lock_holder_paths.empty())
|
||||
{
|
||||
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)
|
||||
lock_futures.push_back(zookeeper->asyncGetChildren(queue.zookeeper_path + "/block_numbers/" + partition));
|
||||
|
||||
@ -1349,7 +1349,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
|
||||
String partition;
|
||||
Int64 number;
|
||||
String zk_path;
|
||||
std::future<zkutil::GetResponse> contents_future;
|
||||
std::future<Coordination::GetResponse> contents_future;
|
||||
};
|
||||
|
||||
std::vector<BlockInfo> block_infos;
|
||||
@ -1372,7 +1372,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
|
||||
|
||||
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))
|
||||
committing_blocks[block.partition].insert(block.number);
|
||||
}
|
||||
@ -1380,11 +1380,11 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
|
||||
|
||||
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)
|
||||
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)
|
||||
{
|
||||
ReplicatedMergeTreeQuorumEntry quorum_status;
|
||||
|
@ -242,11 +242,11 @@ public:
|
||||
* 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).
|
||||
*/
|
||||
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.
|
||||
/// 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).
|
||||
* And also wait for the completion of their execution, if they are now being executed.
|
||||
|
@ -86,7 +86,7 @@ void ReplicatedMergeTreeRestartingThread::run()
|
||||
{
|
||||
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.
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
@ -196,7 +196,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
|
||||
{
|
||||
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());
|
||||
return false;
|
||||
@ -218,7 +218,7 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
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;
|
||||
|
||||
/// Firstly, remove parts from ZooKeeper
|
||||
@ -273,23 +273,23 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
|
||||
* This is possible only when session in ZooKeeper expires.
|
||||
*/
|
||||
String data;
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
bool has_is_active = zookeeper->tryGet(is_active_path, data, &stat);
|
||||
if (has_is_active && data == active_node_identifier)
|
||||
{
|
||||
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."
|
||||
" You shouldn't run multiple instances of same replica. You need to check configuration files.",
|
||||
ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);
|
||||
|
||||
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
throw zkutil::KeeperException(code, is_active_path);
|
||||
if (code && code != Coordination::ZNONODE)
|
||||
throw Coordination::Exception(code, is_active_path);
|
||||
}
|
||||
|
||||
/// 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::makeSetRequest(storage.replica_path + "/host", address.toString(), -1));
|
||||
|
||||
@ -297,9 +297,9 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
|
||||
{
|
||||
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, "
|
||||
"try again in a minute or remove znode " + storage.replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);
|
||||
|
||||
|
@ -498,7 +498,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
|
||||
/// We write metadata of table so that the replicas can check table parameters with them.
|
||||
String metadata = TableMetadata(data).toString();
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "",
|
||||
zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata,
|
||||
@ -520,10 +520,10 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "",
|
||||
zkutil::CreateMode::Persistent));
|
||||
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses responses;
|
||||
auto code = zookeeper->tryMulti(ops, responses);
|
||||
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
|
||||
throw zkutil::KeeperException(code);
|
||||
if (code && code != Coordination::ZNODEEXISTS)
|
||||
throw Coordination::Exception(code);
|
||||
}
|
||||
|
||||
|
||||
@ -537,7 +537,7 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo
|
||||
String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
|
||||
TableMetadata(data).check(metadata_str);
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
|
||||
columns_version = stat.version;
|
||||
|
||||
@ -575,7 +575,7 @@ static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const Strin
|
||||
time_t res = 0;
|
||||
|
||||
/// We get creation time of part, if it still exists (was not merged, for example).
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
String unused;
|
||||
if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat))
|
||||
res = stat.ctime / 1000;
|
||||
@ -591,7 +591,7 @@ void StorageReplicatedMergeTree::createReplica()
|
||||
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.
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
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 + "/log_pointer", "", zkutil::CreateMode::Persistent));
|
||||
@ -603,9 +603,9 @@ void StorageReplicatedMergeTree::createReplica()
|
||||
{
|
||||
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;
|
||||
@ -623,7 +623,7 @@ void StorageReplicatedMergeTree::createReplica()
|
||||
*/
|
||||
String source_replica;
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
zookeeper->exists(replica_path, &stat);
|
||||
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);
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
checkPartChecksumsAndAddCommitOps(zookeeper, part, ops);
|
||||
zookeeper->multi(ops);
|
||||
}
|
||||
@ -881,7 +881,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
|
||||
log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);
|
||||
|
||||
/// We assume that this occurs before the queue is loaded (queue.initialize).
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
removePartFromZooKeeper(name, ops);
|
||||
ops.emplace_back(zkutil::makeCreateRequest(
|
||||
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,
|
||||
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())
|
||||
part_name = part->name;
|
||||
@ -913,7 +913,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
|
||||
|
||||
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 columns_str;
|
||||
@ -986,7 +986,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
|
||||
|
||||
while (true)
|
||||
{
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
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`.
|
||||
@ -999,7 +999,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
|
||||
/// Will check that the part did not suddenly appear on skipped replicas
|
||||
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)
|
||||
{
|
||||
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 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");
|
||||
}
|
||||
@ -1488,11 +1488,11 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
|
||||
|
||||
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
|
||||
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";
|
||||
zookeeper->get(path, &stat);
|
||||
ops.emplace_back(zkutil::makeCheckRequest(path, stat.version));
|
||||
@ -1506,7 +1506,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
|
||||
|
||||
if (replica.empty())
|
||||
{
|
||||
zkutil::Stat quorum_stat;
|
||||
Coordination::Stat quorum_stat;
|
||||
String quorum_path = zookeeper_path + "/quorum/status";
|
||||
String quorum_str = zookeeper->get(quorum_path, &quorum_stat);
|
||||
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))
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1));
|
||||
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses 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.");
|
||||
queue.removeFromVirtualParts(part_info);
|
||||
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 "
|
||||
<< entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code));
|
||||
}
|
||||
else
|
||||
throw zkutil::KeeperException(code);
|
||||
throw Coordination::Exception(code);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1700,7 +1700,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
|
||||
continue;
|
||||
|
||||
/// Update part metadata in ZooKeeper.
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
@ -2003,7 +2003,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
|
||||
auto zookeeper = getZooKeeper();
|
||||
MergeTreeData::Transaction transaction;
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
for (PartDescriptionPtr & part_desc : final_parts)
|
||||
{
|
||||
data.renameTempPartAndReplace(part_desc->res_part, nullptr, &transaction);
|
||||
@ -2057,11 +2057,11 @@ void StorageReplicatedMergeTree::queueUpdatingTask()
|
||||
last_queue_update_finish_time.store(time(nullptr));
|
||||
queue_update_in_progress = false;
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
if (e.code == Coordination::ZSESSIONEXPIRED)
|
||||
{
|
||||
restarting_thread.wakeup();
|
||||
return;
|
||||
@ -2083,11 +2083,11 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask()
|
||||
{
|
||||
queue.updateMutations(getZooKeeper(), mutations_updating_task->getWatchCallback());
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
if (e.code == Coordination::ZSESSIONEXPIRED)
|
||||
return;
|
||||
|
||||
mutations_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
|
||||
@ -2282,7 +2282,7 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
|
||||
bool deduplicate,
|
||||
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());
|
||||
for (const auto & part : parts)
|
||||
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)
|
||||
{
|
||||
/// 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;
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -2388,7 +2388,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
|
||||
log_entry->source_replica = "";
|
||||
log_entry->new_part_name = part_name;
|
||||
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest(
|
||||
replica_path + "/queue/queue-", log_entry->toString(),
|
||||
zkutil::CreateMode::PersistentSequential));
|
||||
@ -2397,7 +2397,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
|
||||
|
||||
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);
|
||||
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";
|
||||
|
||||
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.
|
||||
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.
|
||||
|
||||
zkutil::Requests ops;
|
||||
zkutil::Responses responses;
|
||||
Coordination::Requests ops;
|
||||
Coordination::Responses responses;
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version));
|
||||
ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1));
|
||||
auto code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
if (code == Coordination::ZOK)
|
||||
{
|
||||
break;
|
||||
}
|
||||
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
else if (code == Coordination::ZNONODE)
|
||||
{
|
||||
/// The quorum has already been achieved.
|
||||
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.
|
||||
continue;
|
||||
}
|
||||
else
|
||||
throw zkutil::KeeperException(code, quorum_status_path);
|
||||
throw Coordination::Exception(code, quorum_status_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We update the node, registering there one more replica.
|
||||
auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);
|
||||
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
if (code == Coordination::ZOK)
|
||||
{
|
||||
break;
|
||||
}
|
||||
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
else if (code == Coordination::ZNONODE)
|
||||
{
|
||||
/// The quorum has already been achieved.
|
||||
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.
|
||||
continue;
|
||||
}
|
||||
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.
|
||||
String new_columns_str;
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
|
||||
{
|
||||
/// 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.
|
||||
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)");
|
||||
zookeeper->tryRemoveRecursive(zookeeper_path);
|
||||
@ -3452,7 +3452,7 @@ StorageReplicatedMergeTree::allocateBlockNumber(
|
||||
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
|
||||
zkutil::Requests deduplication_check_ops;
|
||||
Coordination::Requests deduplication_check_ops;
|
||||
if (!zookeeper_block_id_path.empty())
|
||||
{
|
||||
deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(zookeeper_block_id_path, "", zkutil::CreateMode::Persistent));
|
||||
@ -3464,16 +3464,16 @@ StorageReplicatedMergeTree::allocateBlockNumber(
|
||||
|
||||
if (!existsNodeCached(partition_path))
|
||||
{
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent));
|
||||
/// 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
|
||||
/// (unfortunately there is no CheckChildren op).
|
||||
ops.push_back(zkutil::makeSetRequest(block_numbers_path, "", -1));
|
||||
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses responses;
|
||||
int code = zookeeper->tryMulti(ops, responses);
|
||||
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
|
||||
if (code && code != Coordination::ZNODEEXISTS)
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
}
|
||||
|
||||
@ -3486,12 +3486,12 @@ StorageReplicatedMergeTree::allocateBlockNumber(
|
||||
}
|
||||
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 {};
|
||||
|
||||
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);
|
||||
}
|
||||
@ -3913,7 +3913,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const
|
||||
String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
|
||||
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);
|
||||
size_t queue_size = stat.numChildren;
|
||||
|
||||
@ -4085,7 +4085,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const
|
||||
{
|
||||
auto zookeeper = getZooKeeper();
|
||||
|
||||
zkutil::Stat mutations_stat;
|
||||
Coordination::Stat mutations_stat;
|
||||
zookeeper->get(mutations_path, &mutations_stat);
|
||||
|
||||
EphemeralLocksInAllPartitions block_number_locks(
|
||||
@ -4096,29 +4096,29 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const
|
||||
|
||||
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::makeCreateRequest(
|
||||
mutations_path + "/", entry.toString(), zkutil::CreateMode::PersistentSequential));
|
||||
|
||||
zkutil::Responses responses;
|
||||
Coordination::Responses responses;
|
||||
int32_t rc = zookeeper->tryMulti(requests, responses);
|
||||
|
||||
if (rc == ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
if (rc == Coordination::ZOK)
|
||||
{
|
||||
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);
|
||||
LOG_TRACE(log, "Created mutation with ID " << entry.znode_name);
|
||||
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...");
|
||||
continue;
|
||||
}
|
||||
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)
|
||||
{
|
||||
using MultiFuture = std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>;
|
||||
using MultiFuture = std::future<Coordination::MultiResponse>;
|
||||
|
||||
size_t num_tries = 0;
|
||||
bool sucess = false;
|
||||
@ -4254,7 +4254,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St
|
||||
|
||||
for (const String & part_name : part_names)
|
||||
{
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
removePartFromZooKeeper(part_name, ops);
|
||||
|
||||
futures.emplace_back(zookeeper->tryAsyncMulti(ops));
|
||||
@ -4264,23 +4264,23 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St
|
||||
{
|
||||
auto response = future.get();
|
||||
|
||||
if (response.error == 0 || response.error == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (response.error == 0 || response.error == Coordination::ZNONODE)
|
||||
continue;
|
||||
|
||||
if (ZooKeeperImpl::ZooKeeper::isHardwareError(response.error))
|
||||
if (Coordination::isHardwareError(response.error))
|
||||
{
|
||||
sucess = false;
|
||||
continue;
|
||||
}
|
||||
|
||||
throw ZooKeeperImpl::Exception(response.error);
|
||||
throw Coordination::Exception(response.error);
|
||||
}
|
||||
}
|
||||
catch (ZooKeeperImpl::Exception & e)
|
||||
catch (Coordination::Exception & e)
|
||||
{
|
||||
sucess = false;
|
||||
|
||||
if (ZooKeeperImpl::ZooKeeper::isHardwareError(e.code))
|
||||
if (Coordination::isHardwareError(e.code))
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
else
|
||||
throw;
|
||||
@ -4297,7 +4297,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St
|
||||
void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
|
||||
NameSet * parts_should_be_retried)
|
||||
{
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
auto it_first_node_in_batch = part_names.cbegin();
|
||||
|
||||
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);
|
||||
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);
|
||||
ops.clear();
|
||||
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (code == Coordination::ZNONODE)
|
||||
{
|
||||
/// Fallback
|
||||
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)
|
||||
{
|
||||
zkutil::Requests cur_ops;
|
||||
Coordination::Requests cur_ops;
|
||||
removePartFromZooKeeper(*it_in_batch, cur_ops);
|
||||
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");
|
||||
}
|
||||
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);
|
||||
}
|
||||
@ -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)
|
||||
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);
|
||||
|
||||
String partition_prefix = partition_id + "_";
|
||||
zkutil::AsyncResponses<zkutil::GetResponse> get_futures;
|
||||
zkutil::AsyncResponses<Coordination::GetResponse> get_futures;
|
||||
for (const String & block_id : blocks)
|
||||
{
|
||||
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)
|
||||
{
|
||||
const String & path = pair.first;
|
||||
auto result = pair.second.get();
|
||||
|
||||
if (result.error == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (result.error == Coordination::ZNONODE)
|
||||
continue;
|
||||
|
||||
ReadBufferFromString buf(result.data);
|
||||
@ -4391,7 +4391,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
|
||||
{
|
||||
const String & path = pair.first;
|
||||
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.
|
||||
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);
|
||||
|
||||
MergeTreeData::DataPartsVector parts_to_remove;
|
||||
zkutil::Responses op_results;
|
||||
Coordination::Responses op_results;
|
||||
|
||||
try
|
||||
{
|
||||
zkutil::Requests ops;
|
||||
Coordination::Requests ops;
|
||||
for (size_t i = 0; i < dst_parts.size(); ++i)
|
||||
{
|
||||
getCommitPartOps(ops, dst_parts[i], block_id_paths[i]);
|
||||
@ -4554,7 +4554,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
|
||||
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);
|
||||
|
||||
for (auto & lock : ephemeral_locks)
|
||||
@ -4573,7 +4573,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
|
||||
}
|
||||
|
||||
void StorageReplicatedMergeTree::getCommitPartOps(
|
||||
zkutil::Requests & ops,
|
||||
Coordination::Requests & ops,
|
||||
MergeTreeData::MutableDataPartPtr & part,
|
||||
const String & block_id_path) const
|
||||
{
|
||||
|
@ -340,7 +340,7 @@ private:
|
||||
* Call under TableStructureLock.
|
||||
*/
|
||||
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;
|
||||
|
||||
@ -349,12 +349,12 @@ private:
|
||||
const MergeTreeData::DataPartPtr & part);
|
||||
|
||||
void getCommitPartOps(
|
||||
zkutil::Requests & ops,
|
||||
Coordination::Requests & ops,
|
||||
MergeTreeData::MutableDataPartPtr & part,
|
||||
const String & block_id_path = "") const;
|
||||
|
||||
/// 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)
|
||||
void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
|
||||
|
@ -118,7 +118,7 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Contex
|
||||
if (path_part == "/")
|
||||
path_part.clear();
|
||||
|
||||
std::vector<std::future<zkutil::GetResponse>> futures;
|
||||
std::vector<std::future<Coordination::GetResponse>> futures;
|
||||
futures.reserve(nodes.size());
|
||||
for (const String & node : nodes)
|
||||
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)
|
||||
{
|
||||
auto res = futures[i].get();
|
||||
if (res.error == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
if (res.error == Coordination::ZNONODE)
|
||||
continue; /// Node was deleted meanwhile.
|
||||
|
||||
const zkutil::Stat & stat = res.stat;
|
||||
const Coordination::Stat & stat = res.stat;
|
||||
|
||||
size_t col_num = 0;
|
||||
res_columns[col_num++]->insert(nodes[i]);
|
||||
|
@ -54,7 +54,7 @@ try
|
||||
<< " partitions, elapsed: " << stage.elapsedSeconds() << "s." << std::endl;
|
||||
stage.restart();
|
||||
|
||||
std::vector<std::future<zkutil::ListResponse>> lock_futures;
|
||||
std::vector<std::future<Coordination::ListResponse>> lock_futures;
|
||||
for (const String & partition : partitions)
|
||||
lock_futures.push_back(zookeeper->asyncGetChildren(zookeeper_path + "/block_numbers/" + partition));
|
||||
|
||||
@ -63,7 +63,7 @@ try
|
||||
String partition;
|
||||
Int64 number;
|
||||
String zk_path;
|
||||
std::future<zkutil::GetResponse> contents_future;
|
||||
std::future<Coordination::GetResponse> contents_future;
|
||||
};
|
||||
|
||||
std::vector<BlockInfo> block_infos;
|
||||
@ -85,7 +85,7 @@ try
|
||||
size_t total_count = 0;
|
||||
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))
|
||||
{
|
||||
++total_count;
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
|
||||
|
||||
void printStat(const zkutil::Stat & s)
|
||||
void printStat(const Coordination::Stat & s)
|
||||
{
|
||||
std::cout << "Stat:\n";
|
||||
std::cout << " czxid: " << s.czxid << '\n';
|
||||
@ -148,7 +148,7 @@ int main(int argc, char ** argv)
|
||||
ss >> w;
|
||||
bool watch = w == "w";
|
||||
zkutil::EventPtr event = watch ? std::make_shared<Poco::Event>() : nullptr;
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
bool e = zk.exists(path, &stat, event);
|
||||
if (e)
|
||||
printStat(stat);
|
||||
@ -163,7 +163,7 @@ int main(int argc, char ** argv)
|
||||
ss >> w;
|
||||
bool watch = w == "w";
|
||||
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::cout << "Data: " << data << std::endl;
|
||||
printStat(stat);
|
||||
@ -187,7 +187,7 @@ int main(int argc, char ** argv)
|
||||
if (!in.eof())
|
||||
DB::readText(version, in);
|
||||
|
||||
zkutil::Stat stat;
|
||||
Coordination::Stat stat;
|
||||
zk.set(path, data, version, &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;
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
std::cerr << "KeeperException: " << e.displayText() << std::endl;
|
||||
return 1;
|
||||
|
@ -33,19 +33,19 @@ try
|
||||
|
||||
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));
|
||||
|
||||
for (auto it = list_futures.begin(); it != list_futures.end(); ++it)
|
||||
{
|
||||
zkutil::ListResponse response;
|
||||
Coordination::ListResponse response;
|
||||
try
|
||||
{
|
||||
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;
|
||||
throw;
|
||||
}
|
||||
|
@ -29,7 +29,7 @@ try
|
||||
zkutil::ZooKeeper zookeeper(options.at("address").as<std::string>());
|
||||
|
||||
DB::ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
std::list<std::future<zkutil::RemoveResponse>> futures;
|
||||
std::list<std::future<Coordination::RemoveResponse>> futures;
|
||||
|
||||
std::cerr << "Requested: ";
|
||||
while (!in.eof())
|
||||
|
Loading…
Reference in New Issue
Block a user