未验证 提交 debc9366 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #5643 from yandex/levysh-testkeeper

Cleanup for TestKeeper
<yandex>
<!-- <zookeeper>
<node>
<host>localhost</host>
<port>2181</port>
</node>
</zookeeper>-->
<zookeeper>
<implementation>testkeeper</implementation>
</zookeeper>
</yandex>
......@@ -19,6 +19,7 @@ namespace ErrorCodes
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
extern const int CANNOT_TRUNCATE_FILE;
extern const int NOT_IMPLEMENTED;
}
std::string errnoToString(int code, int e)
......
此差异已折叠。
#pragma once
#include <mutex>
#include <map>
#include <atomic>
#include <thread>
#include <chrono>
#include <Poco/Timespan.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ThreadPool.h>
#include <Common/ConcurrentBoundedQueue.h>
namespace Coordination
{
struct TestKeeperRequest;
using TestKeeperRequestPtr = std::shared_ptr<TestKeeperRequest>;
/** Looks like ZooKeeper but stores all data in memory of server process.
* All data is not shared between different servers and is lost after server restart.
*
* The only purpose is to more simple testing for interaction with ZooKeeper within a single server.
* This still makes sense, because multiple replicas of a single table can be created on a single server,
* and it is used to test replication logic.
*
* Does not support ACLs. Does not support NULL node values.
*
* NOTE: You can add various failure modes for better testing.
*/
class TestKeeper : public IKeeper
{
public:
TestKeeper(const String & root_path, Poco::Timespan operation_timeout);
~TestKeeper() override;
bool isExpired() const override { return expired; }
int64_t getSessionID() const override { return 0; }
void create(
const String & path,
const String & data,
bool is_ephemeral,
bool is_sequential,
const ACLs & acls,
CreateCallback callback) override;
void remove(
const String & path,
int32_t version,
RemoveCallback callback) override;
void exists(
const String & path,
ExistsCallback callback,
WatchCallback watch) override;
void get(
const String & path,
GetCallback callback,
WatchCallback watch) override;
void set(
const String & path,
const String & data,
int32_t version,
SetCallback callback) override;
void list(
const String & path,
ListCallback callback,
WatchCallback watch) override;
void check(
const String & path,
int32_t version,
CheckCallback callback) override;
void multi(
const Requests & requests,
MultiCallback callback) override;
struct Node
{
String data;
ACLs acls;
bool is_ephemeral = false;
bool is_sequental = false;
Stat stat{};
int32_t seq_num = 0;
};
using Container = std::map<std::string, Node>;
using WatchCallbacks = std::vector<WatchCallback>;
using Watches = std::map<String /* path, relative of root_path */, WatchCallbacks>;
private:
using clock = std::chrono::steady_clock;
struct RequestInfo
{
TestKeeperRequestPtr request;
ResponseCallback callback;
WatchCallback watch;
clock::time_point time;
};
Container container;
String root_path;
ACLs default_acls;
Poco::Timespan operation_timeout;
std::mutex push_request_mutex;
std::atomic<bool> expired{false};
int64_t zxid = 0;
Watches watches;
Watches list_watches; /// Watches for 'list' request (watches on children).
void createWatchCallBack(const String & path);
using RequestsQueue = ConcurrentBoundedQueue<RequestInfo>;
RequestsQueue requests_queue{1};
void pushRequest(RequestInfo && request);
void finalize();
ThreadFromGlobalPool processing_thread;
void processingThread();
};
}
#include "ZooKeeper.h"
#include "ZooKeeperImpl.h"
#include "KeeperException.h"
#include "TestKeeper.h"
#include <random>
#include <pcg_random.hpp>
......@@ -24,6 +25,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
}
......@@ -44,7 +46,7 @@ static void check(int32_t code, const std::string & path)
}
void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
void ZooKeeper::init(const std::string & implementation, const std::string & hosts_, const std::string & identity_,
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_)
{
log = &Logger::get("ZooKeeper");
......@@ -54,48 +56,61 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
operation_timeout_ms = operation_timeout_ms_;
chroot = chroot_;
if (hosts.empty())
throw KeeperException("No addresses passed to ZooKeeper constructor.", Coordination::ZBADARGUMENTS);
if (implementation == "zookeeper")
{
if (hosts.empty())
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(","));
Coordination::ZooKeeper::Addresses addresses;
addresses.reserve(addresses_strings.size());
std::vector<std::string> addresses_strings;
boost::split(addresses_strings, hosts, boost::is_any_of(","));
Coordination::ZooKeeper::Addresses addresses;
addresses.reserve(addresses_strings.size());
for (const auto & address_string : addresses_strings)
{
try
for (const auto &address_string : addresses_strings)
{
addresses.emplace_back(address_string);
}
catch (const Poco::Net::DNSException & e)
{
LOG_ERROR(log, "Cannot use ZooKeeper address " << address_string << ", reason: " << e.displayText());
try
{
addresses.emplace_back(address_string);
}
catch (const Poco::Net::DNSException &e)
{
LOG_ERROR(log, "Cannot use ZooKeeper address " << address_string << ", reason: " << e.displayText());
}
}
}
if (addresses.empty())
throw KeeperException("Cannot use any of provided ZooKeeper addresses", Coordination::ZBADARGUMENTS);
if (addresses.empty())
throw KeeperException("Cannot use any of provided ZooKeeper addresses", Coordination::ZBADARGUMENTS);
impl = std::make_unique<Coordination::ZooKeeper>(
addresses,
chroot,
identity_.empty() ? "" : "digest",
identity_,
Poco::Timespan(0, session_timeout_ms_ * 1000),
Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000),
Poco::Timespan(0, operation_timeout_ms_ * 1000));
impl = std::make_unique<Coordination::ZooKeeper>(
addresses,
chroot,
identity_.empty() ? "" : "digest",
identity_,
Poco::Timespan(0, session_timeout_ms_ * 1000),
Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000),
Poco::Timespan(0, operation_timeout_ms_ * 1000));
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
}
else if (implementation == "testkeeper")
{
impl = std::make_unique<Coordination::TestKeeper>(
chroot,
Poco::Timespan(0, operation_timeout_ms_ * 1000));
}
else
{
throw DB::Exception("Unknown implementation of coordination service: " + implementation, DB::ErrorCodes::NOT_IMPLEMENTED);
}
if (!chroot.empty() && !exists("/"))
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,
int32_t session_timeout_ms, int32_t operation_timeout_ms, const std::string & chroot)
ZooKeeper::ZooKeeper(const std::string & hosts, const std::string & identity, int32_t session_timeout_ms,
int32_t operation_timeout_ms, const std::string & chroot, const std::string & implementation)
{
init(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot);
init(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot);
}
struct ZooKeeperArgs
......@@ -109,13 +124,14 @@ struct ZooKeeperArgs
session_timeout_ms = DEFAULT_SESSION_TIMEOUT;
operation_timeout_ms = DEFAULT_OPERATION_TIMEOUT;
implementation = "zookeeper";
for (const auto & key : keys)
{
if (startsWith(key, "node"))
{
hosts_strings.push_back(
config.getString(config_name + "." + key + ".host") + ":"
+ config.getString(config_name + "." + key + ".port", "2181")
config.getString(config_name + "." + key + ".host") + ":"
+ config.getString(config_name + "." + key + ".port", "2181")
);
}
else if (key == "session_timeout_ms")
......@@ -134,6 +150,10 @@ struct ZooKeeperArgs
{
chroot = config.getString(config_name + "." + key);
}
else if (key == "implementation")
{
implementation = config.getString(config_name + "." + key);
}
else
throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::ZBADARGUMENTS);
}
......@@ -163,12 +183,13 @@ struct ZooKeeperArgs
int session_timeout_ms;
int operation_timeout_ms;
std::string chroot;
std::string implementation;
};
ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name)
{
ZooKeeperArgs args(config, config_name);
init(args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot);
init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot);
}
......@@ -181,8 +202,8 @@ static Coordination::WatchCallback callbackForEvent(const EventPtr & watch)
int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res,
Coordination::Stat * stat,
Coordination::WatchCallback watch_callback)
Coordination::Stat * stat,
Coordination::WatchCallback watch_callback)
{
int32_t code = 0;
Poco::Event event;
......@@ -205,7 +226,7 @@ int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res,
}
Strings ZooKeeper::getChildren(
const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
{
Strings res;
check(tryGetChildren(path, res, stat, watch), path);
......@@ -213,7 +234,7 @@ Strings ZooKeeper::getChildren(
}
Strings ZooKeeper::getChildrenWatch(
const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
Strings res;
check(tryGetChildrenWatch(path, res, stat, watch_callback), path);
......@@ -221,7 +242,7 @@ Strings ZooKeeper::getChildrenWatch(
}
int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res,
Coordination::Stat * stat, const EventPtr & watch)
Coordination::Stat * stat, const EventPtr & watch)
{
int32_t code = getChildrenImpl(path, res, stat, callbackForEvent(watch));
......@@ -232,7 +253,7 @@ int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res,
}
int32_t ZooKeeper::tryGetChildrenWatch(const std::string & path, Strings & res,
Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
int32_t code = getChildrenImpl(path, res, stat, watch_callback);
......@@ -272,9 +293,9 @@ int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data,
int32_t code = createImpl(path, data, mode, path_created);
if (!(code == Coordination::ZOK ||
code == Coordination::ZNONODE ||
code == Coordination::ZNODEEXISTS ||
code == Coordination::ZNOCHILDRENFOREPHEMERALS))
code == Coordination::ZNONODE ||
code == Coordination::ZNODEEXISTS ||
code == Coordination::ZNOCHILDRENFOREPHEMERALS))
throw KeeperException(code, path);
return code;
......@@ -336,9 +357,9 @@ int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version)
{
int32_t code = removeImpl(path, version);
if (!(code == Coordination::ZOK ||
code == Coordination::ZNONODE ||
code == Coordination::ZBADVERSION ||
code == Coordination::ZNOTEMPTY))
code == Coordination::ZNONODE ||
code == Coordination::ZBADVERSION ||
code == Coordination::ZNOTEMPTY))
throw KeeperException(code, path);
return code;
}
......@@ -439,7 +460,7 @@ bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Coordin
}
int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data,
int32_t version, Coordination::Stat * stat)
int32_t version, Coordination::Stat * stat)
{
int32_t code = 0;
Poco::Event event;
......@@ -474,13 +495,13 @@ void ZooKeeper::createOrUpdate(const std::string & path, const std::string & dat
}
int32_t ZooKeeper::trySet(const std::string & path, const std::string & data,
int32_t version, Coordination::Stat * stat)
int32_t version, Coordination::Stat * stat)
{
int32_t code = setImpl(path, data, version, stat);
if (!(code == Coordination::ZOK ||
code == Coordination::ZNONODE ||
code == Coordination::ZBADVERSION))
code == Coordination::ZNONODE ||
code == Coordination::ZBADVERSION))
throw KeeperException(code, path);
return code;
}
......@@ -866,15 +887,15 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t exception_code, const Coor
if (!Coordination::isUserError(exception_code))
throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(exception_code) + "' is not valid response code for that",
DB::ErrorCodes::LOGICAL_ERROR);
DB::ErrorCodes::LOGICAL_ERROR);
throw DB::Exception("There is no failed OpResult", DB::ErrorCodes::LOGICAL_ERROR);
}
KeeperMultiException::KeeperMultiException(int32_t exception_code, const Coordination::Requests & requests, const Coordination::Responses & responses)
: KeeperException("Transaction failed", exception_code),
requests(requests), responses(responses), failed_op_index(getFailedOpIndex(exception_code, responses))
: KeeperException("Transaction failed", exception_code),
requests(requests), responses(responses), failed_op_index(getFailedOpIndex(exception_code, responses))
{
addMessage("Op #" + std::to_string(failed_op_index) + ", path: " + getPathForFirstFailedOp());
}
......@@ -931,5 +952,4 @@ Coordination::RequestPtr makeCheckRequest(const std::string & path, int version)
request->version = version;
return request;
}
}
......@@ -55,7 +55,8 @@ public:
ZooKeeper(const std::string & hosts, const std::string & identity = "",
int32_t session_timeout_ms = DEFAULT_SESSION_TIMEOUT,
int32_t operation_timeout_ms = DEFAULT_OPERATION_TIMEOUT,
const std::string & chroot = "");
const std::string & chroot = "",
const std::string & implementation = "zookeeper");
/** Config of the form:
<zookeeper>
......@@ -127,7 +128,7 @@ public:
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, Coordination::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);
......@@ -136,7 +137,7 @@ 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, Coordination::Stat * stat = nullptr);
int32_t version = -1, Coordination::Stat * stat = nullptr);
Strings getChildren(const std::string & path,
Coordination::Stat * stat = nullptr,
......@@ -149,12 +150,12 @@ public:
/// Doesn't not throw in the following cases:
/// * The node doesn't exist.
int32_t tryGetChildren(const std::string & path, Strings & res,
Coordination::Stat * stat = nullptr,
const EventPtr & watch = nullptr);
Coordination::Stat * stat = nullptr,
const EventPtr & watch = nullptr);
int32_t tryGetChildrenWatch(const std::string & path, Strings & res,
Coordination::Stat * stat,
Coordination::WatchCallback watch_callback);
Coordination::Stat * stat,
Coordination::WatchCallback watch_callback);
/// Performs several operations in a transaction.
/// Throws on every error.
......@@ -235,7 +236,7 @@ public:
private:
friend class EphemeralNodeHolder;
void init(const std::string & hosts_, const std::string & identity_,
void init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_,
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_);
void removeChildrenRecursive(const std::string & path);
......@@ -274,7 +275,7 @@ public:
using Ptr = std::shared_ptr<EphemeralNodeHolder>;
EphemeralNodeHolder(const std::string & path_, ZooKeeper & zookeeper_, bool create, bool sequential, const std::string & data)
: path(path_), zookeeper(zookeeper_)
: path(path_), zookeeper(zookeeper_)
{
if (create)
path = zookeeper.create(path, data, sequential ? CreateMode::EphemeralSequential : CreateMode::Ephemeral);
......@@ -320,5 +321,4 @@ private:
};
using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr;
}
<yandex>
<zookeeper>
<node>
<host>localhost</host>
<port>2181</port>
</node>
<node>
<host>yandex.ru</host>
<port>2181</port>
</node>
<node>
<host>111.0.1.2</host>
<port>2181</port>
</node>
<implementation>testkeeper</implementation>
</zookeeper>
</yandex>
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册