Define auth for sessions with preprocess

This commit is contained in:
Antonio Andelic 2022-05-10 12:53:18 +00:00
parent f8d0aa4bc1
commit 02319f92a3
2 changed files with 98 additions and 105 deletions

View File

@ -55,33 +55,6 @@ String generateDigest(const String & userdata)
return user_password[0] + ":" + base64Encode(getSHA1(userdata));
}
bool checkACL(int32_t permission, const Coordination::ACLs & node_acls, const std::vector<KeeperStorage::AuthID> & session_auths)
{
if (node_acls.empty())
return true;
for (const auto & session_auth : session_auths)
if (session_auth.scheme == "super")
return true;
for (const auto & node_acl : node_acls)
{
if (node_acl.permissions & permission)
{
if (node_acl.scheme == "world" && node_acl.id == "anyone")
return true;
for (const auto & session_auth : session_auths)
{
if (node_acl.scheme == session_auth.scheme && node_acl.id == session_auth.id)
return true;
}
}
}
return false;
}
bool fixupACL(
const std::vector<Coordination::ACL> & request_acls,
const std::vector<KeeperStorage::AuthID> & current_ids,
@ -384,6 +357,11 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid, int64_t session_i
finish_subdelta = true;
return Coordination::Error::ZOK;
}
else if constexpr (std::same_as<DeltaType, KeeperStorage::AddAuthDelta>)
{
session_and_auth[operation.session_id].emplace_back(std::move(operation.auth_id));
return Coordination::Error::ZOK;
}
else
{
// shouldn't be called in any process functions
@ -531,7 +509,7 @@ struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProc
namespace
{
Coordination::ACLs getACLs(KeeperStorage & storage, StringRef path, bool is_local)
Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_local)
{
if (is_local)
{
@ -546,6 +524,33 @@ namespace
}
}
bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local)
{
const auto node_acls = getNodeACLs(*this, path, is_local);
if (node_acls.empty())
return true;
if (current_nodes.hasAcl(session_id, is_local, [](const auto & auth_id) { return auth_id.scheme == "super"; }))
return true;
for (const auto & node_acl : node_acls)
{
if (node_acl.permissions & permission)
{
if (node_acl.scheme == "world" && node_acl.id == "anyone")
return true;
if (current_nodes.hasAcl(
session_id,
is_local,
[&](const auto & auth_id) { return auth_id.scheme == node_acl.scheme && auth_id.id == node_acl.id; }))
return true;
}
}
return false;
}
struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor
{
@ -560,14 +565,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
auto path = zk_request->getPath();
auto parent_path = parentPath(path);
const auto node_acls = getACLs(storage, parent_path, is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
return checkACL(Coordination::ACL::Create, node_acls, session_auths);
return storage.checkACL(parentPath(path), Coordination::ACL::Create, session_id, is_local);
}
std::vector<KeeperStorage::Delta> preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time) const override
@ -674,12 +672,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
const auto node_acls = getACLs(storage, zk_request->getPath(), is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
return checkACL(Coordination::ACL::Read, node_acls, session_auths);
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
@ -751,12 +744,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
const auto node_acls = getACLs(storage, parentPath(zk_request->getPath()), is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
return checkACL(Coordination::ACL::Delete, node_acls, session_auths);
return storage.checkACL(parentPath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
@ -898,12 +886,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
const auto node_acls = getACLs(storage, parentPath(zk_request->getPath()), is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
return checkACL(Coordination::ACL::Write, node_acls, session_auths);
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Write, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
@ -977,12 +960,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
const auto node_acls = getACLs(storage, parentPath(zk_request->getPath()), is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
return checkACL(Coordination::ACL::Read, node_acls, session_auths);
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
@ -1062,12 +1040,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
const auto node_acls = getACLs(storage, parentPath(zk_request->getPath()), is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
return checkACL(Coordination::ACL::Read, node_acls, session_auths);
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
@ -1144,12 +1117,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
const auto node_acls = getACLs(storage, parentPath(zk_request->getPath()), is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
return checkACL(Coordination::ACL::Admin, node_acls, session_auths);
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
@ -1205,14 +1173,9 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
const auto node_acls = getACLs(storage, parentPath(zk_request->getPath()), is_local);
if (node_acls.empty())
return true;
const auto & session_auths = storage.session_and_auth[session_id];
/// LOL, GetACL require more permissions, then SetACL...
return checkACL(Coordination::ACL::Admin | Coordination::ACL::Read, node_acls, session_auths);
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
std::vector<KeeperStorage::Delta>
@ -1450,35 +1413,39 @@ struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestPro
struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor
{
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
Coordination::ZooKeeperResponsePtr
process(KeeperStorage & storage, int64_t /*zxid*/, int64_t session_id, int64_t /* time */) const override
std::vector<KeeperStorage::Delta> preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/) const override
{
Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast<Coordination::ZooKeeperAuthRequest &>(*zk_request);
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast<Coordination::ZooKeeperAuthResponse &>(*response_ptr);
auto & sessions_and_auth = storage.session_and_auth;
if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1)
return {{zxid, Coordination::Error::ZAUTHFAILED}};
std::vector<KeeperStorage::Delta> new_deltas;
auto digest = generateDigest(auth_request.data);
if (digest == storage.superdigest)
{
auth_response.error = Coordination::Error::ZAUTHFAILED;
KeeperStorage::AuthID auth{"super", ""};
new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(auth)});
}
else
{
auto digest = generateDigest(auth_request.data);
if (digest == storage.superdigest)
{
KeeperStorage::AuthID auth{"super", ""};
sessions_and_auth[session_id].emplace_back(auth);
}
else
{
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())
session_ids.emplace_back(auth);
}
KeeperStorage::AuthID new_auth{auth_request.scheme, digest};
if (storage.current_nodes.hasAcl(session_id, false, [&](const auto & auth_id) { return new_auth == auth_id; }))
new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(new_auth)});
}
return new_deltas;
}
Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /* time */) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast<Coordination::ZooKeeperAuthResponse &>(*response_ptr);
if (const auto result = storage.commit(zxid, session_id); result != Coordination::Error::ZOK)
auth_response.error = result;
return response_ptr;
}
};
@ -1566,7 +1533,6 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory()
void KeeperStorage::preprocessRequest(
const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, int64_t time, int64_t new_last_zxid, bool check_acl)
{
current_nodes.current_zxid = new_last_zxid;
KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request);
if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special

