More fixes

This commit is contained in:
Antonio Andelic 2024-09-02 11:44:21 +02:00
parent 596ba574e3
commit 79fc8d67ad
4 changed files with 35 additions and 29 deletions

View File

@ -471,12 +471,21 @@ void KeeperStorageSnapshot<Storage>::deserialize(SnapshotDeserializationResult<S
{
#ifdef NDEBUG
/// TODO (alesapin) remove this, it should be always CORRUPTED_DATA.
LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}", itr.value.stats.numChildren(), itr.value.getChildren().size(), itr.key);
LOG_ERROR(
getLogger("KeeperSnapshotManager"),
"Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}",
itr.value.stats.numChildren(),
itr.value.getChildren().size(),
itr.key);
#else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}",
itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}",
itr.value.stats.numChildren(),
itr.value.getChildren().size(),
itr.key);
#endif
}
}

View File

@ -143,7 +143,7 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl(
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
/// CHANGED event never trigger list watches
for (const auto & path_to_check : paths_to_check_for_list_watches)
{
@ -712,17 +712,19 @@ void KeeperStorage<Container>::UncommittedState::addDeltas(std::list<Delta> new_
template<typename Container>
void KeeperStorage<Container>::UncommittedState::cleanup(int64_t commit_zxid)
{
for (const auto & [transaction_zxid, transaction_nodes] : zxid_to_nodes)
for (auto it = zxid_to_nodes.begin(); it != zxid_to_nodes.end(); it = zxid_to_nodes.erase(it))
{
const auto & [transaction_zxid, transaction_nodes] = *it;
if (transaction_zxid > commit_zxid)
break;
std::cout << transaction_zxid << std::endl;
for (const auto node : transaction_nodes)
{
std::cout << node << std::endl;
auto node_it = nodes.find(node);
chassert(node_it != nodes.end());
if (node_it == nodes.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing expected uncommitted node '{}'", node);
node_it->second.applied_zxids.erase(transaction_zxid);
if (node_it->second.applied_zxids.empty())
nodes.erase(node_it);
@ -1295,7 +1297,6 @@ bool checkAuth(const T & /*zk_request*/, Storage & /*storage*/, int64_t /*sessio
/// Default implementations ///
/// HEARTBEAT Request ///
template <typename Storage>
Coordination::ZooKeeperResponsePtr process(
const Coordination::ZooKeeperHeartbeatRequest & zk_request,
@ -1306,11 +1307,9 @@ Coordination::ZooKeeperResponsePtr process(
response_ptr->error = storage.commit(std::move(deltas));
return response_ptr;
}
/// HEARTBEAT Request ///
/// SYNC Request ///
template <typename Storage>
Coordination::ZooKeeperResponsePtr
process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, std::list<KeeperStorageBase::Delta> /* deltas */)
@ -1319,7 +1318,6 @@ process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* stor
response->path = zk_request.path;
return response;
}
/// SYNC Request ///
/// CREATE Request ///
@ -1793,18 +1791,19 @@ std::list<KeeperStorageBase::Delta> preprocess(
if (zk_request.version != -1 && zk_request.version != node->stats.version)
return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}};
new_deltas.emplace_back(
zk_request.path,
zxid,
KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version});
KeeperStorageBase::UpdateNodeStatDelta node_delta(*node);
node_delta.version = zk_request.version;
auto & new_stats = node_delta.new_stats;
new_stats.version++;
new_stats.mzxid = zxid;
new_stats.mtime = time;
new_stats.data_size = static_cast<uint32_t>(zk_request.data.size());
new_deltas.emplace_back(zk_request.path, zxid, std::move(node_delta));
new_deltas.emplace_back(
zk_request.path,
zxid,
KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version});
auto parent_path = parentNodePath(zk_request.path);
auto parent_node = storage.uncommitted_state.getNode(parent_path);
@ -1868,7 +1867,9 @@ std::list<KeeperStorageBase::Delta> preprocess(
template <bool local, typename Storage>
Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list<KeeperStorageBase::Delta> deltas)
{
auto response = std::make_shared<Coordination::ZooKeeperListResponse>();
std::shared_ptr<Coordination::ZooKeeperListResponse> response = zk_request.getOpNum() == Coordination::OpNum::SimpleList
? std::make_shared<Coordination::ZooKeeperSimpleListResponse>()
: std::make_shared<Coordination::ZooKeeperListResponse>();
if constexpr (!local)
{
@ -2215,17 +2216,16 @@ Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMul
return response;
}
template <typename Storage>
KeeperStorageBase::ResponsesForSessions processWatches(
const Coordination::ZooKeeperMultiRequest & zk_request, typename Storage::Watches & watches, typename Storage::Watches & list_watches)
const Coordination::ZooKeeperMultiRequest & zk_request, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches)
{
typename Storage::ResponsesForSessions result;
KeeperStorageBase::ResponsesForSessions result;
const auto & subrequests = zk_request.requests;
for (const auto & generic_request : subrequests)
{
auto responses = callOnConcreteRequestType(
generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); });
*generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); });
result.insert(result.end(), responses.begin(), responses.end());
}
return result;
@ -2506,7 +2506,7 @@ UInt64 KeeperStorage<Container>::calculateNodesDigest(UInt64 current_digest, con
auto updated_node_it = updated_nodes.find(delta.path);
if (updated_node_it == updated_nodes.end())
{
node = std::make_shared<KeeperStorage::Node>();
node = std::make_shared<Node>();
node->shallowCopy(*uncommitted_state.getNode(delta.path));
current_digest -= node->getDigest(delta.path);
updated_nodes.emplace(delta.path, node);
@ -2772,7 +2772,6 @@ KeeperStorage<Container>::ResponsesForSessions KeeperStorage<Container>::process
{
Coordination::ZooKeeperResponsePtr response;
uncommitted_state.tryGetNodeFromStorage("/node_with_acl");
if (is_local)
{
chassert(zk_request->isReadRequest());
@ -2793,7 +2792,6 @@ KeeperStorage<Container>::ResponsesForSessions KeeperStorage<Container>::process
std::lock_guard lock(storage_mutex);
response = process(concrete_zk_request, *this, std::move(deltas));
}
uncommitted_state.tryGetNodeFromStorage("/node_with_acl");
/// Watches for this requests are added to the watches lists
if (zk_request->has_watch)

View File

@ -357,8 +357,6 @@ public:
NodeStats old_stats;
NodeStats new_stats;
int32_t old_seq_num;
int32_t new_seq_num;
int32_t version{-1};
};

View File

@ -1114,6 +1114,7 @@ void Runner::runBenchmarkFromLog()
else
{
request_from_log->connection = get_zookeeper_connection(request_from_log->session_id);
request_from_log->executor_id %= concurrency;
push_request(std::move(*request_from_log));
}