mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Add tests
This commit is contained in:
parent
cd815efaee
commit
11ac483e12
@ -88,7 +88,7 @@ std::string checkAndGetSuperdigest(const Poco::Util::AbstractConfiguration & con
|
|||||||
if (scheme_and_id.size() != 2 || scheme_and_id[0] != "super")
|
if (scheme_and_id.size() != 2 || scheme_and_id[0] != "super")
|
||||||
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Incorrect superdigest in keeper_server config. Must be 'super:base64string'");
|
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Incorrect superdigest in keeper_server config. Must be 'super:base64string'");
|
||||||
|
|
||||||
return scheme_and_id[1];
|
return user_and_digest;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -57,7 +57,7 @@ static String generateDigest(const String & userdata)
|
|||||||
{
|
{
|
||||||
std::vector<String> user_password;
|
std::vector<String> user_password;
|
||||||
boost::split(user_password, userdata, [](char c) { return c == ':'; });
|
boost::split(user_password, userdata, [](char c) { return c == ':'; });
|
||||||
return user_password[0] + base64Encode(getSHA1(user_password[1]));
|
return user_password[0] + ":" + base64Encode(getSHA1(user_password[1]));
|
||||||
}
|
}
|
||||||
|
|
||||||
static bool checkACL(int32_t permission, const Coordination::ACLs & node_acls, const std::vector<KeeperStorage::AuthID> & session_auths)
|
static bool checkACL(int32_t permission, const Coordination::ACLs & node_acls, const std::vector<KeeperStorage::AuthID> & session_auths)
|
||||||
@ -65,7 +65,7 @@ static bool checkACL(int32_t permission, const Coordination::ACLs & node_acls, c
|
|||||||
if (node_acls.empty())
|
if (node_acls.empty())
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
for (const auto & session_auth : session_auths)
|
for (const auto & session_auth : session_auths)
|
||||||
if (session_auth.scheme == "super")
|
if (session_auth.scheme == "super")
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
@ -76,8 +76,9 @@ static bool checkACL(int32_t permission, const Coordination::ACLs & node_acls, c
|
|||||||
if (node_acls[i].scheme == "world" && node_acls[i].id == "anyone")
|
if (node_acls[i].scheme == "world" && node_acls[i].id == "anyone")
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
if (node_acls[i].scheme == session_auths[i].scheme && node_acls[i].id == session_auths[i].id)
|
for (size_t j = 0; j < session_auths.size(); ++j)
|
||||||
return true;
|
if (node_acls[i].scheme == session_auths[j].scheme && node_acls[i].id == session_auths[j].id)
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -90,26 +91,40 @@ static bool fixupACL(
|
|||||||
std::vector<Coordination::ACL> & result_acls)
|
std::vector<Coordination::ACL> & result_acls)
|
||||||
{
|
{
|
||||||
if (request_acls.empty())
|
if (request_acls.empty())
|
||||||
return false;
|
return true;
|
||||||
|
|
||||||
|
bool valid_found = false;
|
||||||
for (const auto & request_acl : request_acls)
|
for (const auto & request_acl : request_acls)
|
||||||
{
|
{
|
||||||
if (request_acl.scheme == "world" && request_acl.id == "anyone")
|
if (request_acl.scheme == "auth")
|
||||||
{
|
|
||||||
result_acls.push_back(request_acl);
|
|
||||||
}
|
|
||||||
else if (request_acl.scheme == "auth")
|
|
||||||
{
|
{
|
||||||
for (const auto & current_id : current_ids)
|
for (const auto & current_id : current_ids)
|
||||||
{
|
{
|
||||||
|
valid_found = true;
|
||||||
Coordination::ACL new_acl = request_acl;
|
Coordination::ACL new_acl = request_acl;
|
||||||
new_acl.scheme = current_id.scheme;
|
new_acl.scheme = current_id.scheme;
|
||||||
new_acl.id = current_id.id;
|
new_acl.id = current_id.id;
|
||||||
result_acls.push_back(new_acl);
|
result_acls.push_back(new_acl);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
else if (request_acl.scheme == "world" && request_acl.id == "anyone")
|
||||||
|
{
|
||||||
|
valid_found = true;
|
||||||
|
}
|
||||||
|
else if (request_acl.scheme == "digest")
|
||||||
|
{
|
||||||
|
Coordination::ACL new_acl = request_acl;
|
||||||
|
|
||||||
|
/// Bad auth
|
||||||
|
if (std::count(new_acl.id.begin(), new_acl.id.end(), ':') != 1)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
valid_found = true;
|
||||||
|
new_acl.id = generateDigest(new_acl.id);
|
||||||
|
result_acls.push_back(new_acl);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return !result_acls.empty();
|
return valid_found;
|
||||||
}
|
}
|
||||||
|
|
||||||
static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type)
|
static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type)
|
||||||
@ -242,10 +257,6 @@ struct KeeperStorageCreateRequest final : public KeeperStorageRequest
|
|||||||
{
|
{
|
||||||
response.error = Coordination::Error::ZNOCHILDRENFOREPHEMERALS;
|
response.error = Coordination::Error::ZNOCHILDRENFOREPHEMERALS;
|
||||||
}
|
}
|
||||||
else if (request.acls.size() != storage.session_and_auth[session_id].size())
|
|
||||||
{
|
|
||||||
response.error = Coordination::Error::ZBADARGUMENTS;
|
|
||||||
}
|
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
auto & session_auth_ids = storage.session_and_auth[session_id];
|
auto & session_auth_ids = storage.session_and_auth[session_id];
|
||||||
@ -364,7 +375,7 @@ struct KeeperStorageRemoveRequest final : public KeeperStorageRequest
|
|||||||
bool checkAuth(KeeperStorage & storage, int64_t session_id) const override
|
bool checkAuth(KeeperStorage & storage, int64_t session_id) const override
|
||||||
{
|
{
|
||||||
auto & container = storage.container;
|
auto & container = storage.container;
|
||||||
auto it = container.find(zk_request->getPath());
|
auto it = container.find(parentPath(zk_request->getPath()));
|
||||||
if (it == container.end())
|
if (it == container.end())
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
@ -446,18 +457,6 @@ struct KeeperStorageRemoveRequest final : public KeeperStorageRequest
|
|||||||
|
|
||||||
struct KeeperStorageExistsRequest final : public KeeperStorageRequest
|
struct KeeperStorageExistsRequest final : public KeeperStorageRequest
|
||||||
{
|
{
|
||||||
bool checkAuth(KeeperStorage & storage, int64_t session_id) const override
|
|
||||||
{
|
|
||||||
auto & container = storage.container;
|
|
||||||
auto it = container.find(zk_request->getPath());
|
|
||||||
if (it == container.end())
|
|
||||||
return true;
|
|
||||||
|
|
||||||
const auto & node_acls = it->value.acls;
|
|
||||||
const auto & session_auths = storage.session_and_auth[session_id];
|
|
||||||
return checkACL(Coordination::ACL::Read, node_acls, session_auths);
|
|
||||||
}
|
|
||||||
|
|
||||||
using KeeperStorageRequest::KeeperStorageRequest;
|
using KeeperStorageRequest::KeeperStorageRequest;
|
||||||
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage & storage, int64_t /*zxid*/, int64_t /* session_id */) const override
|
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage & storage, int64_t /*zxid*/, int64_t /* session_id */) const override
|
||||||
{
|
{
|
||||||
@ -763,33 +762,26 @@ struct KeeperStorageAuthRequest final : public KeeperStorageRequest
|
|||||||
Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast<Coordination::ZooKeeperAuthResponse &>(*response_ptr);
|
Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast<Coordination::ZooKeeperAuthResponse &>(*response_ptr);
|
||||||
auto & sessions_and_auth = storage.session_and_auth;
|
auto & sessions_and_auth = storage.session_and_auth;
|
||||||
|
|
||||||
if (auth_request.scheme == "super")
|
if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1)
|
||||||
{
|
{
|
||||||
if (generateDigest(auth_request.data) == storage.superdigest)
|
auth_response.error = Coordination::Error::ZAUTHFAILED;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto digest = generateDigest(auth_request.data);
|
||||||
|
if (digest == storage.superdigest)
|
||||||
{
|
{
|
||||||
KeeperStorage::AuthID auth{"super", ""};
|
KeeperStorage::AuthID auth{"super", ""};
|
||||||
sessions_and_auth[session_id].emplace_back(auth);
|
sessions_and_auth[session_id].emplace_back(auth);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
auth_response.error = Coordination::Error::ZAUTHFAILED;
|
KeeperStorage::AuthID auth{auth_request.scheme, digest};
|
||||||
|
auto & session_ids = sessions_and_auth[session_id];
|
||||||
|
if (std::find(session_ids.begin(), session_ids.end(), auth) == session_ids.end())
|
||||||
|
sessions_and_auth[session_id].emplace_back(auth);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
else if (auth_request.scheme == "world" && auth_request.data == "anyone")
|
|
||||||
{
|
|
||||||
KeeperStorage::AuthID auth{"world", "anyone"};
|
|
||||||
sessions_and_auth[session_id].emplace_back(auth);
|
|
||||||
}
|
|
||||||
else if (auth_request.scheme != "digest")
|
|
||||||
{
|
|
||||||
auth_response.error = Coordination::Error::ZAUTHFAILED;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
KeeperStorage::AuthID auth{auth_request.scheme, generateDigest(auth_request.data)};
|
|
||||||
auto & session_ids = sessions_and_auth[session_id];
|
|
||||||
if (std::find(session_ids.begin(), session_ids.end(), auth) == session_ids.end())
|
|
||||||
sessions_and_auth[session_id].emplace_back(auth);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return { response_ptr, {} };
|
return { response_ptr, {} };
|
||||||
@ -932,7 +924,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina
|
|||||||
if (!storage_request->checkAuth(*this, session_id))
|
if (!storage_request->checkAuth(*this, session_id))
|
||||||
{
|
{
|
||||||
response = zk_request->makeResponse();
|
response = zk_request->makeResponse();
|
||||||
response->error = Coordination::Error::ZAUTHFAILED;
|
/// Original ZooKeeper always throws no auth, even when user provided some credentials
|
||||||
|
response->error = Coordination::Error::ZNOAUTH;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
1
tests/integration/test_keeper_auth/__init__.py
Normal file
1
tests/integration/test_keeper_auth/__init__.py
Normal file
@ -0,0 +1 @@
|
|||||||
|
#!/usr/bin/env python3
|
24
tests/integration/test_keeper_auth/configs/keeper_config.xml
Normal file
24
tests/integration/test_keeper_auth/configs/keeper_config.xml
Normal file
@ -0,0 +1,24 @@
|
|||||||
|
<yandex>
|
||||||
|
<keeper_server>
|
||||||
|
<tcp_port>9181</tcp_port>
|
||||||
|
<server_id>1</server_id>
|
||||||
|
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||||
|
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||||
|
<superdigest>super:0DPiKuNIrrVmD8IUCuw1hQxNqZc=</superdigest>
|
||||||
|
|
||||||
|
<coordination_settings>
|
||||||
|
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||||
|
<session_timeout_ms>10000</session_timeout_ms>
|
||||||
|
<raft_logs_level>trace</raft_logs_level>
|
||||||
|
<force_sync>false</force_sync>
|
||||||
|
</coordination_settings>
|
||||||
|
|
||||||
|
<raft_configuration>
|
||||||
|
<server>
|
||||||
|
<id>1</id>
|
||||||
|
<hostname>localhost</hostname>
|
||||||
|
<port>44444</port>
|
||||||
|
</server>
|
||||||
|
</raft_configuration>
|
||||||
|
</keeper_server>
|
||||||
|
</yandex>
|
13
tests/integration/test_keeper_auth/configs/logs_conf.xml
Normal file
13
tests/integration/test_keeper_auth/configs/logs_conf.xml
Normal file
@ -0,0 +1,13 @@
|
|||||||
|
<?xml version="1.0" encoding="utf-8"?>
|
||||||
|
<yandex>
|
||||||
|
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||||
|
<logger>
|
||||||
|
<level>trace</level>
|
||||||
|
<log>/var/log/clickhouse-server/log.log</log>
|
||||||
|
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||||
|
<size>1000M</size>
|
||||||
|
<count>10</count>
|
||||||
|
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||||
|
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||||
|
</logger>
|
||||||
|
</yandex>
|
248
tests/integration/test_keeper_auth/test.py
Normal file
248
tests/integration/test_keeper_auth/test.py
Normal file
@ -0,0 +1,248 @@
|
|||||||
|
#!/usr/bin/env python3
|
||||||
|
import pytest
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
from kazoo.client import KazooClient, KazooState
|
||||||
|
from kazoo.security import ACL, make_digest_acl, make_acl
|
||||||
|
from kazoo.exceptions import AuthFailedError, InvalidACLError, NoAuthError, KazooException
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
node = cluster.add_instance('node', main_configs=['configs/keeper_config.xml', 'configs/logs_conf.xml'], with_zookeeper=True, use_keeper=False)
|
||||||
|
|
||||||
|
SUPERAUTH = "super:admin"
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
def get_fake_zk(timeout=30.0):
|
||||||
|
_fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip('node') + ":9181", timeout=timeout)
|
||||||
|
_fake_zk_instance.start()
|
||||||
|
return _fake_zk_instance
|
||||||
|
|
||||||
|
def get_genuine_zk():
|
||||||
|
print("Zoo1", cluster.get_instance_ip("zoo1"))
|
||||||
|
return cluster.get_kazoo_client('zoo1')
|
||||||
|
|
||||||
|
@pytest.mark.parametrize(
|
||||||
|
('get_zk'),
|
||||||
|
[
|
||||||
|
get_genuine_zk,
|
||||||
|
get_fake_zk
|
||||||
|
]
|
||||||
|
)
|
||||||
|
def test_digest_auth_basic(started_cluster, get_zk):
|
||||||
|
auth_connection = get_zk()
|
||||||
|
|
||||||
|
auth_connection.add_auth('digest', 'user1:password1')
|
||||||
|
|
||||||
|
auth_connection.create("/test_no_acl", b"")
|
||||||
|
auth_connection.create("/test_all_acl", b"data", acl=[make_acl("auth", "", all=True)])
|
||||||
|
# for some reason original zookeeper accepts this ACL, but doesn't allow to do anything with this node
|
||||||
|
# even with correct credentials.
|
||||||
|
auth_connection.create("/test_all_digest_acl", b"dataX", acl=[make_acl("digest", "user1:password1", all=True)])
|
||||||
|
|
||||||
|
assert auth_connection.get("/test_all_acl")[0] == b"data"
|
||||||
|
#assert auth_connection.get("/test_all_digest_acl")[0] == b"dataX"
|
||||||
|
|
||||||
|
no_auth_connection = get_zk()
|
||||||
|
no_auth_connection.set("/test_no_acl", b"hello")
|
||||||
|
|
||||||
|
# no ACL, so cannot access these nodes
|
||||||
|
assert no_auth_connection.get("/test_no_acl")[0] == b"hello"
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
no_auth_connection.set("/test_all_acl", b"hello")
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
no_auth_connection.get("/test_all_acl")
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
no_auth_connection.get("/test_all_digest_acl")
|
||||||
|
|
||||||
|
# still doesn't help
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
no_auth_connection.add_auth("world", "anyone")
|
||||||
|
|
||||||
|
# session became broken, reconnect
|
||||||
|
no_auth_connection = get_zk()
|
||||||
|
|
||||||
|
# wrong auth
|
||||||
|
no_auth_connection.add_auth('digest', 'user2:password2')
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
no_auth_connection.set("/test_all_acl", b"hello")
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
no_auth_connection.set("/test_all_acl", b"hello")
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
no_auth_connection.get("/test_all_acl")
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
no_auth_connection.get("/test_all_digest_acl")
|
||||||
|
|
||||||
|
# but can access some non restricted nodes
|
||||||
|
no_auth_connection.create("/some_allowed_node", b"data")
|
||||||
|
|
||||||
|
# auth added, go on
|
||||||
|
no_auth_connection.add_auth('digest', 'user1:password1')
|
||||||
|
for path in ["/test_no_acl", "/test_all_acl"]:
|
||||||
|
no_auth_connection.set(path, b"auth_added")
|
||||||
|
assert no_auth_connection.get(path)[0] == b"auth_added"
|
||||||
|
|
||||||
|
|
||||||
|
def test_super_auth(started_cluster):
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
|
||||||
|
auth_connection.add_auth('digest', 'user1:password1')
|
||||||
|
|
||||||
|
auth_connection.create("/test_super_no_acl", b"")
|
||||||
|
auth_connection.create("/test_super_all_acl", b"data", acl=[make_acl("auth", "", all=True)])
|
||||||
|
|
||||||
|
super_connection = get_fake_zk()
|
||||||
|
super_connection.add_auth('digest', 'super:admin')
|
||||||
|
|
||||||
|
for path in ["/test_super_no_acl", "/test_super_all_acl"]:
|
||||||
|
super_connection.set(path, b"value")
|
||||||
|
assert super_connection.get(path)[0] == b"value"
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.mark.parametrize(
|
||||||
|
('get_zk'),
|
||||||
|
[
|
||||||
|
get_genuine_zk,
|
||||||
|
get_fake_zk
|
||||||
|
]
|
||||||
|
)
|
||||||
|
def test_digest_auth_multiple(started_cluster, get_zk):
|
||||||
|
auth_connection = get_zk()
|
||||||
|
auth_connection.add_auth('digest', 'user1:password1')
|
||||||
|
auth_connection.add_auth('digest', 'user2:password2')
|
||||||
|
auth_connection.add_auth('digest', 'user3:password3')
|
||||||
|
|
||||||
|
auth_connection.create("/test_multi_all_acl", b"data", acl=[make_acl("auth", "", all=True)])
|
||||||
|
|
||||||
|
one_auth_connection = get_zk()
|
||||||
|
one_auth_connection.add_auth('digest', 'user1:password1')
|
||||||
|
|
||||||
|
one_auth_connection.set("/test_multi_all_acl", b"X")
|
||||||
|
assert one_auth_connection.get("/test_multi_all_acl")[0] == b"X"
|
||||||
|
|
||||||
|
other_auth_connection = get_zk()
|
||||||
|
other_auth_connection.add_auth('digest', 'user2:password2')
|
||||||
|
|
||||||
|
other_auth_connection.set("/test_multi_all_acl", b"Y")
|
||||||
|
|
||||||
|
assert other_auth_connection.get("/test_multi_all_acl")[0] == b"Y"
|
||||||
|
|
||||||
|
@pytest.mark.parametrize(
|
||||||
|
('get_zk'),
|
||||||
|
[
|
||||||
|
get_genuine_zk,
|
||||||
|
get_fake_zk
|
||||||
|
]
|
||||||
|
)
|
||||||
|
def test_partial_auth(started_cluster, get_zk):
|
||||||
|
auth_connection = get_zk()
|
||||||
|
auth_connection.add_auth('digest', 'user1:password1')
|
||||||
|
|
||||||
|
auth_connection.create("/test_partial_acl", b"data", acl=[make_acl("auth", "", read=False, write=True, create=True, delete=True, admin=True)])
|
||||||
|
|
||||||
|
auth_connection.set("/test_partial_acl", b"X")
|
||||||
|
auth_connection.create("/test_partial_acl/subnode", b"X", acl=[make_acl("auth", "", read=False, write=True, create=True, delete=True, admin=True)])
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
auth_connection.get("/test_partial_acl")
|
||||||
|
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
auth_connection.get_children("/test_partial_acl")
|
||||||
|
|
||||||
|
# exists works without read perm
|
||||||
|
assert auth_connection.exists("/test_partial_acl") is not None
|
||||||
|
|
||||||
|
auth_connection.create("/test_partial_acl_create", b"data", acl=[make_acl("auth", "", read=True, write=True, create=False, delete=True, admin=True)])
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
auth_connection.create("/test_partial_acl_create/subnode")
|
||||||
|
|
||||||
|
auth_connection.create("/test_partial_acl_set", b"data", acl=[make_acl("auth", "", read=True, write=False, create=True, delete=True, admin=True)])
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
auth_connection.set("/test_partial_acl_set", b"X")
|
||||||
|
|
||||||
|
# not allowed to delete child node
|
||||||
|
auth_connection.create("/test_partial_acl_delete", b"data", acl=[make_acl("auth", "", read=True, write=True, create=True, delete=False, admin=True)])
|
||||||
|
auth_connection.create("/test_partial_acl_delete/subnode")
|
||||||
|
with pytest.raises(NoAuthError):
|
||||||
|
auth_connection.delete("/test_partial_acl_delete/subnode")
|
||||||
|
|
||||||
|
|
||||||
|
def test_bad_auth(started_cluster):
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
auth_connection.add_auth('world', 'anyone')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
print("Sending 1")
|
||||||
|
auth_connection.add_auth('adssagf', 'user1:password1')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
print("Sending 2")
|
||||||
|
auth_connection.add_auth('digest', '')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
print("Sending 3")
|
||||||
|
auth_connection.add_auth('', 'user1:password1')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
print("Sending 4")
|
||||||
|
auth_connection.add_auth('digest', 'user1')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
print("Sending 5")
|
||||||
|
auth_connection.add_auth('digest', 'user1:password:otherpassword')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
print("Sending 6")
|
||||||
|
auth_connection.add_auth('auth', 'user1:password')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(AuthFailedError):
|
||||||
|
print("Sending 7")
|
||||||
|
auth_connection.add_auth('world', 'somebody')
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(InvalidACLError):
|
||||||
|
print("Sending 8")
|
||||||
|
auth_connection.create("/test_bad_acl", b"data", acl=[make_acl("dasd", "", read=True, write=False, create=True, delete=True, admin=True)])
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(InvalidACLError):
|
||||||
|
print("Sending 9")
|
||||||
|
auth_connection.create("/test_bad_acl", b"data", acl=[make_acl("digest", "", read=True, write=False, create=True, delete=True, admin=True)])
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(InvalidACLError):
|
||||||
|
print("Sending 10")
|
||||||
|
auth_connection.create("/test_bad_acl", b"data", acl=[make_acl("", "", read=True, write=False, create=True, delete=True, admin=True)])
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(InvalidACLError):
|
||||||
|
print("Sending 11")
|
||||||
|
auth_connection.create("/test_bad_acl", b"data", acl=[make_acl("digest", "dsdasda", read=True, write=False, create=True, delete=True, admin=True)])
|
||||||
|
|
||||||
|
auth_connection = get_fake_zk()
|
||||||
|
with pytest.raises(InvalidACLError):
|
||||||
|
print("Sending 12")
|
||||||
|
auth_connection.create("/test_bad_acl", b"data", acl=[make_acl("digest", "dsad:DSAa:d", read=True, write=False, create=True, delete=True, admin=True)])
|
@ -7,7 +7,7 @@ import time
|
|||||||
from multiprocessing.dummy import Pool
|
from multiprocessing.dummy import Pool
|
||||||
|
|
||||||
cluster = ClickHouseCluster(__file__)
|
cluster = ClickHouseCluster(__file__)
|
||||||
node = cluster.add_instance('node', main_configs=['configs/enable_keeper.xml', 'configs/logs_conf.xml'], with_zookeeper=True)
|
node = cluster.add_instance('node', main_configs=['configs/enable_keeper.xml', 'configs/logs_conf.xml'], with_zookeeper=True, use_keeper=False)
|
||||||
from kazoo.client import KazooClient, KazooState, KeeperState
|
from kazoo.client import KazooClient, KazooState, KeeperState
|
||||||
|
|
||||||
def get_genuine_zk():
|
def get_genuine_zk():
|
||||||
|
@ -163,12 +163,12 @@ def test_secure_connection():
|
|||||||
"configs_secure/conf.d/remote_servers.xml",
|
"configs_secure/conf.d/remote_servers.xml",
|
||||||
"configs_secure/conf.d/ssl_conf.xml"],
|
"configs_secure/conf.d/ssl_conf.xml"],
|
||||||
with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name,
|
with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name,
|
||||||
zookeeper_use_tmpfs=False)
|
zookeeper_use_tmpfs=False, use_keeper=False)
|
||||||
node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key",
|
node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key",
|
||||||
"configs_secure/conf.d/remote_servers.xml",
|
"configs_secure/conf.d/remote_servers.xml",
|
||||||
"configs_secure/conf.d/ssl_conf.xml"],
|
"configs_secure/conf.d/ssl_conf.xml"],
|
||||||
with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name,
|
with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name,
|
||||||
zookeeper_use_tmpfs=False)
|
zookeeper_use_tmpfs=False, use_keeper=False)
|
||||||
|
|
||||||
try:
|
try:
|
||||||
cluster.start()
|
cluster.start()
|
||||||
|
Loading…
Reference in New Issue
Block a user