View File

@ -140,8 +140,14 @@ public:
{
};
struct AddAuthDelta
{
int64_t session_id;
AuthID auth_id;
};
using Operation
= std::variant<CreateNodeDelta, RemoveNodeDelta, UpdateNodeDelta, SetACLDelta, ErrorDelta, SubDeltaEnd, FailedMultiDelta>;
= std::variant<CreateNodeDelta, RemoveNodeDelta, UpdateNodeDelta, SetACLDelta, AddAuthDelta, ErrorDelta, SubDeltaEnd, FailedMultiDelta>;
struct Delta
{
@ -149,9 +155,7 @@ public:
Delta(int64_t zxid_, Coordination::Error error) : Delta("", zxid_, ErrorDelta{error}) { }
Delta(int64_t zxid_, SubDeltaEnd subdelta) : Delta("", zxid_, subdelta) { }
Delta(int64_t zxid_, FailedMultiDelta failed_multi) : Delta("", zxid_, failed_multi) { }
Delta(int64_t zxid_, Operation subdelta) : Delta("", zxid_, subdelta) { }
String path;
int64_t zxid;
@ -167,19 +171,40 @@ public:
{
for (const auto & delta : deltas)
{
if (delta.path == path)
if (path.empty() || delta.path == path)
std::visit(visitor, delta.operation);
}
}
template <typename Predicate>
bool hasAcl(int64_t session_id, bool is_local, Predicate predicate)
{
for (const auto & session_auth : storage.session_and_auth[session_id])
{
if (predicate(session_auth))
return true;
}
if (is_local)
return false;
for (const auto & delta : deltas)
{
if (auto * auth_delta = std::get_if<KeeperStorage::AddAuthDelta>(&delta.operation);
auth_delta && auth_delta->session_id == session_id && predicate(auth_delta->auth_id))
return true;
}
return false;
}
std::shared_ptr<Node> getNode(StringRef path);
bool hasNode(StringRef path) const;
Coordination::ACLs getACLs(StringRef path) const;
std::unordered_map<std::string, std::deque<Delta>> node_to_deltas;
std::deque<Delta> deltas;
KeeperStorage & storage;
int64_t current_zxid{0};
};
CurrentNodes current_nodes{*this};
@ -196,6 +221,8 @@ public:
int64_t session_id);
bool removeNode(const std::string & path, int32_t version);
bool checkACL(StringRef path, int32_t permissions, int64_t session_id, bool is_local);
/// Mapping session_id -> set of ephemeral nodes paths
Ephemerals ephemerals;
/// Mapping session_id -> set of watched nodes paths