Backport #55119 to 23.8: Fix deadlock in LDAP assigned role update

This commit is contained in:
robot-clickhouse 2023-10-09 14:13:40 +00:00
parent 745df5d43b
commit bbdffd0789
7 changed files with 207 additions and 9 deletions

View File

@ -0,0 +1,16 @@
version: '2.3'
services:
openldap:
image: bitnami/openldap:2.6.6
restart: always
environment:
LDAP_ROOT: dc=example,dc=org
LDAP_ADMIN_DN: cn=admin,dc=example,dc=org
LDAP_ADMIN_USERNAME: admin
LDAP_ADMIN_PASSWORD: clickhouse
LDAP_USER_DC: users
LDAP_USERS: janedoe,johndoe
LDAP_PASSWORDS: qwerty,qwertz
LDAP_PORT_NUMBER: ${LDAP_INTERNAL_PORT:-1389}
ports:
- ${LDAP_EXTERNAL_PORT:-1389}:${LDAP_INTERNAL_PORT:-1389}

View File

@ -61,7 +61,7 @@ private: // IAccessStorage implementations.
bool areLDAPCredentialsValidNoLock(const User & user, const Credentials & credentials,
const ExternalAuthenticators & external_authenticators, LDAPClient::SearchResultsList & role_search_results) const;
mutable std::recursive_mutex mutex;
mutable std::recursive_mutex mutex; // Note: Reentrace possible by internal role lookup via access_control
AccessControl & access_control;
String ldap_server_name;
LDAPClient::RoleSearchParamsList role_search_params;

View File

@ -44,12 +44,12 @@ private:
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) TSA_REQUIRES(mutex);
bool removeNoLock(const UUID & id, bool throw_if_not_exists) TSA_REQUIRES(mutex);
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) TSA_REQUIRES(mutex);
bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
bool removeNoLock(const UUID & id, bool throw_if_not_exists);
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
void removeAllExceptNoLock(const std::vector<UUID> & ids_to_keep) TSA_REQUIRES(mutex);
void removeAllExceptNoLock(const boost::container::flat_set<UUID> & ids_to_keep) TSA_REQUIRES(mutex);
void removeAllExceptNoLock(const std::vector<UUID> & ids_to_keep);
void removeAllExceptNoLock(const boost::container::flat_set<UUID> & ids_to_keep);
struct Entry
{
@ -57,9 +57,9 @@ private:
AccessEntityPtr entity;
};
mutable std::mutex mutex;
std::unordered_map<UUID, Entry> entries_by_id TSA_GUARDED_BY(mutex); /// We want to search entries both by ID and by the pair of name and type.
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(AccessEntityType::MAX)] TSA_GUARDED_BY(mutex);
mutable std::recursive_mutex mutex; // Note: Reentrace possible via LDAPAccessStorage
std::unordered_map<UUID, Entry> entries_by_id; /// We want to search entries both by ID and by the pair of name and type.
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(AccessEntityType::MAX)];
AccessChangesNotifier & changes_notifier;
const bool backup_allowed = false;
};

View File

