mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #40918 from ClickHouse/fix-keeper-system-path-check
Fix check for Keeper system path modification
This commit is contained in:
commit
8ae7fb14b3
@ -898,4 +898,25 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory()
|
|||||||
registerZooKeeperRequest<OpNum::FilteredList, ZooKeeperFilteredListRequest>(*this);
|
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;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -554,4 +554,13 @@ private:
|
|||||||
ZooKeeperRequestFactory();
|
ZooKeeperRequestFactory();
|
||||||
};
|
};
|
||||||
|
|
||||||
|
enum class PathMatchResult
|
||||||
|
{
|
||||||
|
NOT_MATCH,
|
||||||
|
EXACT,
|
||||||
|
IS_CHILD
|
||||||
|
};
|
||||||
|
|
||||||
|
PathMatchResult matchPath(std::string_view path, std::string_view match_to);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
15
src/Common/ZooKeeper/tests/gtest_zookeeper.cpp
Normal file
15
src/Common/ZooKeeper/tests/gtest_zookeeper.cpp
Normal 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);
|
||||||
|
}
|
@ -13,8 +13,10 @@
|
|||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
#include "Coordination/KeeperContext.h"
|
#include <Coordination/KeeperContext.h>
|
||||||
#include <Coordination/KeeperConstants.h>
|
#include <Coordination/KeeperConstants.h>
|
||||||
|
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
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)
|
void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
|
||||||
{
|
{
|
||||||
writeBinary(static_cast<uint8_t>(snapshot.version), out);
|
writeBinary(static_cast<uint8_t>(snapshot.version), out);
|
||||||
@ -217,7 +192,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
|||||||
const auto & path = it->key;
|
const auto & path = it->key;
|
||||||
|
|
||||||
// write only the root system path because of digest
|
// 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;
|
++it;
|
||||||
continue;
|
continue;
|
||||||
@ -365,8 +340,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
|||||||
KeeperStorage::Node node{};
|
KeeperStorage::Node node{};
|
||||||
readNode(node, in, current_version, storage.acl_map);
|
readNode(node, in, current_version, storage.acl_map);
|
||||||
|
|
||||||
using enum PathMatchResult;
|
using enum Coordination::PathMatchResult;
|
||||||
auto match_result = matchPath(path, keeper_system_path);
|
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);
|
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)
|
if (match_result == IS_CHILD)
|
||||||
|
@ -879,7 +879,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
|
|||||||
path_created += seq_num_str.str();
|
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);
|
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;
|
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);
|
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;
|
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);
|
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);
|
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);
|
auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path);
|
||||||
|
|
||||||
|
@ -2141,6 +2141,38 @@ TEST_P(CoordinationTest, TestCurrentApiVersion)
|
|||||||
EXPECT_EQ(keeper_version, static_cast<uint8_t>(current_keeper_api_version));
|
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,
|
INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
|
||||||
CoordinationTest,
|
CoordinationTest,
|
||||||
::testing::ValuesIn(std::initializer_list<CompressionParam>{
|
::testing::ValuesIn(std::initializer_list<CompressionParam>{
|
||||||
|
Loading…
Reference in New Issue
Block a user