Skip to content

Commit 0570c95

Browse files
Merge pull request ClickHouse#88742 from ClickHouse/backport/25.8/88513
Backport ClickHouse#88513 to 25.8: Keeper improvement: add config for checking node ACL on removal
2 parents 261935c + 1139895 commit 0570c95

File tree

6 files changed

+135
-1
lines changed

6 files changed

+135
-1
lines changed

src/Coordination/CoordinationSettings.cpp

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,8 @@ namespace ErrorCodes
6969
DECLARE(UInt64, log_slow_total_threshold_ms, 5000, "Requests for which the total latency is larger than this settings will be logged", 0) \
7070
DECLARE(UInt64, log_slow_cpu_threshold_ms, 100, "Requests for which the CPU (preprocessing and processing) latency is larger than this settings will be logged", 0) \
7171
DECLARE(UInt64, log_slow_connection_operation_threshold_ms, 1000, "Log message if a certain operation took too long inside a single connection", 0) \
72-
DECLARE(Bool, use_xid_64, false, "Enable 64-bit XID. It is disabled by default because of backward compatibility", 0)
72+
DECLARE(Bool, use_xid_64, false, "Enable 64-bit XID. It is disabled by default because of backward compatibility", 0) \
73+
DECLARE(Bool, check_node_acl_on_remove, false, "When trying to remove a node, check ACLs from both the node itself and the parent node. If disabled, default behaviour will be used where only ACL from the parent node is checked", 0) \
7374

7475
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
7576
IMPLEMENT_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
@@ -277,6 +278,11 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const
277278
write_int(coordination_settings[CoordinationSetting::log_slow_cpu_threshold_ms]);
278279
writeText("log_slow_connection_operation_threshold_ms=", buf);
279280
write_int(coordination_settings[CoordinationSetting::log_slow_connection_operation_threshold_ms]);
281+
282+
writeText("use_xid_64=", buf);
283+
write_bool(coordination_settings[CoordinationSetting::use_xid_64]);
284+
writeText("check_node_acl_on_remove=", buf);
285+
write_bool(coordination_settings[CoordinationSetting::check_node_acl_on_remove]);
280286
}
281287

282288
KeeperConfigurationAndSettingsPtr

src/Coordination/KeeperStorage.cpp

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,7 @@ namespace DB
5252
namespace CoordinationSetting
5353
{
5454
extern const CoordinationSettingsUInt64 log_slow_cpu_threshold_ms;
55+
extern const CoordinationSettingsBool check_node_acl_on_remove;
5556
}
5657

5758
namespace ErrorCodes
@@ -1828,6 +1829,12 @@ processLocal(const Coordination::ZooKeeperGetRequest & zk_request, Storage & sto
18281829
template <typename Storage>
18291830
bool checkAuth(const Coordination::ZooKeeperRemoveRequest & zk_request, Storage & storage, int64_t session_id, bool is_local)
18301831
{
1832+
if (auto check_node_acl = storage.keeper_context->getCoordinationSettings()[CoordinationSetting::check_node_acl_on_remove];
1833+
check_node_acl && !storage.checkACL(zk_request.getPath(), Coordination::ACL::Delete, session_id, is_local))
1834+
{
1835+
return false;
1836+
}
1837+
18311838
return storage.checkACL(Coordination::parentNodePath(zk_request.getPath()), Coordination::ACL::Delete, session_id, is_local);
18321839
}
18331840

tests/integration/test_keeper_remove_acl/__init__.py

Whitespace-only changes.
Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,7 @@
1+
<clickhouse>
2+
<keeper_server>
3+
<coordination_settings>
4+
<check_node_acl_on_remove>0</check_node_acl_on_remove>
5+
</coordination_settings>
6+
</keeper_server>
7+
</clickhouse>
Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,25 @@
1+
<clickhouse>
2+
<keeper_server>
3+
<tcp_port>9181</tcp_port>
4+
<server_id>1</server_id>
5+
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
6+
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
7+
8+
<coordination_settings>
9+
<operation_timeout_ms>5000</operation_timeout_ms>
10+
<session_timeout_ms>10000</session_timeout_ms>
11+
<snapshot_distance>75</snapshot_distance>
12+
<raft_logs_level>trace</raft_logs_level>
13+
</coordination_settings>
14+
15+
<raft_configuration>
16+
<server>
17+
<id>1</id>
18+
<hostname>node</hostname>
19+
<port>9234</port>
20+
<can_become_leader>true</can_become_leader>
21+
<priority>3</priority>
22+
</server>
23+
</raft_configuration>
24+
</keeper_server>
25+
</clickhouse>
Lines changed: 89 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,89 @@
1+
#!/usr/bin/env python3
2+
3+
import pytest
4+
import logging
5+
6+
import helpers.keeper_utils as keeper_utils
7+
from helpers.cluster import ClickHouseCluster
8+
9+
from kazoo.security import make_digest_acl
10+
from kazoo.exceptions import NoAuthError
11+
12+
13+
cluster = ClickHouseCluster(__file__)
14+
node = cluster.add_instance(
15+
"node",
16+
main_configs=["configs/enable_keeper1.xml", "configs/check_node_acl_on_remove.xml"],
17+
stay_alive=True,
18+
)
19+
20+
21+
@pytest.fixture(scope="module")
22+
def started_cluster():
23+
try:
24+
cluster.start()
25+
26+
yield cluster
27+
28+
finally:
29+
cluster.shutdown()
30+
31+
32+
def wait_nodes():
33+
keeper_utils.wait_nodes(cluster, [node])
34+
35+
36+
def get_fake_zk(nodename, timeout=30.0):
37+
return keeper_utils.get_fake_zk(cluster, nodename, timeout=timeout)
38+
39+
40+
def stop_zk_connection(zk_conn):
41+
zk_conn.stop()
42+
zk_conn.close()
43+
44+
45+
def test_server_restart(started_cluster):
46+
try:
47+
wait_nodes()
48+
49+
node.stop_clickhouse()
50+
node.replace_in_config(
51+
"/etc/clickhouse-server/config.d/check_node_acl_on_remove.xml", "1", "0"
52+
)
53+
node.start_clickhouse()
54+
55+
def create_node_with_acl():
56+
node_zk = get_fake_zk("node")
57+
node_zk.add_auth("digest", "clickhouse:password")
58+
59+
if node_zk.exists("/test_acl_node"):
60+
node_zk.delete("/test_acl_node")
61+
62+
acl = make_digest_acl("clickhouse", "password", all=True)
63+
node_zk.create("/test_acl_node", b"test_data", acl=[acl])
64+
stop_zk_connection(node_zk)
65+
66+
def delete_node():
67+
node_zk = get_fake_zk("node")
68+
node_zk.delete("/test_acl_node")
69+
stop_zk_connection(node_zk)
70+
71+
create_node_with_acl()
72+
delete_node()
73+
node.stop_clickhouse()
74+
node.replace_in_config(
75+
"/etc/clickhouse-server/config.d/check_node_acl_on_remove.xml", "0", "1"
76+
)
77+
node.start_clickhouse()
78+
79+
create_node_with_acl()
80+
81+
with pytest.raises(NoAuthError):
82+
delete_node()
83+
finally:
84+
try:
85+
stop_zk_connection(
86+
node_zk,
87+
)
88+
except:
89+
pass

0 commit comments

Comments
 (0)