Merge pull request #40918 from ClickHouse/fix-keeper-system-path-check

Fix check for Keeper system path modification
This commit is contained in:
Antonio Andelic 2022-09-09 09:12:49 +02:00 committed by GitHub
commit 8ae7fb14b3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 87 additions and 35 deletions

View File

@ -898,4 +898,25 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory()
registerZooKeeperRequest<OpNum::FilteredList, ZooKeeperFilteredListRequest>(*this);
}
PathMatchResult matchPath(std::string_view path, std::string_view match_to)
{
using enum PathMatchResult;
if (path.ends_with('/'))
path.remove_suffix(1);
if (match_to.ends_with('/'))
match_to.remove_suffix(1);
auto [first_it, second_it] = std::mismatch(path.begin(), path.end(), match_to.begin(), match_to.end());
if (second_it != match_to.end())
return NOT_MATCH;
if (first_it == path.end())
return EXACT;
return *first_it == '/' ? IS_CHILD : NOT_MATCH;
}
}

View File

@ -554,4 +554,13 @@ private:
ZooKeeperRequestFactory();
};
enum class PathMatchResult
{
NOT_MATCH,
EXACT,
IS_CHILD
};
PathMatchResult matchPath(std::string_view path, std::string_view match_to);
}

View File

@ -0,0 +1,15 @@
#include <gtest/gtest.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
TEST(ZooKeeperTest, TestMatchPath)
{
using namespace Coordination;
ASSERT_EQ(matchPath("/path/file", "/path"), PathMatchResult::IS_CHILD);
ASSERT_EQ(matchPath("/path/file", "/path/"), PathMatchResult::IS_CHILD);
ASSERT_EQ(matchPath("/path/file", "/"), PathMatchResult::IS_CHILD);
ASSERT_EQ(matchPath("/", "/"), PathMatchResult::EXACT);
ASSERT_EQ(matchPath("/path", "/path/"), PathMatchResult::EXACT);
ASSERT_EQ(matchPath("/path/", "/path"), PathMatchResult::EXACT);
}

View File

@ -13,8 +13,10 @@
#include <filesystem>
#include <memory>
#include <Common/logger_useful.h>
#include "Coordination/KeeperContext.h"
#include <Coordination/KeeperContext.h>
#include <Coordination/KeeperConstants.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
namespace DB
{
@ -146,33 +148,6 @@ namespace
}
}
namespace
{
enum class PathMatchResult
{
NOT_MATCH,
EXACT,
IS_CHILD
};
PathMatchResult matchPath(const std::string_view path, const std::string_view match_to)
{
using enum PathMatchResult;
auto [first_it, second_it] = std::mismatch(path.begin(), path.end(), match_to.begin(), match_to.end());
if (second_it != match_to.end())
return NOT_MATCH;
if (first_it == path.end())
return EXACT;
return *first_it == '/' ? IS_CHILD : NOT_MATCH;
}
}
void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
{
writeBinary(static_cast<uint8_t>(snapshot.version), out);
@ -217,7 +192,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
const auto & path = it->key;
// write only the root system path because of digest
if (matchPath(path.toView(), keeper_system_path) == PathMatchResult::IS_CHILD)
if (Coordination::matchPath(path.toView(), keeper_system_path) == Coordination::PathMatchResult::IS_CHILD)
{
++it;
continue;
@ -365,8 +340,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
KeeperStorage::Node node{};
readNode(node, in, current_version, storage.acl_map);
using enum PathMatchResult;
auto match_result = matchPath(path, keeper_system_path);
using enum Coordination::PathMatchResult;
auto match_result = Coordination::matchPath(path, keeper_system_path);
const std::string error_msg = fmt::format("Cannot read node on path {} from a snapshot because it is used as a system node", path);
if (match_result == IS_CHILD)

View File

@ -879,7 +879,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
path_created += seq_num_str.str();
}
if (path_created.starts_with(keeper_system_path))
if (Coordination::matchPath(path_created, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH)
{
auto error_msg = fmt::format("Trying to create a node inside the internal Keeper path ({}) which is not allowed. Path: {}", keeper_system_path, path_created);
@ -1049,7 +1049,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta> new_deltas;
if (request.path.starts_with(keeper_system_path))
if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH)
{
auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path);
@ -1203,7 +1203,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
std::vector<KeeperStorage::Delta> new_deltas;
if (request.path.starts_with(keeper_system_path))
if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH)
{
auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path);
@ -1472,7 +1472,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr
{
Coordination::ZooKeeperSetACLRequest & request = dynamic_cast<Coordination::ZooKeeperSetACLRequest &>(*zk_request);
if (request.path.starts_with(keeper_system_path))
if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH)
{
auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path);

View File

@ -2141,6 +2141,38 @@ TEST_P(CoordinationTest, TestCurrentApiVersion)
EXPECT_EQ(keeper_version, static_cast<uint8_t>(current_keeper_api_version));
}
TEST_P(CoordinationTest, TestSystemNodeModify)
{
using namespace Coordination;
int64_t zxid{0};
// On INIT we abort when a system path is modified
keeper_context->server_state = KeeperContext::Phase::RUNNING;
KeeperStorage storage{500, "", keeper_context};
const auto assert_create = [&](const std::string_view path, const auto expected_code)
{
auto request = std::make_shared<ZooKeeperCreateRequest>();
request->path = path;
storage.preprocessRequest(request, 0, 0, zxid);
auto responses = storage.processRequest(request, 0, zxid);
ASSERT_FALSE(responses.empty());
const auto & response = responses[0];
ASSERT_EQ(response.response->error, expected_code) << "Unexpected error for path " << path;
++zxid;
};
assert_create("/keeper", Error::ZBADARGUMENTS);
assert_create("/keeper/with_child", Error::ZBADARGUMENTS);
assert_create(DB::keeper_api_version_path, Error::ZBADARGUMENTS);
assert_create("/keeper_map", Error::ZOK);
assert_create("/keeper1", Error::ZOK);
assert_create("/keepe", Error::ZOK);
assert_create("/keeper1/test", Error::ZOK);
}
INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
CoordinationTest,
::testing::ValuesIn(std::initializer_list<CompressionParam>{