@ -436,6 +436,7 @@ class ClickHouseCluster:
self.with_net_trics = False
self.with_redis = False
self.with_cassandra = False
self.with_ldap = False
self.with_jdbc_bridge = False
self.with_nginx = False
self.with_hive = False
@ -522,6 +523,13 @@ class ClickHouseCluster:
self.cassandra_ip = None
self.cassandra_id = self.get_instance_docker_id(self.cassandra_host)
# available when with_ldap == True
self.ldap_host = "openldap"
self.ldap_ip = None
self.ldap_container = None
self.ldap_port = 1389
self.ldap_id = self.get_instance_docker_id(self.ldap_host)
# available when with_rabbitmq == True
self.rabbitmq_host = "rabbitmq1"
self.rabbitmq_ip = None
@ -1442,6 +1450,23 @@ class ClickHouseCluster:
]
return self.base_cassandra_cmd
def setup_ldap_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_ldap = True
env_variables["LDAP_EXTERNAL_PORT"] = str(self.ldap_port)
self.base_cmd.extend(
["--file", p.join(docker_compose_yml_dir, "docker_compose_ldap.yml")]
)
self.base_ldap_cmd = [
"docker-compose",
"--env-file",
instance.env_file,
"--project-name",
self.project_name,
"--file",
p.join(docker_compose_yml_dir, "docker_compose_ldap.yml"),
]
return self.base_ldap_cmd
def setup_jdbc_bridge_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_jdbc_bridge = True
env_variables["JDBC_DRIVER_LOGS"] = self.jdbc_driver_logs_dir
@ -1530,6 +1555,7 @@ class ClickHouseCluster:
with_minio=False,
with_azurite=False,
with_cassandra=False,
with_ldap=False,
with_jdbc_bridge=False,
with_hive=False,
with_coredns=False,
@ -1631,6 +1657,7 @@ class ClickHouseCluster:
with_hive=with_hive,
with_coredns=with_coredns,
with_cassandra=with_cassandra,
with_ldap=with_ldap,
allow_analyzer=allow_analyzer,
server_bin_path=self.server_bin_path,
odbc_bridge_bin_path=self.odbc_bridge_bin_path,
@ -1860,6 +1887,11 @@ class ClickHouseCluster:
)
)
if with_ldap and not self.with_ldap:
cmds.append(
self.setup_ldap_cmd(instance, env_variables, docker_compose_yml_dir)
)
if with_jdbc_bridge and not self.with_jdbc_bridge:
cmds.append(
self.setup_jdbc_bridge_cmd(
@ -2566,6 +2598,32 @@ class ClickHouseCluster:
raise Exception("Can't wait Cassandra to start")
def wait_ldap_to_start(self, timeout=180):
self.ldap_ip = self.get_instance_ip(self.ldap_host)
self.ldap_container = self.get_docker_handle(self.ldap_id)
start = time.time()
while time.time() - start < timeout:
try:
logging.info(
f"Check LDAP Online {self.ldap_id} {self.ldap_ip} {self.ldap_port}"
)
self.exec_in_container(
self.ldap_id,
[
"bash",
"-c",
f"/opt/bitnami/openldap/bin/ldapsearch -x -H ldap://{self.ldap_ip}:{self.ldap_port} -D cn=admin,dc=example,dc=org -w clickhouse -b dc=example,dc=org",
],
user="root",
)
logging.info("LDAP Online")
return
except Exception as ex:
logging.warning("Can't connect to LDAP: %s", str(ex))
time.sleep(1)
raise Exception("Can't wait LDAP to start")
def start(self):
pytest_xdist_logging_to_separate_files.setup()
logging.info("Running tests in {}".format(self.base_path))
@ -2894,6 +2952,11 @@ class ClickHouseCluster:
self.up_called = True
self.wait_cassandra_to_start()
if self.with_ldap and self.base_ldap_cmd:
subprocess_check_call(self.base_ldap_cmd + ["up", "-d"])
self.up_called = True
self.wait_ldap_to_start()
if self.with_jdbc_bridge and self.base_jdbc_bridge_cmd:
os.makedirs(self.jdbc_driver_logs_dir)
os.chmod(self.jdbc_driver_logs_dir, stat.S_IRWXU | stat.S_IRWXO)
@ -3171,6 +3234,7 @@ class ClickHouseInstance:
with_hive,
with_coredns,
with_cassandra,
with_ldap,
allow_analyzer,
server_bin_path,
odbc_bridge_bin_path,
@ -3255,6 +3319,7 @@ class ClickHouseInstance:
self.with_minio = with_minio
self.with_azurite = with_azurite
self.with_cassandra = with_cassandra
self.with_ldap = with_ldap
self.with_jdbc_bridge = with_jdbc_bridge
self.with_hive = with_hive
self.with_coredns = with_coredns

View File

@ -0,0 +1,22 @@
<clickhouse>
<ldap_servers>
<openldap>
<host>openldap</host>
<port>1389</port>
<bind_dn>cn={user_name},ou=users,dc=example,dc=org</bind_dn>
<enable_tls>no</enable_tls>
</openldap>
</ldap_servers>
<user_directories>
<ldap>
<server>openldap</server>
<role_mapping>
<base_dn>dc=example,dc=org</base_dn>
<scope>subtree</scope>
<search_filter>(&amp;(objectClass=groupOfNames)(member={bind_dn}))</search_filter>
<attribute>cn</attribute>
<prefix>clickhouse-</prefix>
</role_mapping>
</ldap>
</user_directories>
</clickhouse>

View File

@ -0,0 +1,95 @@
import logging
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
LDAP_ADMIN_BIND_DN = "cn=admin,dc=example,dc=org"
LDAP_ADMIN_PASSWORD = "clickhouse"
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance(
"instance", main_configs=["configs/ldap_with_role_mapping.xml"], with_ldap=True
)
@pytest.fixture(scope="module", autouse=True)
def ldap_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def add_ldap_group(ldap_cluster, group_cn, member_cn):
code, (stdout, stderr) = ldap_cluster.ldap_container.exec_run(
[
"sh",
"-c",
"""echo "dn: cn={group_cn},dc=example,dc=org
objectClass: top
objectClass: groupOfNames
member: cn={member_cn},ou=users,dc=example,dc=org" | \
ldapadd -H ldap://{host}:{port} -D "{admin_bind_dn}" -x -w {admin_password}
""".format(
host=ldap_cluster.ldap_host,
port=ldap_cluster.ldap_port,
admin_bind_dn=LDAP_ADMIN_BIND_DN,
admin_password=LDAP_ADMIN_PASSWORD,
group_cn=group_cn,
member_cn=member_cn,
),
],
demux=True,
)
logging.debug(
f"test_ldap_external_user_directory code:{code} stdout:{stdout}, stderr:{stderr}"
)
assert code == 0
def test_authentication_pass():
assert instance.query(
"select currentUser()", user="janedoe", password="qwerty"
) == TSV([["janedoe"]])
def test_authentication_fail():
# User doesn't exist.
assert "doesnotexist: Authentication failed" in instance.query_and_get_error(
"SELECT currentUser()", user="doesnotexist"
)
# Wrong password.
assert "janedoe: Authentication failed" in instance.query_and_get_error(
"SELECT currentUser()", user="janedoe", password="123"
)
def test_role_mapping(ldap_cluster):
instance.query("CREATE ROLE role_1")
instance.query("CREATE ROLE role_2")
add_ldap_group(ldap_cluster, group_cn="clickhouse-role_1", member_cn="johndoe")
add_ldap_group(ldap_cluster, group_cn="clickhouse-role_2", member_cn="johndoe")
assert instance.query(
"select currentUser()", user="johndoe", password="qwertz"
) == TSV([["johndoe"]])
assert instance.query(
"select role_name from system.current_roles ORDER BY role_name",
user="johndoe",
password="qwertz",
) == TSV([["role_1"], ["role_2"]])
instance.query("CREATE ROLE role_3")
add_ldap_group(ldap_cluster, group_cn="clickhouse-role_3", member_cn="johndoe")
# Check that non-existing role in ClickHouse is ignored during role update
# See https://github.com/ClickHouse/ClickHouse/issues/54318
add_ldap_group(ldap_cluster, group_cn="clickhouse-role_4", member_cn="johndoe")
assert instance.query(
"select role_name from system.current_roles ORDER BY role_name",
user="johndoe",
password="qwertz",
) == TSV([["role_1"], ["role_2"], ["role_3"]])