2022-05-06 12:25:25 +00:00
|
|
|
#include <iterator>
|
2022-05-19 09:45:38 +00:00
|
|
|
#include <variant>
|
2021-03-29 08:24:56 +00:00
|
|
|
#include <Coordination/KeeperStorage.h>
|
2022-05-06 12:25:25 +00:00
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <boost/algorithm/string.hpp>
|
|
|
|
#include <Poco/Base64Encoder.h>
|
|
|
|
#include <Poco/SHA1Engine.h>
|
2022-06-29 08:30:39 +00:00
|
|
|
#include "Common/ZooKeeper/ZooKeeperCommon.h"
|
2022-06-13 15:43:10 +00:00
|
|
|
#include <Common/SipHash.h>
|
|
|
|
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
2020-11-04 18:54:55 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2022-05-06 12:25:25 +00:00
|
|
|
#include <Common/ZooKeeper/IKeeper.h>
|
2022-01-10 19:01:41 +00:00
|
|
|
#include <Common/hex.h>
|
2022-04-27 15:05:45 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2022-05-06 12:25:25 +00:00
|
|
|
#include <Common/setThreadName.h>
|
2022-05-25 14:45:32 +00:00
|
|
|
#include <Coordination/pathUtils.h>
|
|
|
|
#include <sstream>
|
|
|
|
#include <iomanip>
|
|
|
|
#include <mutex>
|
|
|
|
#include <functional>
|
2022-06-11 22:01:41 +00:00
|
|
|
#include <base/defines.h>
|
2022-06-27 06:53:09 +00:00
|
|
|
#include <filesystem>
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2020-11-11 13:55:28 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2020-11-25 13:19:09 +00:00
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int BAD_ARGUMENTS;
|
|
|
|
}
|
|
|
|
|
2022-04-05 06:27:03 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
String base64Encode(const String & decoded)
|
|
|
|
{
|
|
|
|
std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
|
|
|
ostr.exceptions(std::ios::failbit);
|
|
|
|
Poco::Base64Encoder encoder(ostr);
|
|
|
|
encoder.rdbuf()->setLineLength(0);
|
|
|
|
encoder << decoded;
|
|
|
|
encoder.close();
|
|
|
|
return ostr.str();
|
|
|
|
}
|
2021-05-21 21:19:22 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
String getSHA1(const String & userdata)
|
|
|
|
{
|
|
|
|
Poco::SHA1Engine engine;
|
|
|
|
engine.update(userdata);
|
|
|
|
const auto & digest_id = engine.digest();
|
|
|
|
return String{digest_id.begin(), digest_id.end()};
|
|
|
|
}
|
2021-05-22 16:07:47 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
String generateDigest(const String & userdata)
|
|
|
|
{
|
|
|
|
std::vector<String> user_password;
|
|
|
|
boost::split(user_password, userdata, [](char character) { return character == ':'; });
|
|
|
|
return user_password[0] + ":" + base64Encode(getSHA1(userdata));
|
|
|
|
}
|
2021-05-21 21:19:22 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
bool fixupACL(
|
|
|
|
const std::vector<Coordination::ACL> & request_acls,
|
|
|
|
const std::vector<KeeperStorage::AuthID> & current_ids,
|
|
|
|
std::vector<Coordination::ACL> & result_acls)
|
|
|
|
{
|
|
|
|
if (request_acls.empty())
|
|
|
|
return true;
|
|
|
|
|
|
|
|
bool valid_found = false;
|
|
|
|
for (const auto & request_acl : request_acls)
|
|
|
|
{
|
|
|
|
if (request_acl.scheme == "auth")
|
2021-05-22 16:21:52 +00:00
|
|
|
{
|
2022-05-09 09:57:06 +00:00
|
|
|
for (const auto & current_id : current_ids)
|
2021-05-22 16:21:52 +00:00
|
|
|
{
|
2021-05-24 12:18:04 +00:00
|
|
|
valid_found = true;
|
2022-05-06 12:25:25 +00:00
|
|
|
Coordination::ACL new_acl = request_acl;
|
2022-05-09 09:57:06 +00:00
|
|
|
new_acl.scheme = current_id.scheme;
|
|
|
|
new_acl.id = current_id.id;
|
2022-05-06 12:25:25 +00:00
|
|
|
result_acls.push_back(new_acl);
|
|
|
|
}
|
2021-05-24 12:18:04 +00:00
|
|
|
}
|
2022-05-09 09:57:06 +00:00
|
|
|
else if (request_acl.scheme == "world" && request_acl.id == "anyone")
|
|
|
|
{
|
|
|
|
/// We don't need to save default ACLs
|
|
|
|
valid_found = true;
|
|
|
|
}
|
|
|
|
else if (request_acl.scheme == "digest")
|
|
|
|
{
|
|
|
|
Coordination::ACL new_acl = request_acl;
|
|
|
|
|
|
|
|
/// Bad auth
|
|
|
|
if (std::count(new_acl.id.begin(), new_acl.id.end(), ':') != 1)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
valid_found = true;
|
|
|
|
result_acls.push_back(new_acl);
|
|
|
|
}
|
2021-05-22 16:21:52 +00:00
|
|
|
}
|
2022-05-09 09:57:06 +00:00
|
|
|
return valid_found;
|
|
|
|
}
|
2021-05-22 16:21:52 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
KeeperStorage::ResponsesForSessions processWatchesImpl(
|
|
|
|
const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type)
|
|
|
|
{
|
|
|
|
KeeperStorage::ResponsesForSessions result;
|
|
|
|
auto watch_it = watches.find(path);
|
|
|
|
if (watch_it != watches.end())
|
2020-11-10 13:43:10 +00:00
|
|
|
{
|
2022-05-09 09:57:06 +00:00
|
|
|
std::shared_ptr<Coordination::ZooKeeperWatchResponse> watch_response = std::make_shared<Coordination::ZooKeeperWatchResponse>();
|
|
|
|
watch_response->path = path;
|
|
|
|
watch_response->xid = Coordination::WATCH_XID;
|
|
|
|
watch_response->zxid = -1;
|
|
|
|
watch_response->type = event_type;
|
|
|
|
watch_response->state = Coordination::State::CONNECTED;
|
|
|
|
for (auto watcher_session : watch_it->second)
|
|
|
|
result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_response});
|
|
|
|
|
|
|
|
watches.erase(watch_it);
|
|
|
|
}
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
auto parent_path = parentPath(path);
|
2021-08-26 11:50:08 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
Strings paths_to_check_for_list_watches;
|
|
|
|
if (event_type == Coordination::Event::CREATED)
|
|
|
|
{
|
|
|
|
paths_to_check_for_list_watches.push_back(parent_path.toString()); /// Trigger list watches for parent
|
|
|
|
}
|
|
|
|
else if (event_type == Coordination::Event::DELETED)
|
|
|
|
{
|
|
|
|
paths_to_check_for_list_watches.push_back(path); /// Trigger both list watches for this path
|
|
|
|
paths_to_check_for_list_watches.push_back(parent_path.toString()); /// And for parent path
|
|
|
|
}
|
|
|
|
/// CHANGED event never trigger list wathes
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
for (const auto & path_to_check : paths_to_check_for_list_watches)
|
|
|
|
{
|
|
|
|
watch_it = list_watches.find(path_to_check);
|
|
|
|
if (watch_it != list_watches.end())
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-09 09:57:06 +00:00
|
|
|
std::shared_ptr<Coordination::ZooKeeperWatchResponse> watch_list_response
|
|
|
|
= std::make_shared<Coordination::ZooKeeperWatchResponse>();
|
|
|
|
watch_list_response->path = path_to_check;
|
|
|
|
watch_list_response->xid = Coordination::WATCH_XID;
|
|
|
|
watch_list_response->zxid = -1;
|
|
|
|
if (path_to_check == parent_path)
|
|
|
|
watch_list_response->type = Coordination::Event::CHILD;
|
|
|
|
else
|
|
|
|
watch_list_response->type = Coordination::Event::DELETED;
|
|
|
|
|
|
|
|
watch_list_response->state = Coordination::State::CONNECTED;
|
|
|
|
for (auto watcher_session : watch_it->second)
|
|
|
|
result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_list_response});
|
2021-08-26 11:50:08 +00:00
|
|
|
|
2022-05-09 09:57:06 +00:00
|
|
|
list_watches.erase(watch_it);
|
2021-08-26 11:50:08 +00:00
|
|
|
}
|
2020-11-10 13:43:10 +00:00
|
|
|
}
|
2022-05-09 09:57:06 +00:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
// When this function is updated, update CURRENT_DIGEST_VERSION!!
|
2022-06-15 12:48:30 +00:00
|
|
|
uint64_t calculateDigest(std::string_view path, std::string_view data, const Coordination::Stat & stat)
|
2022-05-13 13:43:42 +00:00
|
|
|
{
|
|
|
|
SipHash hash;
|
|
|
|
|
|
|
|
hash.update(path);
|
|
|
|
|
|
|
|
hash.update(data);
|
|
|
|
|
|
|
|
hash.update(stat.czxid);
|
|
|
|
hash.update(stat.czxid);
|
|
|
|
hash.update(stat.mzxid);
|
|
|
|
hash.update(stat.ctime);
|
|
|
|
hash.update(stat.mtime);
|
|
|
|
hash.update(stat.version);
|
|
|
|
hash.update(stat.cversion);
|
|
|
|
hash.update(stat.aversion);
|
|
|
|
hash.update(stat.ephemeralOwner);
|
|
|
|
hash.update(stat.dataLength);
|
|
|
|
hash.update(stat.numChildren);
|
|
|
|
hash.update(stat.pzxid);
|
|
|
|
|
|
|
|
return hash.get64();
|
|
|
|
}
|
|
|
|
|
2022-04-05 06:27:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void KeeperStorage::Node::setData(String new_data)
|
|
|
|
{
|
|
|
|
size_bytes = size_bytes - data.size() + new_data.size();
|
|
|
|
data = std::move(new_data);
|
|
|
|
}
|
|
|
|
|
|
|
|
void KeeperStorage::Node::addChild(StringRef child_path)
|
|
|
|
{
|
|
|
|
size_bytes += sizeof child_path;
|
|
|
|
children.insert(child_path);
|
|
|
|
}
|
|
|
|
|
|
|
|
void KeeperStorage::Node::removeChild(StringRef child_path)
|
|
|
|
{
|
|
|
|
size_bytes -= sizeof child_path;
|
|
|
|
children.erase(child_path);
|
|
|
|
}
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
void KeeperStorage::Node::invalidateDigestCache() const
|
|
|
|
{
|
|
|
|
cached_digest.reset();
|
|
|
|
}
|
|
|
|
|
|
|
|
UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const
|
|
|
|
{
|
|
|
|
if (!cached_digest)
|
|
|
|
cached_digest = calculateDigest(path, data, stat);
|
|
|
|
|
|
|
|
return *cached_digest;
|
|
|
|
};
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other)
|
2022-05-13 13:43:42 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
stat = other.stat;
|
|
|
|
seq_num = other.seq_num;
|
|
|
|
setData(other.getData());
|
|
|
|
cached_digest = other.cached_digest;
|
2022-05-13 13:43:42 +00:00
|
|
|
}
|
|
|
|
|
2022-05-16 13:08:10 +00:00
|
|
|
KeeperStorage::KeeperStorage(int64_t tick_time_ms, const String & superdigest_, const bool digest_enabled_)
|
|
|
|
: session_expiry_queue(tick_time_ms), digest_enabled(digest_enabled_), superdigest(superdigest_)
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
Node root_node;
|
|
|
|
container.insert("/", root_node);
|
|
|
|
nodes_digest += root_node.getDigest("/");
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
template <class... Ts>
|
|
|
|
struct Overloaded : Ts...
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
using Ts::operator()...;
|
|
|
|
};
|
2022-05-17 06:45:51 +00:00
|
|
|
|
|
|
|
// explicit deduction guide
|
|
|
|
// https://en.cppreference.com/w/cpp/language/class_template_argument_deduction
|
2022-05-06 12:25:25 +00:00
|
|
|
template <class... Ts>
|
|
|
|
Overloaded(Ts...) -> Overloaded<Ts...>;
|
|
|
|
|
2022-06-15 12:48:30 +00:00
|
|
|
std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
if (auto node_it = storage.container.find(path); node_it != storage.container.end())
|
|
|
|
{
|
|
|
|
const auto & committed_node = node_it->value;
|
|
|
|
auto node = std::make_shared<KeeperStorage::Node>();
|
|
|
|
node->shallowCopy(committed_node);
|
|
|
|
return node;
|
|
|
|
}
|
|
|
|
|
|
|
|
return nullptr;
|
|
|
|
}
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
void KeeperStorage::UncommittedState::applyDelta(const Delta & delta)
|
|
|
|
{
|
|
|
|
assert(!delta.path.empty());
|
|
|
|
if (!nodes.contains(delta.path))
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-06-15 12:48:30 +00:00
|
|
|
if (auto storage_node = tryGetNodeFromStorage(delta.path))
|
2022-05-19 09:45:38 +00:00
|
|
|
nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)});
|
|
|
|
else
|
|
|
|
nodes.emplace(delta.path, UncommittedNode{.node = nullptr});
|
2022-05-05 10:32:41 +00:00
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
auto & [node, acls, last_applied_zxid] = nodes.at(delta.path);
|
|
|
|
|
|
|
|
std::visit(
|
|
|
|
[&, &node = node, &acls = acls, &last_applied_zxid = last_applied_zxid]<typename DeltaType>(const DeltaType & operation)
|
|
|
|
{
|
|
|
|
if constexpr (std::same_as<DeltaType, CreateNodeDelta>)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
assert(!node);
|
|
|
|
node = std::make_shared<Node>();
|
2022-05-19 09:45:38 +00:00
|
|
|
node->stat = operation.stat;
|
|
|
|
node->setData(operation.data);
|
|
|
|
acls = operation.acls;
|
|
|
|
last_applied_zxid = delta.zxid;
|
|
|
|
}
|
|
|
|
else if constexpr (std::same_as<DeltaType, RemoveNodeDelta>)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
assert(node);
|
|
|
|
node = nullptr;
|
2022-05-19 09:45:38 +00:00
|
|
|
last_applied_zxid = delta.zxid;
|
|
|
|
}
|
|
|
|
else if constexpr (std::same_as<DeltaType, UpdateNodeDelta>)
|
2022-05-09 11:53:19 +00:00
|
|
|
{
|
|
|
|
assert(node);
|
2022-05-16 12:12:29 +00:00
|
|
|
node->invalidateDigestCache();
|
2022-05-19 09:45:38 +00:00
|
|
|
operation.update_fn(*node);
|
|
|
|
last_applied_zxid = delta.zxid;
|
|
|
|
}
|
|
|
|
else if constexpr (std::same_as<DeltaType, SetACLDelta>)
|
|
|
|
{
|
|
|
|
acls = operation.acls;
|
|
|
|
last_applied_zxid = delta.zxid;
|
|
|
|
}
|
2022-05-13 13:43:42 +00:00
|
|
|
},
|
2022-05-19 09:45:38 +00:00
|
|
|
delta.operation);
|
|
|
|
}
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
void KeeperStorage::UncommittedState::addDeltas(std::vector<Delta> new_deltas)
|
|
|
|
{
|
|
|
|
for (auto & delta : new_deltas)
|
|
|
|
{
|
|
|
|
if (!delta.path.empty())
|
|
|
|
applyDelta(delta);
|
|
|
|
|
|
|
|
deltas.push_back(std::move(delta));
|
|
|
|
}
|
2022-05-06 12:25:25 +00:00
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
void KeeperStorage::UncommittedState::commit(int64_t commit_zxid)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
assert(deltas.empty() || deltas.front().zxid >= commit_zxid);
|
|
|
|
|
|
|
|
while (!deltas.empty() && deltas.front().zxid == commit_zxid)
|
|
|
|
{
|
|
|
|
if (std::holds_alternative<SubDeltaEnd>(deltas.front().operation))
|
|
|
|
{
|
|
|
|
deltas.pop_front();
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
deltas.pop_front();
|
|
|
|
}
|
|
|
|
|
2022-05-23 07:55:23 +00:00
|
|
|
// delete all cached nodes that were not modified after the commit_zxid
|
2022-05-23 14:37:57 +00:00
|
|
|
// the commit can end on SubDeltaEnd so we don't want to clear cached nodes too soon
|
|
|
|
if (deltas.empty() || deltas.front().zxid > commit_zxid)
|
|
|
|
std::erase_if(nodes, [commit_zxid](const auto & node) { return node.second.zxid == commit_zxid; });
|
2022-05-19 09:45:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid)
|
|
|
|
{
|
|
|
|
// we can only rollback the last zxid (if there is any)
|
|
|
|
// if there is a delta with a larger zxid, we have invalid state
|
2022-05-25 14:21:26 +00:00
|
|
|
if (!deltas.empty() && deltas.back().zxid > rollback_zxid)
|
2022-05-23 07:55:23 +00:00
|
|
|
throw DB::Exception(
|
2022-05-19 09:45:38 +00:00
|
|
|
DB::ErrorCodes::LOGICAL_ERROR,
|
|
|
|
"Invalid state of deltas found while trying to rollback request. Last ZXID ({}) is larger than the requested ZXID ({})",
|
2022-05-25 14:21:26 +00:00
|
|
|
deltas.back().zxid,
|
2022-05-23 07:55:23 +00:00
|
|
|
rollback_zxid);
|
2022-05-19 09:45:38 +00:00
|
|
|
|
2022-05-23 14:37:57 +00:00
|
|
|
// we need to undo ephemeral mapping modifications
|
|
|
|
// CreateNodeDelta added ephemeral for session id -> we need to remove it
|
|
|
|
// RemoveNodeDelta removed ephemeral for session id -> we need to add it back
|
|
|
|
for (auto delta_it = deltas.rbegin(); delta_it != deltas.rend(); ++delta_it)
|
|
|
|
{
|
|
|
|
if (delta_it->zxid < rollback_zxid)
|
|
|
|
break;
|
|
|
|
|
|
|
|
assert(delta_it->zxid == rollback_zxid);
|
|
|
|
if (!delta_it->path.empty())
|
|
|
|
{
|
|
|
|
std::visit(
|
|
|
|
[&]<typename DeltaType>(const DeltaType & operation)
|
|
|
|
{
|
|
|
|
if constexpr (std::same_as<DeltaType, CreateNodeDelta>)
|
|
|
|
{
|
|
|
|
if (operation.stat.ephemeralOwner != 0)
|
|
|
|
storage.unregisterEphemeralPath(operation.stat.ephemeralOwner, delta_it->path);
|
|
|
|
}
|
|
|
|
else if constexpr (std::same_as<DeltaType, RemoveNodeDelta>)
|
|
|
|
{
|
|
|
|
if (operation.ephemeral_owner != 0)
|
|
|
|
storage.ephemerals[operation.ephemeral_owner].emplace(delta_it->path);
|
|
|
|
}
|
|
|
|
},
|
|
|
|
delta_it->operation);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
std::erase_if(deltas, [rollback_zxid](const auto & delta) { return delta.zxid == rollback_zxid; });
|
|
|
|
|
|
|
|
std::unordered_set<std::string> deleted_nodes;
|
|
|
|
std::erase_if(
|
|
|
|
nodes,
|
|
|
|
[&, rollback_zxid](const auto & node)
|
|
|
|
{
|
|
|
|
if (node.second.zxid == rollback_zxid)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
deleted_nodes.emplace(node.first);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
return false;
|
2022-05-06 12:25:25 +00:00
|
|
|
});
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
// recalculate all the uncommitted deleted nodes
|
|
|
|
for (const auto & delta : deltas)
|
|
|
|
{
|
|
|
|
if (!delta.path.empty() && deleted_nodes.contains(delta.path))
|
|
|
|
applyDelta(delta);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::getNode(StringRef path) const
|
|
|
|
{
|
|
|
|
if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end())
|
|
|
|
return node_it->second.node;
|
|
|
|
|
2022-06-15 12:48:30 +00:00
|
|
|
return tryGetNodeFromStorage(path);
|
2022-05-05 10:32:41 +00:00
|
|
|
}
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const
|
2022-05-09 09:16:05 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end())
|
|
|
|
return node_it->second.acls;
|
2022-05-09 09:16:05 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
auto node_it = storage.container.find(path);
|
|
|
|
if (node_it == storage.container.end())
|
|
|
|
return {};
|
2022-05-09 09:16:05 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
return storage.acl_map.convertNumber(node_it->value.acl_id);
|
2022-05-09 09:16:05 +00:00
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
namespace
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-05-11 12:10:17 +00:00
|
|
|
|
2022-05-20 10:12:09 +00:00
|
|
|
[[noreturn]] void onStorageInconsistency()
|
|
|
|
{
|
|
|
|
LOG_ERROR(
|
|
|
|
&Poco::Logger::get("KeeperStorage"),
|
|
|
|
"Inconsistency found between uncommitted and committed data. Keeper will terminate to avoid undefined behaviour.");
|
|
|
|
std::terminate();
|
|
|
|
}
|
2022-05-11 12:10:17 +00:00
|
|
|
|
2022-05-05 10:32:41 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::Error KeeperStorage::commit(int64_t commit_zxid)
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-05-17 10:15:12 +00:00
|
|
|
// Deltas are added with increasing ZXIDs
|
|
|
|
// If there are no deltas for the commit_zxid (e.g. read requests), we instantly return
|
|
|
|
// on first delta
|
2022-05-11 09:08:39 +00:00
|
|
|
for (auto & delta : uncommitted_state.deltas)
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
if (delta.zxid > commit_zxid)
|
|
|
|
break;
|
|
|
|
|
2022-05-09 07:02:11 +00:00
|
|
|
bool finish_subdelta = false;
|
2022-05-06 12:25:25 +00:00
|
|
|
auto result = std::visit(
|
2022-05-10 09:53:15 +00:00
|
|
|
[&, &path = delta.path]<typename DeltaType>(DeltaType & operation) -> Coordination::Error
|
2022-05-09 09:42:23 +00:00
|
|
|
{
|
|
|
|
if constexpr (std::same_as<DeltaType, KeeperStorage::CreateNodeDelta>)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
if (!createNode(
|
|
|
|
path,
|
2022-05-09 09:42:23 +00:00
|
|
|
std::move(operation.data),
|
|
|
|
operation.stat,
|
|
|
|
operation.is_sequental,
|
2022-05-23 07:51:26 +00:00
|
|
|
std::move(operation.acls)))
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
return Coordination::Error::ZOK;
|
2022-05-09 09:42:23 +00:00
|
|
|
}
|
|
|
|
else if constexpr (std::same_as<DeltaType, KeeperStorage::UpdateNodeDelta>)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
auto node_it = container.find(path);
|
|
|
|
if (node_it == container.end())
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-09 09:42:23 +00:00
|
|
|
if (operation.version != -1 && operation.version != node_it->value.stat.version)
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-16 13:08:10 +00:00
|
|
|
removeDigest(node_it->value, path);
|
2022-05-13 13:43:42 +00:00
|
|
|
auto updated_node = container.updateValue(path, operation.update_fn);
|
2022-05-16 13:08:10 +00:00
|
|
|
addDigest(updated_node->value, path);
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return Coordination::Error::ZOK;
|
2022-05-09 09:42:23 +00:00
|
|
|
}
|
|
|
|
else if constexpr (std::same_as<DeltaType, KeeperStorage::RemoveNodeDelta>)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-09 09:42:23 +00:00
|
|
|
if (!removeNode(path, operation.version))
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
return Coordination::Error::ZOK;
|
2022-05-09 09:42:23 +00:00
|
|
|
}
|
|
|
|
else if constexpr (std::same_as<DeltaType, KeeperStorage::SetACLDelta>)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
auto node_it = container.find(path);
|
2022-05-11 07:53:32 +00:00
|
|
|
if (node_it == container.end())
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-09 09:42:23 +00:00
|
|
|
if (operation.version != -1 && operation.version != node_it->value.stat.aversion)
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
acl_map.removeUsage(node_it->value.acl_id);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-09 09:42:23 +00:00
|
|
|
uint64_t acl_id = acl_map.convertACLs(operation.acls);
|
2022-05-06 12:25:25 +00:00
|
|
|
acl_map.addUsage(acl_id);
|
|
|
|
|
|
|
|
container.updateValue(path, [acl_id](KeeperStorage::Node & node) { node.acl_id = acl_id; });
|
|
|
|
|
|
|
|
return Coordination::Error::ZOK;
|
2022-05-09 09:42:23 +00:00
|
|
|
}
|
2022-05-09 10:19:20 +00:00
|
|
|
else if constexpr (std::same_as<DeltaType, KeeperStorage::ErrorDelta>)
|
2022-05-09 09:42:23 +00:00
|
|
|
return operation.error;
|
|
|
|
else if constexpr (std::same_as<DeltaType, KeeperStorage::SubDeltaEnd>)
|
2022-05-09 07:02:11 +00:00
|
|
|
{
|
|
|
|
finish_subdelta = true;
|
|
|
|
return Coordination::Error::ZOK;
|
2022-05-09 09:42:23 +00:00
|
|
|
}
|
2022-05-10 12:53:18 +00:00
|
|
|
else if constexpr (std::same_as<DeltaType, KeeperStorage::AddAuthDelta>)
|
|
|
|
{
|
|
|
|
session_and_auth[operation.session_id].emplace_back(std::move(operation.auth_id));
|
|
|
|
return Coordination::Error::ZOK;
|
|
|
|
}
|
2022-05-09 09:42:23 +00:00
|
|
|
else
|
2022-05-09 07:02:11 +00:00
|
|
|
{
|
2022-05-09 09:42:23 +00:00
|
|
|
// shouldn't be called in any process functions
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-09 09:42:23 +00:00
|
|
|
}
|
|
|
|
},
|
2022-05-06 12:25:25 +00:00
|
|
|
delta.operation);
|
|
|
|
|
|
|
|
if (result != Coordination::Error::ZOK)
|
|
|
|
return result;
|
2022-05-09 07:02:11 +00:00
|
|
|
|
|
|
|
if (finish_subdelta)
|
|
|
|
return Coordination::Error::ZOK;
|
2022-05-06 12:25:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return Coordination::Error::ZOK;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool KeeperStorage::createNode(
|
|
|
|
const std::string & path,
|
|
|
|
String data,
|
|
|
|
const Coordination::Stat & stat,
|
|
|
|
bool is_sequental,
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ACLs node_acls)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
auto parent_path = parentPath(path);
|
2022-05-09 09:16:05 +00:00
|
|
|
auto node_it = container.find(parent_path);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
if (node_it == container.end())
|
2022-05-06 12:25:25 +00:00
|
|
|
return false;
|
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
if (node_it->value.stat.ephemeralOwner != 0)
|
2022-05-06 12:25:25 +00:00
|
|
|
return false;
|
|
|
|
|
|
|
|
if (container.contains(path))
|
|
|
|
return false;
|
|
|
|
|
|
|
|
KeeperStorage::Node created_node;
|
|
|
|
|
|
|
|
uint64_t acl_id = acl_map.convertACLs(node_acls);
|
|
|
|
acl_map.addUsage(acl_id);
|
|
|
|
|
|
|
|
created_node.acl_id = acl_id;
|
|
|
|
created_node.stat = stat;
|
|
|
|
created_node.setData(std::move(data));
|
|
|
|
created_node.is_sequental = is_sequental;
|
|
|
|
auto [map_key, _] = container.insert(path, created_node);
|
|
|
|
/// Take child path from key owned by map.
|
|
|
|
auto child_path = getBaseName(map_key->getKey());
|
2022-06-14 07:17:11 +00:00
|
|
|
container.updateValue(
|
|
|
|
parent_path,
|
|
|
|
[child_path](KeeperStorage::Node & parent)
|
|
|
|
{
|
|
|
|
parent.addChild(child_path);
|
|
|
|
chassert(parent.stat.numChildren == static_cast<int32_t>(parent.getChildren().size()));
|
|
|
|
}
|
|
|
|
);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-16 13:08:10 +00:00
|
|
|
addDigest(map_key->getMapped()->value, map_key->getKey().toView());
|
2022-05-06 12:25:25 +00:00
|
|
|
return true;
|
|
|
|
};
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
bool KeeperStorage::removeNode(const std::string & path, int32_t version)
|
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
auto node_it = container.find(path);
|
|
|
|
if (node_it == container.end())
|
2022-05-06 12:25:25 +00:00
|
|
|
return false;
|
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
if (version != -1 && version != node_it->value.stat.version)
|
2022-05-06 12:25:25 +00:00
|
|
|
return false;
|
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
if (node_it->value.stat.numChildren)
|
2022-05-06 12:25:25 +00:00
|
|
|
return false;
|
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
auto prev_node = node_it->value;
|
2022-05-06 12:25:25 +00:00
|
|
|
acl_map.removeUsage(prev_node.acl_id);
|
|
|
|
|
|
|
|
container.updateValue(
|
2022-05-09 09:16:05 +00:00
|
|
|
parentPath(path),
|
2022-06-14 07:17:11 +00:00
|
|
|
[child_basename = getBaseName(node_it->key)](KeeperStorage::Node & parent)
|
|
|
|
{
|
|
|
|
parent.removeChild(child_basename);
|
|
|
|
chassert(parent.stat.numChildren == static_cast<int32_t>(parent.getChildren().size()));
|
|
|
|
}
|
|
|
|
);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
container.erase(path);
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2022-05-16 13:08:10 +00:00
|
|
|
removeDigest(prev_node, path);
|
2022-05-06 12:25:25 +00:00
|
|
|
return true;
|
2022-05-05 10:32:41 +00:00
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageRequestProcessor
|
2020-10-30 19:57:30 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperRequestPtr zk_request;
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
explicit KeeperStorageRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) { }
|
2022-05-23 07:51:26 +00:00
|
|
|
virtual Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const = 0;
|
2022-05-06 12:25:25 +00:00
|
|
|
virtual std::vector<KeeperStorage::Delta>
|
2022-05-19 09:45:38 +00:00
|
|
|
preprocess(KeeperStorage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/) const
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
2022-05-09 08:32:25 +00:00
|
|
|
|
|
|
|
// process the request using locally committed data
|
|
|
|
virtual Coordination::ZooKeeperResponsePtr
|
2022-05-23 07:51:26 +00:00
|
|
|
processLocal(KeeperStorage & /*storage*/, int64_t /*zxid*/) const
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
|
|
|
throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"};
|
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
virtual KeeperStorage::ResponsesForSessions
|
|
|
|
processWatches(KeeperStorage::Watches & /*watches*/, KeeperStorage::Watches & /*list_watches*/) const
|
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
2022-05-09 09:16:05 +00:00
|
|
|
virtual bool checkAuth(KeeperStorage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) const { return true; }
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
virtual ~KeeperStorageRequestProcessor() = default;
|
2020-11-20 12:36:10 +00:00
|
|
|
};
|
2020-10-30 19:57:30 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-06 12:25:25 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr
|
2022-05-23 07:51:26 +00:00
|
|
|
process(KeeperStorage & /* storage */, int64_t /* zxid */) const override
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2022-05-09 11:53:19 +00:00
|
|
|
return zk_request->makeResponse();
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor
|
2021-02-10 13:01:05 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-06 12:25:25 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr
|
2022-05-23 07:51:26 +00:00
|
|
|
process(KeeperStorage & /* storage */, int64_t /* zxid */) const override
|
2021-02-10 13:01:05 +00:00
|
|
|
{
|
|
|
|
auto response = zk_request->makeResponse();
|
2021-05-08 14:10:06 +00:00
|
|
|
dynamic_cast<Coordination::ZooKeeperSyncResponse &>(*response).path
|
|
|
|
= dynamic_cast<Coordination::ZooKeeperSyncRequest &>(*zk_request).path;
|
2022-05-09 07:02:11 +00:00
|
|
|
return response;
|
2021-02-10 13:01:05 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
namespace
|
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
|
2022-05-10 12:53:18 +00:00
|
|
|
Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_local)
|
2022-05-09 09:16:05 +00:00
|
|
|
{
|
|
|
|
if (is_local)
|
|
|
|
{
|
|
|
|
auto node_it = storage.container.find(path);
|
|
|
|
if (node_it == storage.container.end())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
return storage.acl_map.convertNumber(node_it->value.acl_id);
|
|
|
|
}
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
return storage.uncommitted_state.getACLs(path);
|
2022-05-09 09:16:05 +00:00
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
}
|
2022-05-10 12:53:18 +00:00
|
|
|
bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local)
|
|
|
|
{
|
|
|
|
const auto node_acls = getNodeACLs(*this, path, is_local);
|
|
|
|
if (node_acls.empty())
|
|
|
|
return true;
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
if (uncommitted_state.hasACL(session_id, is_local, [](const auto & auth_id) { return auth_id.scheme == "super"; }))
|
2022-05-10 12:53:18 +00:00
|
|
|
return true;
|
|
|
|
|
|
|
|
for (const auto & node_acl : node_acls)
|
|
|
|
{
|
|
|
|
if (node_acl.permissions & permission)
|
|
|
|
{
|
|
|
|
if (node_acl.scheme == "world" && node_acl.id == "anyone")
|
|
|
|
return true;
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
if (uncommitted_state.hasACL(
|
2022-05-10 12:53:18 +00:00
|
|
|
session_id,
|
|
|
|
is_local,
|
|
|
|
[&](const auto & auth_id) { return auth_id.scheme == node_acl.scheme && auth_id.id == node_acl.id; }))
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2022-05-23 14:37:57 +00:00
|
|
|
void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path)
|
|
|
|
{
|
|
|
|
auto ephemerals_it = ephemerals.find(session_id);
|
|
|
|
assert(ephemerals_it != ephemerals.end());
|
|
|
|
ephemerals_it->second.erase(path);
|
|
|
|
if (ephemerals_it->second.empty())
|
|
|
|
ephemerals.erase(ephemerals_it);
|
|
|
|
}
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-10-30 19:57:30 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
KeeperStorage::ResponsesForSessions
|
|
|
|
processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
|
2020-11-10 13:43:10 +00:00
|
|
|
{
|
2021-01-19 14:22:28 +00:00
|
|
|
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED);
|
2020-11-10 13:43:10 +00:00
|
|
|
}
|
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2020-10-30 19:57:30 +00:00
|
|
|
{
|
2022-01-22 15:29:36 +00:00
|
|
|
auto path = zk_request->getPath();
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(parentPath(path), Coordination::ACL::Create, session_id, is_local);
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest) const override
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperCreateRequest & request = dynamic_cast<Coordination::ZooKeeperCreateRequest &>(*zk_request);
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
std::vector<KeeperStorage::Delta> new_deltas;
|
|
|
|
|
2022-05-05 10:32:41 +00:00
|
|
|
auto parent_path = parentPath(request.path);
|
2022-05-11 09:08:39 +00:00
|
|
|
auto parent_node = storage.uncommitted_state.getNode(parent_path);
|
2022-05-05 10:32:41 +00:00
|
|
|
if (parent_node == nullptr)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-05 10:32:41 +00:00
|
|
|
else if (parent_node->stat.ephemeralOwner != 0)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}};
|
2022-05-05 10:32:41 +00:00
|
|
|
|
|
|
|
std::string path_created = request.path;
|
|
|
|
if (request.is_sequential)
|
|
|
|
{
|
|
|
|
auto seq_num = parent_node->seq_num;
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
2022-05-05 10:32:41 +00:00
|
|
|
seq_num_str.exceptions(std::ios::failbit);
|
|
|
|
seq_num_str << std::setw(10) << std::setfill('0') << seq_num;
|
|
|
|
|
|
|
|
path_created += seq_num_str.str();
|
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (storage.uncommitted_state.getNode(path_created))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}};
|
2022-05-05 10:32:41 +00:00
|
|
|
|
|
|
|
if (getBaseName(path_created).size == 0)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
Coordination::ACLs node_acls;
|
|
|
|
if (!fixupACL(request.acls, storage.session_and_auth[session_id], node_acls))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
if (request.is_ephemeral)
|
|
|
|
storage.ephemerals[session_id].emplace(path_created);
|
|
|
|
|
2022-06-14 08:34:42 +00:00
|
|
|
int32_t parent_cversion = request.parent_cversion;
|
|
|
|
|
2022-06-15 12:48:30 +00:00
|
|
|
auto parent_update = [parent_cversion, zxid](KeeperStorage::Node & node)
|
|
|
|
{
|
|
|
|
/// Increment sequential number even if node is not sequential
|
|
|
|
++node.seq_num;
|
|
|
|
if (parent_cversion == -1)
|
|
|
|
++node.stat.cversion;
|
|
|
|
else if (parent_cversion > node.stat.cversion)
|
|
|
|
node.stat.cversion = parent_cversion;
|
|
|
|
|
|
|
|
if (zxid > node.stat.pzxid)
|
|
|
|
node.stat.pzxid = zxid;
|
|
|
|
++node.stat.numChildren;
|
|
|
|
};
|
|
|
|
|
|
|
|
new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)});
|
2022-06-14 08:34:42 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
Coordination::Stat stat;
|
|
|
|
stat.czxid = zxid;
|
|
|
|
stat.mzxid = zxid;
|
|
|
|
stat.pzxid = zxid;
|
|
|
|
stat.ctime = time;
|
|
|
|
stat.mtime = time;
|
|
|
|
stat.numChildren = 0;
|
2022-05-09 11:53:19 +00:00
|
|
|
stat.version = 0;
|
|
|
|
stat.aversion = 0;
|
2022-05-10 13:31:39 +00:00
|
|
|
stat.cversion = 0;
|
2022-05-06 12:25:25 +00:00
|
|
|
stat.dataLength = request.data.length();
|
|
|
|
stat.ephemeralOwner = request.is_ephemeral ? session_id : 0;
|
|
|
|
|
|
|
|
new_deltas.emplace_back(
|
2022-05-10 13:31:39 +00:00
|
|
|
std::move(path_created),
|
2022-05-06 12:25:25 +00:00
|
|
|
zxid,
|
2022-05-23 06:48:39 +00:00
|
|
|
KeeperStorage::CreateNodeDelta{stat, request.is_sequential, std::move(node_acls), request.data});
|
2022-05-05 10:32:41 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
digest = storage.calculateNodesDigest(digest, new_deltas);
|
2022-05-06 12:25:25 +00:00
|
|
|
return new_deltas;
|
2022-05-05 10:32:41 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
2020-10-30 19:57:30 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperCreateResponse & response = dynamic_cast<Coordination::ZooKeeperCreateResponse &>(*response_ptr);
|
2021-08-03 09:59:08 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2020-10-30 19:57:30 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
response.error = result;
|
2022-05-09 07:02:11 +00:00
|
|
|
return response_ptr;
|
2020-10-30 19:57:30 +00:00
|
|
|
}
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
const auto & deltas = storage.uncommitted_state.deltas;
|
2022-05-06 12:25:25 +00:00
|
|
|
auto create_delta_it = std::find_if(
|
|
|
|
deltas.begin(),
|
|
|
|
deltas.end(),
|
|
|
|
[zxid](const auto & delta)
|
|
|
|
{ return delta.zxid == zxid && std::holds_alternative<KeeperStorage::CreateNodeDelta>(delta.operation); });
|
2021-08-03 09:59:08 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
response.path_created = create_delta_it->path;
|
2021-08-03 09:59:08 +00:00
|
|
|
response.error = Coordination::Error::ZOK;
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2020-10-30 19:57:30 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local);
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
std::vector<KeeperStorage::Delta>
|
2022-05-19 09:45:38 +00:00
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperGetRequest & request = dynamic_cast<Coordination::ZooKeeperGetRequest &>(*zk_request);
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!storage.uncommitted_state.getNode(request.path))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
template <bool local>
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperGetResponse & response = dynamic_cast<Coordination::ZooKeeperGetResponse &>(*response_ptr);
|
|
|
|
Coordination::ZooKeeperGetRequest & request = dynamic_cast<Coordination::ZooKeeperGetRequest &>(*zk_request);
|
|
|
|
|
2022-05-10 07:00:38 +00:00
|
|
|
if constexpr (!local)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
|
|
|
response.error = result;
|
|
|
|
return response_ptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
auto & container = storage.container;
|
|
|
|
auto node_it = container.find(request.path);
|
|
|
|
if (node_it == container.end())
|
|
|
|
{
|
2022-05-17 06:45:51 +00:00
|
|
|
if constexpr (local)
|
|
|
|
response.error = Coordination::Error::ZNONODE;
|
|
|
|
else
|
|
|
|
onStorageInconsistency();
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-05-09 08:32:25 +00:00
|
|
|
response.stat = node_it->value.stat;
|
|
|
|
response.data = node_it->value.getData();
|
2020-11-03 14:49:30 +00:00
|
|
|
response.error = Coordination::Error::ZOK;
|
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
2022-05-09 08:32:25 +00:00
|
|
|
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<false>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<true>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
2020-11-03 14:49:30 +00:00
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(parentPath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-06 12:25:25 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
2022-05-19 09:45:38 +00:00
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest) const override
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperRemoveRequest & request = dynamic_cast<Coordination::ZooKeeperRemoveRequest &>(*zk_request);
|
2020-11-19 16:06:19 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
std::vector<KeeperStorage::Delta> new_deltas;
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
const auto update_parent_pzxid = [&]()
|
|
|
|
{
|
|
|
|
auto parent_path = parentPath(request.path);
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!storage.uncommitted_state.getNode(parent_path))
|
2022-05-06 12:25:25 +00:00
|
|
|
return;
|
|
|
|
|
|
|
|
new_deltas.emplace_back(
|
|
|
|
std::string{parent_path},
|
|
|
|
zxid,
|
2022-05-23 14:37:57 +00:00
|
|
|
KeeperStorage::UpdateNodeDelta
|
|
|
|
{
|
|
|
|
[zxid](KeeperStorage::Node & parent)
|
|
|
|
{
|
|
|
|
if (parent.stat.pzxid < zxid)
|
|
|
|
parent.stat.pzxid = zxid;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
);
|
2022-05-06 12:25:25 +00:00
|
|
|
};
|
2021-02-26 13:53:34 +00:00
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
auto node = storage.uncommitted_state.getNode(request.path);
|
2021-02-19 07:05:52 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
if (!node)
|
|
|
|
{
|
|
|
|
if (request.restored_from_zookeeper_log)
|
|
|
|
update_parent_pzxid();
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
}
|
|
|
|
else if (request.version != -1 && request.version != node->stat.version)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
2022-06-14 13:23:46 +00:00
|
|
|
else if (node->stat.numChildren != 0)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOTEMPTY}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
if (request.restored_from_zookeeper_log)
|
|
|
|
update_parent_pzxid();
|
|
|
|
|
|
|
|
new_deltas.emplace_back(
|
|
|
|
std::string{parentPath(request.path)},
|
|
|
|
zxid,
|
|
|
|
KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent)
|
|
|
|
{
|
|
|
|
++parent.stat.cversion;
|
2022-06-14 08:34:42 +00:00
|
|
|
--parent.stat.numChildren;
|
2022-05-06 12:25:25 +00:00
|
|
|
}});
|
|
|
|
|
2022-06-14 08:34:42 +00:00
|
|
|
new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner});
|
|
|
|
|
2022-05-23 14:37:57 +00:00
|
|
|
if (node->stat.ephemeralOwner != 0)
|
|
|
|
storage.unregisterEphemeralPath(node->stat.ephemeralOwner, request.path);
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
digest = storage.calculateNodesDigest(digest, new_deltas);
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return new_deltas;
|
|
|
|
}
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperRemoveResponse & response = dynamic_cast<Coordination::ZooKeeperRemoveResponse &>(*response_ptr);
|
2020-11-20 08:37:16 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
response.error = storage.commit(zxid);
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
KeeperStorage::ResponsesForSessions
|
|
|
|
processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
|
2020-11-10 13:43:10 +00:00
|
|
|
{
|
2021-01-19 14:22:28 +00:00
|
|
|
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED);
|
2020-11-10 13:43:10 +00:00
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
std::vector<KeeperStorage::Delta>
|
2022-05-19 09:45:38 +00:00
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperExistsRequest & request = dynamic_cast<Coordination::ZooKeeperExistsRequest &>(*zk_request);
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!storage.uncommitted_state.getNode(request.path))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
template <bool local>
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperExistsResponse & response = dynamic_cast<Coordination::ZooKeeperExistsResponse &>(*response_ptr);
|
|
|
|
Coordination::ZooKeeperExistsRequest & request = dynamic_cast<Coordination::ZooKeeperExistsRequest &>(*zk_request);
|
|
|
|
|
2022-05-10 07:00:38 +00:00
|
|
|
if constexpr (!local)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
|
|
|
response.error = result;
|
|
|
|
return response_ptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
auto & container = storage.container;
|
|
|
|
auto node_it = container.find(request.path);
|
|
|
|
if (node_it == container.end())
|
|
|
|
{
|
2022-05-17 06:45:51 +00:00
|
|
|
if constexpr (local)
|
|
|
|
response.error = Coordination::Error::ZNONODE;
|
|
|
|
else
|
|
|
|
onStorageInconsistency();
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-05-09 08:32:25 +00:00
|
|
|
response.stat = node_it->value.stat;
|
|
|
|
response.error = Coordination::Error::ZOK;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2022-05-09 08:32:25 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<false>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<true>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Write, session_id, is_local);
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-19 09:45:38 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperSetRequest & request = dynamic_cast<Coordination::ZooKeeperSetRequest &>(*zk_request);
|
|
|
|
|
|
|
|
std::vector<KeeperStorage::Delta> new_deltas;
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!storage.uncommitted_state.getNode(request.path))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
auto node = storage.uncommitted_state.getNode(request.path);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
if (request.version != -1 && request.version != node->stat.version)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
new_deltas.emplace_back(
|
|
|
|
request.path,
|
|
|
|
zxid,
|
|
|
|
KeeperStorage::UpdateNodeDelta{
|
2022-05-10 09:53:15 +00:00
|
|
|
[zxid, data = request.data, time](KeeperStorage::Node & value)
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
value.stat.version++;
|
|
|
|
value.stat.mzxid = zxid;
|
|
|
|
value.stat.mtime = time;
|
|
|
|
value.stat.dataLength = data.length();
|
2022-05-10 09:53:15 +00:00
|
|
|
value.setData(data);
|
2022-05-06 12:25:25 +00:00
|
|
|
},
|
|
|
|
request.version});
|
|
|
|
|
2022-05-20 10:12:09 +00:00
|
|
|
new_deltas.emplace_back(
|
|
|
|
parentPath(request.path).toString(),
|
|
|
|
zxid,
|
|
|
|
KeeperStorage::UpdateNodeDelta
|
|
|
|
{
|
|
|
|
[](KeeperStorage::Node & parent)
|
|
|
|
{
|
|
|
|
parent.stat.cversion++;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
digest = storage.calculateNodesDigest(digest, new_deltas);
|
2022-05-06 12:25:25 +00:00
|
|
|
return new_deltas;
|
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2021-05-21 21:19:22 +00:00
|
|
|
auto & container = storage.container;
|
|
|
|
|
2020-11-04 18:54:55 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperSetResponse & response = dynamic_cast<Coordination::ZooKeeperSetResponse &>(*response_ptr);
|
|
|
|
Coordination::ZooKeeperSetRequest & request = dynamic_cast<Coordination::ZooKeeperSetRequest &>(*zk_request);
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
response.error = result;
|
2022-05-09 07:02:11 +00:00
|
|
|
return response_ptr;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2021-02-26 13:53:34 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
auto node_it = container.find(request.path);
|
|
|
|
if (node_it == container.end())
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2020-11-04 18:54:55 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
response.stat = node_it->value.stat;
|
2022-05-06 12:25:25 +00:00
|
|
|
response.error = Coordination::Error::ZOK;
|
2020-11-04 18:54:55 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
KeeperStorage::ResponsesForSessions
|
|
|
|
processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
|
2020-11-10 13:43:10 +00:00
|
|
|
{
|
2021-01-19 14:22:28 +00:00
|
|
|
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED);
|
2020-11-10 13:43:10 +00:00
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local);
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-06 12:25:25 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
2022-05-19 09:45:38 +00:00
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperListRequest & request = dynamic_cast<Coordination::ZooKeeperListRequest &>(*zk_request);
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!storage.uncommitted_state.getNode(request.path))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
|
|
|
|
template <bool local>
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperListResponse & response = dynamic_cast<Coordination::ZooKeeperListResponse &>(*response_ptr);
|
|
|
|
Coordination::ZooKeeperListRequest & request = dynamic_cast<Coordination::ZooKeeperListRequest &>(*zk_request);
|
2022-01-22 19:36:23 +00:00
|
|
|
|
2022-05-10 07:00:38 +00:00
|
|
|
if constexpr (!local)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
|
|
|
response.error = result;
|
|
|
|
return response_ptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
auto & container = storage.container;
|
2022-06-21 12:35:58 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
auto node_it = container.find(request.path);
|
|
|
|
if (node_it == container.end())
|
|
|
|
{
|
2022-05-17 06:45:51 +00:00
|
|
|
if constexpr (local)
|
|
|
|
response.error = Coordination::Error::ZNONODE;
|
|
|
|
else
|
|
|
|
onStorageInconsistency();
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto path_prefix = request.path;
|
|
|
|
if (path_prefix.empty())
|
2020-11-25 13:19:09 +00:00
|
|
|
throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR);
|
2020-11-04 18:54:55 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
const auto & children = node_it->value.getChildren();
|
2022-04-05 06:27:03 +00:00
|
|
|
response.names.reserve(children.size());
|
2022-01-19 11:46:29 +00:00
|
|
|
|
2022-06-21 12:35:58 +00:00
|
|
|
const auto add_child = [&](const auto child)
|
|
|
|
{
|
|
|
|
using enum Coordination::ListRequestType;
|
|
|
|
|
2022-06-29 08:30:39 +00:00
|
|
|
auto list_request_type = ALL;
|
|
|
|
if (auto * filtered_list = dynamic_cast<Coordination::ZooKeeperFilteredListRequest *>(&request))
|
2022-07-07 11:35:02 +00:00
|
|
|
{
|
2022-06-29 08:30:39 +00:00
|
|
|
list_request_type = filtered_list->list_request_type;
|
2022-07-07 11:35:02 +00:00
|
|
|
}
|
2022-06-29 08:30:39 +00:00
|
|
|
|
|
|
|
if (list_request_type == ALL)
|
2022-06-21 12:35:58 +00:00
|
|
|
return true;
|
|
|
|
|
2022-06-27 06:53:09 +00:00
|
|
|
auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string();
|
|
|
|
auto child_it = container.find(child_path);
|
2022-06-21 12:35:58 +00:00
|
|
|
if (child_it == container.end())
|
|
|
|
onStorageInconsistency();
|
|
|
|
|
|
|
|
const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0;
|
2022-06-29 08:30:39 +00:00
|
|
|
return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY);
|
2022-06-21 12:35:58 +00:00
|
|
|
};
|
|
|
|
|
2022-04-05 06:27:03 +00:00
|
|
|
for (const auto child : children)
|
2022-06-21 12:35:58 +00:00
|
|
|
{
|
|
|
|
if (add_child(child))
|
|
|
|
response.names.push_back(child.toString());
|
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
response.stat = node_it->value.stat;
|
2020-11-04 18:54:55 +00:00
|
|
|
response.error = Coordination::Error::ZOK;
|
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2022-05-09 08:32:25 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<false>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<true>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local);
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-06 12:25:25 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
2022-05-19 09:45:38 +00:00
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperCheckRequest & request = dynamic_cast<Coordination::ZooKeeperCheckRequest &>(*zk_request);
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!storage.uncommitted_state.getNode(request.path))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
auto node = storage.uncommitted_state.getNode(request.path);
|
2022-05-06 12:25:25 +00:00
|
|
|
if (request.version != -1 && request.version != node->stat.version)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
template <bool local>
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperCheckResponse & response = dynamic_cast<Coordination::ZooKeeperCheckResponse &>(*response_ptr);
|
|
|
|
Coordination::ZooKeeperCheckRequest & request = dynamic_cast<Coordination::ZooKeeperCheckRequest &>(*zk_request);
|
2022-05-09 08:32:25 +00:00
|
|
|
|
2022-05-10 07:00:38 +00:00
|
|
|
if constexpr (!local)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
|
|
|
response.error = result;
|
|
|
|
return response_ptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
const auto on_error = [&]([[maybe_unused]] const auto error_code)
|
|
|
|
{
|
|
|
|
if constexpr (local)
|
|
|
|
response.error = error_code;
|
|
|
|
else
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-09 08:32:25 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
auto & container = storage.container;
|
|
|
|
auto node_it = container.find(request.path);
|
|
|
|
if (node_it == container.end())
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 08:32:25 +00:00
|
|
|
on_error(Coordination::Error::ZNONODE);
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2022-05-09 08:32:25 +00:00
|
|
|
else if (request.version != -1 && request.version != node_it->value.stat.version)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 08:32:25 +00:00
|
|
|
on_error(Coordination::Error::ZBADVERSION);
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
response.error = Coordination::Error::ZOK;
|
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2022-05-09 08:32:25 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<false>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<true>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
};
|
|
|
|
|
2021-06-22 10:49:35 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local);
|
2021-06-22 10:49:35 +00:00
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2021-06-22 10:49:35 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest) const override
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
Coordination::ZooKeeperSetACLRequest & request = dynamic_cast<Coordination::ZooKeeperSetACLRequest &>(*zk_request);
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
auto & uncommitted_state = storage.uncommitted_state;
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!uncommitted_state.getNode(request.path))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
auto node = uncommitted_state.getNode(request.path);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
if (request.version != -1 && request.version != node->stat.aversion)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
|
|
|
|
auto & session_auth_ids = storage.session_and_auth[session_id];
|
|
|
|
Coordination::ACLs node_acls;
|
|
|
|
|
|
|
|
if (!fixupACL(request.acls, session_auth_ids, node_acls))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
std::vector<KeeperStorage::Delta> new_deltas
|
2022-05-24 13:11:11 +00:00
|
|
|
{
|
|
|
|
{
|
|
|
|
request.path,
|
|
|
|
zxid,
|
|
|
|
KeeperStorage::SetACLDelta{std::move(node_acls), request.version}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
request.path,
|
|
|
|
zxid,
|
|
|
|
KeeperStorage::UpdateNodeDelta
|
|
|
|
{
|
|
|
|
[](KeeperStorage::Node & n) { ++n.stat.aversion; }
|
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
2022-05-19 09:45:38 +00:00
|
|
|
|
|
|
|
digest = storage.calculateNodesDigest(digest, new_deltas);
|
|
|
|
|
|
|
|
return new_deltas;
|
2022-05-06 12:25:25 +00:00
|
|
|
}
|
2021-06-22 10:49:35 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2021-06-22 10:49:35 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperSetACLResponse & response = dynamic_cast<Coordination::ZooKeeperSetACLResponse &>(*response_ptr);
|
|
|
|
Coordination::ZooKeeperSetACLRequest & request = dynamic_cast<Coordination::ZooKeeperSetACLRequest &>(*zk_request);
|
2022-05-06 12:25:25 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
response.error = result;
|
2022-05-09 07:02:11 +00:00
|
|
|
return response_ptr;
|
2021-06-22 10:49:35 +00:00
|
|
|
}
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
auto node_it = storage.container.find(request.path);
|
|
|
|
if (node_it == storage.container.end())
|
2022-05-11 12:10:17 +00:00
|
|
|
onStorageInconsistency();
|
2022-05-09 08:32:25 +00:00
|
|
|
response.stat = node_it->value.stat;
|
2022-05-06 12:25:25 +00:00
|
|
|
response.error = Coordination::Error::ZOK;
|
2021-06-22 10:49:35 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2021-06-22 10:49:35 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local);
|
2021-06-22 10:49:35 +00:00
|
|
|
}
|
2022-05-10 12:53:18 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2021-06-22 10:49:35 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
2022-05-19 09:45:38 +00:00
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperGetACLRequest & request = dynamic_cast<Coordination::ZooKeeperGetACLRequest &>(*zk_request);
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!storage.uncommitted_state.getNode(request.path))
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
2022-05-06 12:25:25 +00:00
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
template <bool local>
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperGetACLResponse & response = dynamic_cast<Coordination::ZooKeeperGetACLResponse &>(*response_ptr);
|
|
|
|
Coordination::ZooKeeperGetACLRequest & request = dynamic_cast<Coordination::ZooKeeperGetACLRequest &>(*zk_request);
|
2022-05-09 08:32:25 +00:00
|
|
|
|
2022-05-10 07:00:38 +00:00
|
|
|
if constexpr (!local)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
|
|
|
response.error = result;
|
|
|
|
return response_ptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-22 10:49:35 +00:00
|
|
|
auto & container = storage.container;
|
2022-05-09 08:32:25 +00:00
|
|
|
auto node_it = container.find(request.path);
|
|
|
|
if (node_it == container.end())
|
2021-06-22 10:49:35 +00:00
|
|
|
{
|
2022-05-17 06:45:51 +00:00
|
|
|
if constexpr (local)
|
|
|
|
response.error = Coordination::Error::ZNONODE;
|
|
|
|
else
|
|
|
|
onStorageInconsistency();
|
2021-06-22 10:49:35 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-05-09 08:32:25 +00:00
|
|
|
response.stat = node_it->value.stat;
|
|
|
|
response.acl = storage.acl_map.convertNumber(node_it->value.acl_id);
|
2021-06-22 10:49:35 +00:00
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
return response_ptr;
|
2021-06-22 10:49:35 +00:00
|
|
|
}
|
2022-05-09 08:32:25 +00:00
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<false>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
return processImpl<true>(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
2021-06-22 10:49:35 +00:00
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 09:16:05 +00:00
|
|
|
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
|
|
|
for (const auto & concrete_request : concrete_requests)
|
2022-05-09 09:16:05 +00:00
|
|
|
if (!concrete_request->checkAuth(storage, session_id, is_local))
|
2021-05-21 21:19:22 +00:00
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
std::vector<KeeperStorageRequestProcessorPtr> concrete_requests;
|
|
|
|
explicit KeeperStorageMultiRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_)
|
|
|
|
: KeeperStorageRequestProcessor(zk_request_)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperMultiRequest & request = dynamic_cast<Coordination::ZooKeeperMultiRequest &>(*zk_request);
|
|
|
|
concrete_requests.reserve(request.requests.size());
|
|
|
|
|
2020-11-25 13:19:09 +00:00
|
|
|
for (const auto & sub_request : request.requests)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2020-12-10 21:33:13 +00:00
|
|
|
auto sub_zk_request = std::dynamic_pointer_cast<Coordination::ZooKeeperRequest>(sub_request);
|
2022-04-05 06:27:03 +00:00
|
|
|
switch (sub_zk_request->getOpNum())
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-04-05 06:27:03 +00:00
|
|
|
case Coordination::OpNum::Create:
|
|
|
|
concrete_requests.push_back(std::make_shared<KeeperStorageCreateRequestProcessor>(sub_zk_request));
|
|
|
|
break;
|
|
|
|
case Coordination::OpNum::Remove:
|
|
|
|
concrete_requests.push_back(std::make_shared<KeeperStorageRemoveRequestProcessor>(sub_zk_request));
|
|
|
|
break;
|
|
|
|
case Coordination::OpNum::Set:
|
|
|
|
concrete_requests.push_back(std::make_shared<KeeperStorageSetRequestProcessor>(sub_zk_request));
|
|
|
|
break;
|
|
|
|
case Coordination::OpNum::Check:
|
|
|
|
concrete_requests.push_back(std::make_shared<KeeperStorageCheckRequestProcessor>(sub_zk_request));
|
|
|
|
break;
|
|
|
|
default:
|
2022-05-06 12:25:25 +00:00
|
|
|
throw DB::Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", sub_zk_request->getOpNum());
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest) const override
|
2022-05-06 12:25:25 +00:00
|
|
|
{
|
2022-05-09 07:02:11 +00:00
|
|
|
std::vector<Coordination::Error> response_errors;
|
|
|
|
response_errors.reserve(concrete_requests.size());
|
2022-05-19 09:45:38 +00:00
|
|
|
uint64_t current_digest = digest;
|
2022-05-09 07:02:11 +00:00
|
|
|
for (size_t i = 0; i < concrete_requests.size(); ++i)
|
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
auto new_deltas = concrete_requests[i]->preprocess(storage, zxid, session_id, time, current_digest);
|
2022-05-09 11:53:19 +00:00
|
|
|
|
|
|
|
if (!new_deltas.empty())
|
2022-05-09 07:02:11 +00:00
|
|
|
{
|
2022-05-09 11:53:19 +00:00
|
|
|
if (auto * error = std::get_if<KeeperStorage::ErrorDelta>(&new_deltas.back().operation))
|
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
storage.uncommitted_state.rollback(zxid);
|
2022-05-09 11:53:19 +00:00
|
|
|
response_errors.push_back(error->error);
|
2022-05-09 07:02:11 +00:00
|
|
|
|
2022-05-09 11:53:19 +00:00
|
|
|
for (size_t j = i + 1; j < concrete_requests.size(); ++j)
|
|
|
|
{
|
|
|
|
response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY);
|
|
|
|
}
|
2022-05-09 07:02:11 +00:00
|
|
|
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors)}}};
|
2022-05-09 11:53:19 +00:00
|
|
|
}
|
2022-05-09 07:02:11 +00:00
|
|
|
}
|
|
|
|
new_deltas.emplace_back(zxid, KeeperStorage::SubDeltaEnd{});
|
|
|
|
response_errors.push_back(Coordination::Error::ZOK);
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
// manually add deltas so that the result of previous request in the transaction is used in the next request
|
|
|
|
storage.uncommitted_state.addDeltas(std::move(new_deltas));
|
2022-05-09 07:02:11 +00:00
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
digest = current_digest;
|
|
|
|
|
2022-05-09 07:02:11 +00:00
|
|
|
return {};
|
2022-05-06 12:25:25 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperMultiResponse & response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response_ptr);
|
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
auto & deltas = storage.uncommitted_state.deltas;
|
2022-05-17 10:15:12 +00:00
|
|
|
// the deltas will have at least SubDeltaEnd or FailedMultiDelta
|
2022-05-17 06:45:51 +00:00
|
|
|
assert(!deltas.empty());
|
2022-05-09 07:02:11 +00:00
|
|
|
if (auto * failed_multi = std::get_if<KeeperStorage::FailedMultiDelta>(&deltas.front().operation))
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 07:02:11 +00:00
|
|
|
for (size_t i = 0; i < concrete_requests.size(); ++i)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 07:02:11 +00:00
|
|
|
response.responses[i] = std::make_shared<Coordination::ZooKeeperErrorResponse>();
|
|
|
|
response.responses[i]->error = failed_multi->error_codes[i];
|
|
|
|
}
|
2020-12-08 13:28:39 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
storage.uncommitted_state.commit(zxid);
|
2022-05-09 07:02:11 +00:00
|
|
|
return response_ptr;
|
|
|
|
}
|
2020-12-08 13:28:39 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
for (size_t i = 0; i < concrete_requests.size(); ++i)
|
2022-05-09 07:02:11 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
auto cur_response = concrete_requests[i]->process(storage, zxid);
|
2022-05-09 07:02:11 +00:00
|
|
|
response.responses[i] = cur_response;
|
2022-05-19 09:45:38 +00:00
|
|
|
storage.uncommitted_state.commit(zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
response.error = Coordination::Error::ZOK;
|
|
|
|
return response_ptr;
|
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-09 08:32:25 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperMultiResponse & response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response_ptr);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < concrete_requests.size(); ++i)
|
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
auto cur_response = concrete_requests[i]->processLocal(storage, zxid);
|
2022-05-09 08:32:25 +00:00
|
|
|
|
|
|
|
response.responses[i] = cur_response;
|
|
|
|
if (cur_response->error != Coordination::Error::ZOK)
|
|
|
|
{
|
|
|
|
for (size_t j = 0; j <= i; ++j)
|
|
|
|
{
|
|
|
|
auto response_error = response.responses[j]->error;
|
|
|
|
response.responses[j] = std::make_shared<Coordination::ZooKeeperErrorResponse>();
|
|
|
|
response.responses[j]->error = response_error;
|
|
|
|
}
|
|
|
|
|
|
|
|
for (size_t j = i + 1; j < response.responses.size(); ++j)
|
|
|
|
{
|
|
|
|
response.responses[j] = std::make_shared<Coordination::ZooKeeperErrorResponse>();
|
|
|
|
response.responses[j]->error = Coordination::Error::ZRUNTIMEINCONSISTENCY;
|
|
|
|
}
|
2022-05-09 07:02:11 +00:00
|
|
|
|
2022-05-09 08:32:25 +00:00
|
|
|
return response_ptr;
|
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2022-05-09 07:02:11 +00:00
|
|
|
|
|
|
|
response.error = Coordination::Error::ZOK;
|
|
|
|
return response_ptr;
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
KeeperStorage::ResponsesForSessions
|
|
|
|
processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
|
2020-11-10 13:43:10 +00:00
|
|
|
{
|
2021-03-29 08:24:56 +00:00
|
|
|
KeeperStorage::ResponsesForSessions result;
|
2020-11-10 13:43:10 +00:00
|
|
|
for (const auto & generic_request : concrete_requests)
|
2021-01-19 14:22:28 +00:00
|
|
|
{
|
|
|
|
auto responses = generic_request->processWatches(watches, list_watches);
|
|
|
|
result.insert(result.end(), responses.begin(), responses.end());
|
|
|
|
}
|
|
|
|
return result;
|
2020-11-10 13:43:10 +00:00
|
|
|
}
|
2020-11-04 18:54:55 +00:00
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor
|
2020-11-19 16:06:19 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage &, int64_t) const override
|
2020-11-19 16:06:19 +00:00
|
|
|
{
|
2020-11-25 13:19:09 +00:00
|
|
|
throw DB::Exception("Called process on close request", ErrorCodes::LOGICAL_ERROR);
|
2020-11-19 16:06:19 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor
|
2021-05-15 15:01:00 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
2022-05-19 09:45:38 +00:00
|
|
|
std::vector<KeeperStorage::Delta>
|
|
|
|
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/) const override
|
2021-05-15 15:01:00 +00:00
|
|
|
{
|
2021-05-21 21:19:22 +00:00
|
|
|
Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast<Coordination::ZooKeeperAuthRequest &>(*zk_request);
|
2021-05-15 15:01:00 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
2021-05-21 21:19:22 +00:00
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1)
|
2022-06-15 12:48:30 +00:00
|
|
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}};
|
2022-05-10 12:53:18 +00:00
|
|
|
|
|
|
|
std::vector<KeeperStorage::Delta> new_deltas;
|
2022-05-19 09:45:38 +00:00
|
|
|
auto auth_digest = generateDigest(auth_request.data);
|
|
|
|
if (auth_digest == storage.superdigest)
|
2021-05-24 12:18:04 +00:00
|
|
|
{
|
2022-05-10 12:53:18 +00:00
|
|
|
KeeperStorage::AuthID auth{"super", ""};
|
|
|
|
new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(auth)});
|
2021-05-24 12:18:04 +00:00
|
|
|
}
|
|
|
|
else
|
2021-05-22 16:07:47 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
KeeperStorage::AuthID new_auth{auth_request.scheme, auth_digest};
|
2022-05-11 09:08:39 +00:00
|
|
|
if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return new_auth == auth_id; }))
|
2022-05-10 12:53:18 +00:00
|
|
|
new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(new_auth)});
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
|
2022-05-10 12:53:18 +00:00
|
|
|
return new_deltas;
|
|
|
|
}
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override
|
2022-05-10 12:53:18 +00:00
|
|
|
{
|
|
|
|
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
|
|
|
|
Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast<Coordination::ZooKeeperAuthResponse &>(*response_ptr);
|
|
|
|
|
2022-05-23 07:51:26 +00:00
|
|
|
if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK)
|
2022-05-10 12:53:18 +00:00
|
|
|
auth_response.error = result;
|
|
|
|
|
2022-05-09 07:02:11 +00:00
|
|
|
return response_ptr;
|
2021-05-15 15:01:00 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-03-29 08:24:56 +00:00
|
|
|
void KeeperStorage::finalize()
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2021-01-19 14:22:28 +00:00
|
|
|
if (finalized)
|
2022-06-23 07:53:07 +00:00
|
|
|
throw DB::Exception("KeeperStorage already finalized", ErrorCodes::LOGICAL_ERROR);
|
2020-11-10 13:43:10 +00:00
|
|
|
|
2021-01-19 14:22:28 +00:00
|
|
|
finalized = true;
|
2020-11-03 14:49:30 +00:00
|
|
|
|
2021-02-04 12:07:41 +00:00
|
|
|
ephemerals.clear();
|
2020-11-18 20:36:25 +00:00
|
|
|
|
2021-01-19 14:22:28 +00:00
|
|
|
watches.clear();
|
|
|
|
list_watches.clear();
|
|
|
|
sessions_and_watchers.clear();
|
2021-02-04 12:07:41 +00:00
|
|
|
session_expiry_queue.clear();
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-08-24 12:30:31 +00:00
|
|
|
using Creator = std::function<KeeperStorageRequestProcessorPtr(const Coordination::ZooKeeperRequestPtr &)>;
|
2020-11-11 13:07:06 +00:00
|
|
|
using OpNumToRequest = std::unordered_map<Coordination::OpNum, Creator>;
|
2020-11-04 18:54:55 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
static KeeperStorageRequestProcessorsFactory & instance()
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
static KeeperStorageRequestProcessorsFactory factory;
|
2020-11-04 18:54:55 +00:00
|
|
|
return factory;
|
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
KeeperStorageRequestProcessorPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-09 09:45:03 +00:00
|
|
|
auto request_it = op_num_to_request.find(zk_request->getOpNum());
|
|
|
|
if (request_it == op_num_to_request.end())
|
2020-11-25 13:19:09 +00:00
|
|
|
throw DB::Exception("Unknown operation type " + toString(zk_request->getOpNum()), ErrorCodes::LOGICAL_ERROR);
|
2020-11-04 18:54:55 +00:00
|
|
|
|
2022-05-09 09:45:03 +00:00
|
|
|
return request_it->second(zk_request);
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
2020-11-03 14:49:30 +00:00
|
|
|
|
2020-11-11 13:07:06 +00:00
|
|
|
void registerRequest(Coordination::OpNum op_num, Creator creator)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
|
|
|
if (!op_num_to_request.try_emplace(op_num, creator).second)
|
|
|
|
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Request with op num {} already registered", op_num);
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
OpNumToRequest op_num_to_request;
|
2021-08-24 12:30:31 +00:00
|
|
|
KeeperStorageRequestProcessorsFactory();
|
2020-11-04 18:54:55 +00:00
|
|
|
};
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
template <Coordination::OpNum num, typename RequestT>
|
2021-08-24 12:30:31 +00:00
|
|
|
void registerKeeperRequestProcessor(KeeperStorageRequestProcessorsFactory & factory)
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2022-05-06 12:25:25 +00:00
|
|
|
factory.registerRequest(
|
|
|
|
num, [](const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared<RequestT>(zk_request); });
|
2020-11-04 18:54:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory()
|
2020-11-04 18:54:55 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Heartbeat, KeeperStorageHeartbeatRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Sync, KeeperStorageSyncRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Auth, KeeperStorageAuthRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Close, KeeperStorageCloseRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Create, KeeperStorageCreateRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Remove, KeeperStorageRemoveRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Exists, KeeperStorageExistsRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Get, KeeperStorageGetRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Set, KeeperStorageSetRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::List, KeeperStorageListRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::SimpleList, KeeperStorageListRequestProcessor>(*this);
|
2022-06-29 08:30:39 +00:00
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::FilteredList, KeeperStorageListRequestProcessor>(*this);
|
2021-08-24 12:30:31 +00:00
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Check, KeeperStorageCheckRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::Multi, KeeperStorageMultiRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::SetACL, KeeperStorageSetACLRequestProcessor>(*this);
|
|
|
|
registerKeeperRequestProcessor<Coordination::OpNum::GetACL, KeeperStorageGetACLRequestProcessor>(*this);
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector<Delta> & new_deltas) const
|
2022-05-13 13:43:42 +00:00
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
if (!digest_enabled)
|
|
|
|
return current_digest;
|
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
std::unordered_map<std::string, std::shared_ptr<Node>> updated_nodes;
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
for (const auto & delta : new_deltas)
|
2022-05-13 13:43:42 +00:00
|
|
|
{
|
|
|
|
std::visit(
|
|
|
|
Overloaded{
|
|
|
|
[&](const CreateNodeDelta & create_delta)
|
|
|
|
{
|
|
|
|
auto node = std::make_shared<Node>();
|
|
|
|
node->stat = create_delta.stat;
|
|
|
|
node->setData(create_delta.data);
|
|
|
|
updated_nodes.emplace(delta.path, node);
|
|
|
|
},
|
|
|
|
[&](const RemoveNodeDelta & /* remove_delta */)
|
|
|
|
{
|
|
|
|
if (!updated_nodes.contains(delta.path))
|
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
auto old_digest = uncommitted_state.getNode(delta.path)->getDigest(delta.path);
|
2022-05-16 12:12:29 +00:00
|
|
|
current_digest -= old_digest;
|
2022-05-13 13:43:42 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
updated_nodes.insert_or_assign(delta.path, nullptr);
|
|
|
|
},
|
|
|
|
[&](const UpdateNodeDelta & update_delta)
|
|
|
|
{
|
|
|
|
std::shared_ptr<Node> node{nullptr};
|
|
|
|
|
|
|
|
auto updated_node_it = updated_nodes.find(delta.path);
|
|
|
|
if (updated_node_it == updated_nodes.end())
|
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
node = std::make_shared<KeeperStorage::Node>();
|
|
|
|
node->shallowCopy(*uncommitted_state.getNode(delta.path));
|
2022-05-16 12:12:29 +00:00
|
|
|
current_digest -= node->getDigest(delta.path);
|
2022-05-13 13:43:42 +00:00
|
|
|
updated_nodes.emplace(delta.path, node);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
node = updated_node_it->second;
|
|
|
|
|
|
|
|
update_delta.update_fn(*node);
|
|
|
|
},
|
|
|
|
[](auto && /* delta */) {}},
|
|
|
|
delta.operation);
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & [path, updated_node] : updated_nodes)
|
|
|
|
{
|
|
|
|
if (updated_node)
|
|
|
|
{
|
|
|
|
updated_node->invalidateDigestCache();
|
2022-05-16 12:12:29 +00:00
|
|
|
current_digest += updated_node->getDigest(path);
|
2022-05-13 13:43:42 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
return current_digest;
|
2022-05-13 13:43:42 +00:00
|
|
|
}
|
|
|
|
|
2022-05-09 07:02:11 +00:00
|
|
|
void KeeperStorage::preprocessRequest(
|
2022-05-13 13:43:42 +00:00
|
|
|
const Coordination::ZooKeeperRequestPtr & zk_request,
|
|
|
|
int64_t session_id,
|
|
|
|
int64_t time,
|
|
|
|
int64_t new_last_zxid,
|
2022-05-16 13:08:10 +00:00
|
|
|
bool check_acl,
|
2022-05-17 08:11:08 +00:00
|
|
|
std::optional<Digest> digest)
|
2022-05-05 10:32:41 +00:00
|
|
|
{
|
2022-05-13 13:43:42 +00:00
|
|
|
int64_t last_zxid = getNextZXID() - 1;
|
2022-05-12 08:58:36 +00:00
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
if (uncommitted_transactions.empty())
|
2022-05-13 13:43:42 +00:00
|
|
|
{
|
2022-06-15 13:46:27 +00:00
|
|
|
// if we have no uncommitted transactions it means the last zxid is possibly loaded from snapshot
|
|
|
|
if (last_zxid != old_snapshot_zxid && new_last_zxid <= last_zxid)
|
2022-05-16 12:12:29 +00:00
|
|
|
throw Exception(
|
2022-06-15 13:46:27 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR, "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", new_last_zxid, last_zxid);
|
2022-05-16 12:12:29 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-05-19 09:45:38 +00:00
|
|
|
if (last_zxid == new_last_zxid && digest && checkDigest(*digest, getNodesDigest(false)))
|
|
|
|
// we found the preprocessed request with the same ZXID, we can skip it
|
|
|
|
return;
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
if (new_last_zxid <= last_zxid)
|
2022-05-16 12:12:29 +00:00
|
|
|
throw Exception(
|
2022-06-15 13:46:27 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR, "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", new_last_zxid, last_zxid);
|
2022-05-13 13:43:42 +00:00
|
|
|
}
|
2022-05-12 08:58:36 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
std::vector<Delta> new_deltas;
|
2022-05-13 13:43:42 +00:00
|
|
|
TransactionInfo transaction{.zxid = new_last_zxid};
|
2022-05-19 09:45:38 +00:00
|
|
|
uint64_t new_digest = getNodesDigest(false).value;
|
2022-05-13 13:43:42 +00:00
|
|
|
SCOPE_EXIT({
|
2022-05-17 08:11:08 +00:00
|
|
|
if (digest_enabled)
|
2022-05-17 13:53:12 +00:00
|
|
|
// if the version of digest we got from the leader is the same as the one this instances has, we can simply copy the value
|
|
|
|
// and just check the digest on the commit
|
|
|
|
// a mistake can happen while applying the changes to the uncommitted_state so for now let's just recalculate the digest here also
|
2022-05-19 09:45:38 +00:00
|
|
|
transaction.nodes_digest = Digest{CURRENT_DIGEST_VERSION, new_digest};
|
2022-05-16 12:12:29 +00:00
|
|
|
else
|
2022-05-17 08:11:08 +00:00
|
|
|
transaction.nodes_digest = Digest{DigestVersion::NO_DIGEST};
|
2022-05-16 12:12:29 +00:00
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
uncommitted_transactions.emplace_back(transaction);
|
2022-05-19 09:45:38 +00:00
|
|
|
uncommitted_state.addDeltas(std::move(new_deltas));
|
2022-05-13 13:43:42 +00:00
|
|
|
});
|
2022-05-12 08:58:36 +00:00
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request);
|
|
|
|
|
2022-05-09 09:35:16 +00:00
|
|
|
if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special
|
|
|
|
{
|
|
|
|
auto session_ephemerals = ephemerals.find(session_id);
|
|
|
|
if (session_ephemerals != ephemerals.end())
|
|
|
|
{
|
|
|
|
for (const auto & ephemeral_path : session_ephemerals->second)
|
|
|
|
{
|
2022-05-23 14:37:57 +00:00
|
|
|
new_deltas.emplace_back
|
|
|
|
(
|
|
|
|
parentPath(ephemeral_path).toString(),
|
|
|
|
new_last_zxid,
|
|
|
|
UpdateNodeDelta
|
|
|
|
{
|
|
|
|
[ephemeral_path](Node & parent)
|
|
|
|
{
|
|
|
|
++parent.stat.cversion;
|
2022-06-14 08:34:42 +00:00
|
|
|
--parent.stat.numChildren;
|
2022-05-23 14:37:57 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
);
|
2022-06-14 08:34:42 +00:00
|
|
|
|
|
|
|
new_deltas.emplace_back(ephemeral_path, transaction.zxid, RemoveNodeDelta{.ephemeral_owner = session_id});
|
2022-05-09 09:35:16 +00:00
|
|
|
}
|
2022-05-23 14:37:57 +00:00
|
|
|
|
|
|
|
ephemerals.erase(session_ephemerals);
|
2022-05-09 09:35:16 +00:00
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
new_digest = calculateNodesDigest(new_digest, new_deltas);
|
2022-05-09 09:35:16 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
if (check_acl && !request_processor->checkAuth(*this, session_id, false))
|
|
|
|
{
|
2022-05-11 09:08:39 +00:00
|
|
|
uncommitted_state.deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH);
|
2022-05-09 09:16:05 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
new_deltas = request_processor->preprocess(*this, transaction.zxid, session_id, time, new_digest);
|
2022-05-05 10:32:41 +00:00
|
|
|
}
|
|
|
|
|
2022-05-06 12:25:25 +00:00
|
|
|
KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(
|
|
|
|
const Coordination::ZooKeeperRequestPtr & zk_request,
|
|
|
|
int64_t session_id,
|
|
|
|
std::optional<int64_t> new_last_zxid,
|
2022-05-09 08:32:25 +00:00
|
|
|
bool check_acl,
|
|
|
|
bool is_local)
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2021-02-25 08:34:05 +00:00
|
|
|
if (new_last_zxid)
|
|
|
|
{
|
2022-05-13 13:43:42 +00:00
|
|
|
if (uncommitted_transactions.empty())
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to commit a ZXID ({}) which was not preprocessed", *new_last_zxid);
|
2022-05-12 08:58:36 +00:00
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
if (uncommitted_transactions.front().zxid != *new_last_zxid)
|
2022-05-12 08:58:36 +00:00
|
|
|
throw Exception(
|
2022-05-13 13:43:42 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR,
|
|
|
|
"Trying to commit a ZXID {} while the next ZXID to commit is {}",
|
|
|
|
*new_last_zxid,
|
|
|
|
uncommitted_transactions.front().zxid);
|
2022-05-12 08:58:36 +00:00
|
|
|
|
2021-02-25 08:34:05 +00:00
|
|
|
zxid = *new_last_zxid;
|
2022-05-13 13:43:42 +00:00
|
|
|
uncommitted_transactions.pop_front();
|
2021-02-25 08:34:05 +00:00
|
|
|
}
|
|
|
|
|
2022-05-12 08:58:36 +00:00
|
|
|
KeeperStorage::ResponsesForSessions results;
|
|
|
|
|
2021-09-02 11:54:32 +00:00
|
|
|
/// ZooKeeper update sessions expirity for each request, not only for heartbeats
|
2021-09-02 11:43:34 +00:00
|
|
|
session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]);
|
2021-08-24 12:30:31 +00:00
|
|
|
|
|
|
|
if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
commit(zxid);
|
2021-03-16 12:36:54 +00:00
|
|
|
|
2022-05-11 09:08:39 +00:00
|
|
|
for (const auto & delta : uncommitted_state.deltas)
|
2022-05-09 09:35:16 +00:00
|
|
|
{
|
|
|
|
if (delta.zxid > zxid)
|
|
|
|
break;
|
2022-01-22 19:36:23 +00:00
|
|
|
|
2022-05-09 09:35:16 +00:00
|
|
|
if (std::holds_alternative<RemoveNodeDelta>(delta.operation))
|
|
|
|
{
|
|
|
|
auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED);
|
2021-01-19 14:22:28 +00:00
|
|
|
results.insert(results.end(), responses.begin(), responses.end());
|
|
|
|
}
|
|
|
|
}
|
2022-05-09 09:35:16 +00:00
|
|
|
|
2022-05-19 09:45:38 +00:00
|
|
|
uncommitted_state.commit(zxid);
|
2022-05-09 09:35:16 +00:00
|
|
|
|
2021-01-19 14:22:28 +00:00
|
|
|
clearDeadWatches(session_id);
|
2021-05-27 08:15:46 +00:00
|
|
|
auto auth_it = session_and_auth.find(session_id);
|
|
|
|
if (auth_it != session_and_auth.end())
|
|
|
|
session_and_auth.erase(auth_it);
|
2021-01-19 14:22:28 +00:00
|
|
|
|
|
|
|
/// Finish connection
|
|
|
|
auto response = std::make_shared<Coordination::ZooKeeperCloseResponse>();
|
|
|
|
response->xid = zk_request->xid;
|
|
|
|
response->zxid = getZXID();
|
2021-02-03 20:32:15 +00:00
|
|
|
session_expiry_queue.remove(session_id);
|
|
|
|
session_and_timeout.erase(session_id);
|
|
|
|
results.push_back(ResponseForSession{session_id, response});
|
|
|
|
}
|
2021-08-24 12:30:31 +00:00
|
|
|
else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special
|
2021-02-03 20:32:15 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
KeeperStorageRequestProcessorPtr storage_request = KeeperStorageRequestProcessorsFactory::instance().get(zk_request);
|
2022-05-23 07:51:26 +00:00
|
|
|
auto response = storage_request->process(*this, zxid);
|
2021-02-03 20:32:15 +00:00
|
|
|
response->xid = zk_request->xid;
|
|
|
|
response->zxid = getZXID();
|
|
|
|
|
2021-01-19 14:45:45 +00:00
|
|
|
results.push_back(ResponseForSession{session_id, response});
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
2021-08-24 12:30:31 +00:00
|
|
|
else /// normal requests proccession
|
2020-11-03 14:49:30 +00:00
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request);
|
2021-05-21 21:19:22 +00:00
|
|
|
Coordination::ZooKeeperResponsePtr response;
|
|
|
|
|
2022-05-09 09:16:05 +00:00
|
|
|
if (is_local)
|
2021-05-21 21:19:22 +00:00
|
|
|
{
|
2022-05-17 08:11:08 +00:00
|
|
|
assert(zk_request->isReadRequest());
|
2022-05-09 09:16:05 +00:00
|
|
|
if (check_acl && !request_processor->checkAuth(*this, session_id, true))
|
|
|
|
{
|
|
|
|
response = zk_request->makeResponse();
|
|
|
|
/// Original ZooKeeper always throws no auth, even when user provided some credentials
|
|
|
|
response->error = Coordination::Error::ZNOAUTH;
|
|
|
|
}
|
2022-05-11 07:53:32 +00:00
|
|
|
else
|
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
response = request_processor->processLocal(*this, zxid);
|
2022-05-11 07:53:32 +00:00
|
|
|
}
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-05-23 07:51:26 +00:00
|
|
|
response = request_processor->process(*this, zxid);
|
2022-05-19 09:45:38 +00:00
|
|
|
uncommitted_state.commit(zxid);
|
2021-05-21 21:19:22 +00:00
|
|
|
}
|
2021-01-19 14:22:28 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Watches for this requests are added to the watches lists
|
2021-01-19 14:22:28 +00:00
|
|
|
if (zk_request->has_watch)
|
|
|
|
{
|
|
|
|
if (response->error == Coordination::Error::ZOK)
|
|
|
|
{
|
2022-07-13 09:23:05 +00:00
|
|
|
static constexpr std::array list_requests{
|
|
|
|
Coordination::OpNum::List, Coordination::OpNum::SimpleList, Coordination::OpNum::FilteredList};
|
|
|
|
|
|
|
|
auto & watches_type = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end()
|
2021-01-19 14:22:28 +00:00
|
|
|
? list_watches
|
|
|
|
: watches;
|
|
|
|
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
watches_type[zk_request->getPath()].emplace(session_id);
|
2021-01-19 14:22:28 +00:00
|
|
|
sessions_and_watchers[session_id].emplace(zk_request->getPath());
|
|
|
|
}
|
|
|
|
else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists)
|
|
|
|
{
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
watches[zk_request->getPath()].emplace(session_id);
|
2021-01-19 14:22:28 +00:00
|
|
|
sessions_and_watchers[session_id].emplace(zk_request->getPath());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// If this requests processed successfully we need to check watches
|
2021-01-19 14:22:28 +00:00
|
|
|
if (response->error == Coordination::Error::ZOK)
|
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
auto watch_responses = request_processor->processWatches(watches, list_watches);
|
2021-01-19 14:22:28 +00:00
|
|
|
results.insert(results.end(), watch_responses.begin(), watch_responses.end());
|
|
|
|
}
|
|
|
|
|
|
|
|
response->xid = zk_request->xid;
|
|
|
|
response->zxid = getZXID();
|
|
|
|
|
2021-01-19 14:45:45 +00:00
|
|
|
results.push_back(ResponseForSession{session_id, response});
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
2021-01-19 14:22:28 +00:00
|
|
|
|
|
|
|
return results;
|
2020-11-03 14:49:30 +00:00
|
|
|
}
|
|
|
|
|
2022-05-10 13:04:35 +00:00
|
|
|
void KeeperStorage::rollbackRequest(int64_t rollback_zxid)
|
|
|
|
{
|
2022-05-13 13:43:42 +00:00
|
|
|
if (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid != rollback_zxid)
|
2022-05-12 08:58:36 +00:00
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::LOGICAL_ERROR, "Trying to rollback invalid ZXID ({}). It should be the last preprocessed.", rollback_zxid);
|
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
uncommitted_transactions.pop_back();
|
2022-05-19 09:45:38 +00:00
|
|
|
uncommitted_state.rollback(rollback_zxid);
|
2022-05-13 13:43:42 +00:00
|
|
|
}
|
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const
|
2022-05-13 13:43:42 +00:00
|
|
|
{
|
2022-05-16 13:08:10 +00:00
|
|
|
if (!digest_enabled)
|
|
|
|
return {.version = DigestVersion::NO_DIGEST};
|
|
|
|
|
2022-05-13 13:43:42 +00:00
|
|
|
if (committed || uncommitted_transactions.empty())
|
2022-05-16 12:12:29 +00:00
|
|
|
return {CURRENT_DIGEST_VERSION, nodes_digest};
|
2022-05-13 13:43:42 +00:00
|
|
|
|
2022-05-16 12:12:29 +00:00
|
|
|
return uncommitted_transactions.back().nodes_digest;
|
2022-05-10 13:04:35 +00:00
|
|
|
}
|
2021-01-19 14:22:28 +00:00
|
|
|
|
2022-05-16 13:08:10 +00:00
|
|
|
void KeeperStorage::removeDigest(const Node & node, const std::string_view path)
|
|
|
|
{
|
|
|
|
if (digest_enabled)
|
|
|
|
nodes_digest -= node.getDigest(path);
|
|
|
|
}
|
|
|
|
|
|
|
|
void KeeperStorage::addDigest(const Node & node, const std::string_view path)
|
|
|
|
{
|
|
|
|
if (digest_enabled)
|
|
|
|
{
|
|
|
|
node.invalidateDigestCache();
|
|
|
|
nodes_digest += node.getDigest(path);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-29 08:24:56 +00:00
|
|
|
void KeeperStorage::clearDeadWatches(int64_t session_id)
|
2020-11-26 14:57:32 +00:00
|
|
|
{
|
2021-08-25 09:31:02 +00:00
|
|
|
/// Clear all watches for this session
|
2020-11-26 14:57:32 +00:00
|
|
|
auto watches_it = sessions_and_watchers.find(session_id);
|
|
|
|
if (watches_it != sessions_and_watchers.end())
|
|
|
|
{
|
|
|
|
for (const auto & watch_path : watches_it->second)
|
|
|
|
{
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Maybe it's a normal watch
|
2020-11-26 14:57:32 +00:00
|
|
|
auto watch = watches.find(watch_path);
|
|
|
|
if (watch != watches.end())
|
|
|
|
{
|
|
|
|
auto & watches_for_path = watch->second;
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
watches_for_path.erase(session_id);
|
2020-11-26 14:57:32 +00:00
|
|
|
if (watches_for_path.empty())
|
|
|
|
watches.erase(watch);
|
|
|
|
}
|
2020-12-14 16:01:29 +00:00
|
|
|
|
2021-08-24 12:30:31 +00:00
|
|
|
/// Maybe it's a list watch
|
2020-12-14 16:01:29 +00:00
|
|
|
auto list_watch = list_watches.find(watch_path);
|
|
|
|
if (list_watch != list_watches.end())
|
|
|
|
{
|
|
|
|
auto & list_watches_for_path = list_watch->second;
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
list_watches_for_path.erase(session_id);
|
2020-12-14 16:01:29 +00:00
|
|
|
if (list_watches_for_path.empty())
|
|
|
|
list_watches.erase(list_watch);
|
|
|
|
}
|
2020-11-26 14:57:32 +00:00
|
|
|
}
|
2021-08-24 12:30:31 +00:00
|
|
|
|
2020-11-26 14:57:32 +00:00
|
|
|
sessions_and_watchers.erase(watches_it);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-05 10:21:34 +00:00
|
|
|
void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const
|
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
for (const auto & [session_id, watches_paths] : sessions_and_watchers)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
buf << "0x" << getHexUIntLowercase(session_id) << "\n";
|
|
|
|
for (const String & path : watches_paths)
|
2021-11-12 12:48:42 +00:00
|
|
|
buf << "\t" << path << "\n";
|
2021-11-05 10:21:34 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
|
|
|
{
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
auto write_int_container = [&buf](const auto & session_ids)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
for (int64_t session_id : session_ids)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
buf << "\t0x" << getHexUIntLowercase(session_id) << "\n";
|
2021-11-05 10:21:34 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
for (const auto & [watch_path, sessions] : watches)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
buf << watch_path << "\n";
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
write_int_container(sessions);
|
2021-11-05 10:21:34 +00:00
|
|
|
}
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
for (const auto & [watch_path, sessions] : list_watches)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
buf << watch_path << "\n";
|
keeper: store only unique session IDs for watches
This should speed up keeper, especially in case of incorrect usage (like
the case that had been fixed in #37640), especially in case on non
release build.
And also this should fix SIGKILL in stress tests.
You will find some details for one of such SIGKILL in `<details>` tag [1]:
<details>
$ pigz -cd clickhouse-server.stress.log.gz | tail
2022.05.27 16:17:24.882971 [ 637 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2022.05.27 16:17:24.896749 [ 637 ] {} <Debug> MemoryTracker: Peak memory usage (for query): 4.09 MiB.
2022.05.27 16:17:24.907163 [ 637 ] {} <Debug> Application: Shut down storages.
2022.05.27 16:17:24.907233 [ 637 ] {} <Debug> Application: Waiting for current connections to servers for tables to finish.
2022.05.27 16:17:24.934335 [ 637 ] {} <Information> Application: Closed all listening sockets. Waiting for 1 outstanding connections.
2022.05.27 16:17:29.843491 [ 637 ] {} <Information> Application: Closed connections to servers for tables. But 1 remain. Probably some tables of other users cannot finish their connections after context shutdown.
2022.05.27 16:17:29.843632 [ 637 ] {} <Debug> KeeperDispatcher: Shutting down storage dispatcher
2022.05.27 16:17:34.612616 [ 688 ] {} <Test> virtual Coordination::ZooKeeperRequest::~ZooKeeperRequest(): Processing of request xid=2147483647 took 10000 ms
2022.05.27 16:17:54.612109 [ 3176 ] {} <Debug> KeeperTCPHandler: Session #12 expired
2022.05.27 16:19:59.823038 [ 635 ] {} <Fatal> Application: Child process was terminated by signal 9 (KILL). If it is not done by 'forcestop' command or manually, the possible cause is OOM Killer (see 'dmesg' and look at the '/var/log/kern.log' for the details).
Thread 26 (Thread 0x7f1c7703f700 (LWP 708)):
0 0x000000000b074b2a in __tsan::MemoryAccessImpl(__tsan::ThreadState*, unsigned long, int, bool, bool, unsigned long long*, __tsan::Shadow) ()
1 0x000000000b08630c in __tsan::MemoryAccessRange(__tsan::ThreadState*, unsigned long, unsigned long, unsigned long, bool) ()
2 0x000000000b01ff03 in memmove ()
3 0x000000001bbc8996 in std::__1::__move<long, long> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:57
4 std::__1::move<long*, long*> (__first=0xb8600000d83304, __last=<optimized out>, __result=0x7f1c021cd000) at ../contrib/libcxx/include/__algorithm/move.h:70
5 std::__1::vector<long, std::__1::allocator<long> >::erase (this=0x7b1400584c48, __position=...) at ../contrib/libcxx/include/vector:1608
6 DB::KeeperStorage::clearDeadWatches (this=0x7b5800001ad8, this@entry=0x7b5800001800, session_id=session_id@entry=12) at ../src/Coordination/KeeperStorage.cpp:1228
7 0x000000001bbc5c55 in DB::KeeperStorage::processRequest (this=0x7b5800001800, zk_request=..., session_id=12, time=1, new_last_zxid=..., check_acl=true) at ../src/Coordination/KeeperStorage.cpp:1122
8 0x000000001bba06a3 in DB::KeeperStateMachine::commit (this=<optimized out>, log_idx=3549, data=...) at ../src/Coordination/KeeperStateMachine.cpp:143
9 0x000000001bba6193 in nuraft::state_machine::commit_ext (this=0x7b4c00001f98, params=...) at ../contrib/NuRaft/include/libnuraft/state_machine.hxx:75
10 0x00000000202c5a55 in nuraft::raft_server::commit_app_log (this=this@entry=0x7b6c00002a18, idx_to_commit=idx_to_commit@entry=3549, le=..., need_to_handle_commit_elem=true, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:311
11 0x00000000202c4f98 in nuraft::raft_server::commit_in_bg_exec (this=<optimized out>, this@entry=0x7b6c00002a18, timeout_ms=timeout_ms@entry=0, initial_commit_exec=false) at ../contrib/NuRaft/src/handle_commit.cxx:241
12 0x00000000202c4613 in nuraft::raft_server::commit_in_bg (this=this@entry=0x7b6c00002a18) at ../contrib/NuRaft/src/handle_commit.cxx:149
...
Thread 28 (Thread 0x7f1c7603d700 (LWP 710)):
0 0x00007f1d22a6d110 in __lll_lock_wait () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x00007f1d22a650a3 in pthread_mutex_lock () from /lib/x86_64-linux-gnu/libpthread.so.0
2 0x000000000b0337b0 in pthread_mutex_lock ()
3 0x00000000221884da in std::__1::__libcpp_mutex_lock (__m=0x7b4c00002088) at ../contrib/libcxx/include/__threading_support:303
4 std::__1::mutex::lock (this=0x7b4c00002088) at ../contrib/libcxx/src/mutex.cpp:33
5 0x000000001bba4188 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<optimized out>) at ../contrib/libcxx/include/__mutex_base:91
6 DB::KeeperStateMachine::getDeadSessions (this=0x7b4c00001f98) at ../src/Coordination/KeeperStateMachine.cpp:360
7 0x000000001bb79b4b in DB::KeeperServer::getDeadSessions (this=0x7b4400012700) at ../src/Coordination/KeeperServer.cpp:572
8 0x000000001bb64d1a in DB::KeeperDispatcher::sessionCleanerTask (this=<optimized out>, this@entry=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:399
...
Thread 1 (Thread 0x7f1d227148c0 (LWP 637)):
0 0x00007f1d22a69376 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0
1 0x000000000b0895e0 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*), void (*)(void*), void*) ()
2 0x000000000b017091 in pthread_cond_wait ()
3 0x0000000020569d98 in Poco::EventImpl::waitImpl (this=0x7b2000008798) at ../contrib/poco/Foundation/src/Event_POSIX.cpp:106
4 0x000000001bb636cf in Poco::Event::wait (this=0x7b2000008798) at ../contrib/poco/Foundation/include/Poco/Event.h:97
5 ThreadFromGlobalPool::join (this=<optimized out>) at ../src/Common/ThreadPool.h:217
6 DB::KeeperDispatcher::shutdown (this=0x7b640001c218) at ../src/Coordination/KeeperDispatcher.cpp:322
7 0x0000000019ca8bfc in DB::Context::shutdownKeeperDispatcher (this=<optimized out>) at ../src/Interpreters/Context.cpp:2111
8 0x000000000b0a979b in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_9::operator()() const (this=0x7ffcde44f0a0) at ../programs/server/Server.cpp:1407
</details>
[1]: https://s3.amazonaws.com/clickhouse-test-reports/37593/2613149f6bf4f242bbbf2c3c8539b5176fd77286/stress_test__thread__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-30 08:43:49 +00:00
|
|
|
write_int_container(sessions);
|
2021-11-05 10:21:34 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
auto write_str_set = [&buf](const std::unordered_set<String> & ephemeral_paths)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
for (const String & path : ephemeral_paths)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
buf << "\t" << path << "\n";
|
2021-11-05 10:21:34 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
buf << "Sessions dump (" << session_and_timeout.size() << "):\n";
|
|
|
|
|
|
|
|
for (const auto & [session_id, _] : session_and_timeout)
|
|
|
|
{
|
|
|
|
buf << "0x" << getHexUIntLowercase(session_id) << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCount() << "):\n";
|
|
|
|
for (const auto & [session_id, ephemeral_paths] : ephemerals)
|
2021-11-05 10:21:34 +00:00
|
|
|
{
|
2021-11-18 20:17:22 +00:00
|
|
|
buf << "0x" << getHexUIntLowercase(session_id) << "\n";
|
|
|
|
write_str_set(ephemeral_paths);
|
2021-11-05 10:21:34 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-18 20:17:22 +00:00
|
|
|
uint64_t KeeperStorage::getTotalWatchesCount() const
|
|
|
|
{
|
|
|
|
uint64_t ret = 0;
|
|
|
|
for (const auto & [path, subscribed_sessions] : watches)
|
|
|
|
ret += subscribed_sessions.size();
|
|
|
|
|
|
|
|
for (const auto & [path, subscribed_sessions] : list_watches)
|
|
|
|
ret += subscribed_sessions.size();
|
|
|
|
|
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t KeeperStorage::getSessionsWithWatchesCount() const
|
|
|
|
{
|
|
|
|
std::unordered_set<int64_t> counter;
|
|
|
|
for (const auto & [path, subscribed_sessions] : watches)
|
|
|
|
counter.insert(subscribed_sessions.begin(), subscribed_sessions.end());
|
|
|
|
|
|
|
|
for (const auto & [path, subscribed_sessions] : list_watches)
|
|
|
|
counter.insert(subscribed_sessions.begin(), subscribed_sessions.end());
|
|
|
|
|
|
|
|
return counter.size();
|
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t KeeperStorage::getTotalEphemeralNodesCount() const
|
|
|
|
{
|
|
|
|
uint64_t ret = 0;
|
|
|
|
for (const auto & [session_id, nodes] : ephemerals)
|
|
|
|
ret += nodes.size();
|
|
|
|
|
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-10-30 14:16:47 +00:00
|
|
|
}
|