Correctly fix close session and auth

This commit is contained in:
Antonio Andelic 2024-06-27 09:35:26 +02:00
parent 7b387d984e
commit 8d5f77a6fa
2 changed files with 153 additions and 50 deletions

View File

@ -534,6 +534,10 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool is_local,
if (is_local)
return check_auth(storage.session_and_auth[session_id]);
/// we want to close the session and with that we will remove all the auth related to the session
if (closed_sessions.contains(session_id))
return false;
if (check_auth(storage.session_and_auth[session_id]))
return true;
@ -2389,8 +2393,7 @@ void KeeperStorage::preprocessRequest(
return;
}
if ((check_acl && !request_processor->checkAuth(*this, session_id, false)) ||
uncommitted_state.closed_sessions.contains(session_id)) // Is session closed but not committed yet
if (check_acl && !request_processor->checkAuth(*this, session_id, false))
{
uncommitted_state.deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH);
return;

View File

@ -2028,7 +2028,8 @@ TEST_P(CoordinationTest, TestPreprocessWhenCloseSessionIsPrecommitted)
setSnapshotDirectory("./snapshots");
ResponsesQueue queue(std::numeric_limits<size_t>::max());
SnapshotsQueue snapshots_queue{1};
int64_t session_id = 1;
int64_t session_without_auth = 1;
int64_t session_with_auth = 2;
size_t term = 0;
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, keeper_context, nullptr);
@ -2037,66 +2038,165 @@ TEST_P(CoordinationTest, TestPreprocessWhenCloseSessionIsPrecommitted)
auto & storage = state_machine->getStorageUnsafe();
const auto & uncommitted_state = storage.uncommitted_state;
// Create first node for the session
String node_path_1 = "/node_1";
std::shared_ptr<ZooKeeperCreateRequest> create_req_1 = std::make_shared<ZooKeeperCreateRequest>();
create_req_1->path = node_path_1;
auto create_entry_1 = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), create_req_1);
auto auth_req = std::make_shared<ZooKeeperAuthRequest>();
auth_req->scheme = "digest";
auth_req->data = "test_user:test_password";
state_machine->pre_commit(1, create_entry_1->get_buf());
EXPECT_TRUE(uncommitted_state.nodes.contains(node_path_1));
// Add auth data to the session
auto auth_entry = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), auth_req);
state_machine->pre_commit(1, auth_entry->get_buf());
state_machine->commit(1, auth_entry->get_buf());
state_machine->commit(1, create_entry_1->get_buf());
EXPECT_TRUE(storage.container.contains(node_path_1));
std::string node_without_acl = "/node_without_acl";
{
auto create_req = std::make_shared<ZooKeeperCreateRequest>();
create_req->path = node_without_acl;
create_req->data = "notmodified";
auto create_entry = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), create_req);
state_machine->pre_commit(2, create_entry->get_buf());
state_machine->commit(2, create_entry->get_buf());
ASSERT_TRUE(storage.container.contains(node_without_acl));
}
// Close session
std::shared_ptr<ZooKeeperCloseRequest> close_req = std::make_shared<ZooKeeperCloseRequest>();
auto close_entry = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), close_req);
// Pre-commit close session
state_machine->pre_commit(2, close_entry->get_buf());
std::string node_with_acl = "/node_with_acl";
{
auto create_req = std::make_shared<ZooKeeperCreateRequest>();
create_req->path = node_with_acl;
create_req->data = "notmodified";
create_req->acls = {{.permissions = ACL::All, .scheme = "auth", .id = ""}};
auto create_entry = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), create_req);
state_machine->pre_commit(3, create_entry->get_buf());
state_machine->commit(3, create_entry->get_buf());
ASSERT_TRUE(storage.container.contains(node_with_acl));
}
// Try to create second node after close session is pre-committed
String node_path_2 = "/node_2";
std::shared_ptr<ZooKeeperCreateRequest> create_req_2 = std::make_shared<ZooKeeperCreateRequest>();
create_req_2->path = node_path_2;
auto create_entry_2 = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), create_req_2);
auto set_req_with_acl = std::make_shared<ZooKeeperSetRequest>();
set_req_with_acl->path = node_with_acl;
set_req_with_acl->data = "modified";
// Pre-commit creating second node
state_machine->pre_commit(3, create_entry_2->get_buf());
// Second node wasn't created
EXPECT_FALSE(uncommitted_state.nodes.contains(node_path_2));
auto set_req_without_acl = std::make_shared<ZooKeeperSetRequest>();
set_req_without_acl->path = node_without_acl;
set_req_without_acl->data = "modified";
// Rollback pre-committed closing session
state_machine->rollback(3, create_entry_2->get_buf());
state_machine->rollback(2, close_entry->get_buf());
const auto reset_node_value
= [&](const auto & path) { storage.container.updateValue(path, [](auto & node) { node.setData("notmodified"); }); };
// Pre-commit creating second node
state_machine->pre_commit(2, create_entry_2->get_buf());
// Now second node was created
EXPECT_TRUE(uncommitted_state.nodes.contains(node_path_2));
auto close_req = std::make_shared<ZooKeeperCloseRequest>();
state_machine->commit(2, create_entry_2->get_buf());
EXPECT_TRUE(storage.container.contains(node_path_1));
EXPECT_TRUE(storage.container.contains(node_path_2));
{
SCOPED_TRACE("Session with Auth");
close_entry = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), close_req);
// Pre-commit close session
state_machine->pre_commit(3, close_entry->get_buf());
// test we can modify both nodes
auto set_entry = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), set_req_with_acl);
state_machine->pre_commit(5, set_entry->get_buf());
state_machine->commit(5, set_entry->get_buf());
ASSERT_TRUE(storage.container.find(node_with_acl)->value.getData() == "modified");
reset_node_value(node_with_acl);
auto heartbeat_entry
= getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), std::make_shared<ZooKeeperHeartbeatRequest>());
state_machine->pre_commit(4, heartbeat_entry->get_buf());
set_entry = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), set_req_without_acl);
state_machine->pre_commit(6, set_entry->get_buf());
state_machine->commit(6, set_entry->get_buf());
ASSERT_TRUE(storage.container.find(node_without_acl)->value.getData() == "modified");
reset_node_value(node_without_acl);
state_machine->commit(3, close_entry->get_buf());
state_machine->commit(4, heartbeat_entry->get_buf());
auto close_entry = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), close_req);
auto create_req_3 = std::make_shared<ZooKeeperCreateRequest>();
create_req_3->path = "/node_3";
auto create_entry_3 = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), create_req_3);
state_machine->pre_commit(5, create_entry_3->get_buf());
state_machine->commit(5, create_entry_3->get_buf());
// Pre-commit close session
state_machine->pre_commit(7, close_entry->get_buf());
EXPECT_TRUE(storage.container.contains("/node_3"));
/// will be rejected because we don't have required auth
auto set_entry_with_acl = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), set_req_with_acl);
state_machine->pre_commit(8, set_entry_with_acl->get_buf());
/// will be accepted because no ACL
auto set_entry_without_acl = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), set_req_without_acl);
state_machine->pre_commit(9, set_entry_without_acl->get_buf());
ASSERT_TRUE(uncommitted_state.getNode(node_with_acl)->getData() == "notmodified");
ASSERT_TRUE(uncommitted_state.getNode(node_without_acl)->getData() == "modified");
state_machine->rollback(9, set_entry_without_acl->get_buf());
state_machine->rollback(8, set_entry_with_acl->get_buf());
// let's commit close and verify we get same outcome
state_machine->commit(7, close_entry->get_buf());
/// will be rejected because we don't have required auth
set_entry_with_acl = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), set_req_with_acl);
state_machine->pre_commit(8, set_entry_with_acl->get_buf());
/// will be accepted because no ACL
set_entry_without_acl = getLogEntryFromZKRequest(term, session_with_auth, state_machine->getNextZxid(), set_req_without_acl);
state_machine->pre_commit(9, set_entry_without_acl->get_buf());
ASSERT_TRUE(uncommitted_state.getNode(node_with_acl)->getData() == "notmodified");
ASSERT_TRUE(uncommitted_state.getNode(node_without_acl)->getData() == "modified");
state_machine->commit(8, set_entry_with_acl->get_buf());
state_machine->commit(9, set_entry_without_acl->get_buf());
ASSERT_TRUE(storage.container.find(node_with_acl)->value.getData() == "notmodified");
ASSERT_TRUE(storage.container.find(node_without_acl)->value.getData() == "modified");
reset_node_value(node_without_acl);
}
{
SCOPED_TRACE("Session without Auth");
// test we can modify only node without acl
auto set_entry = getLogEntryFromZKRequest(term, session_without_auth, state_machine->getNextZxid(), set_req_with_acl);
state_machine->pre_commit(10, set_entry->get_buf());
state_machine->commit(10, set_entry->get_buf());
ASSERT_TRUE(storage.container.find(node_with_acl)->value.getData() == "notmodified");
set_entry = getLogEntryFromZKRequest(term, session_without_auth, state_machine->getNextZxid(), set_req_without_acl);
state_machine->pre_commit(11, set_entry->get_buf());
state_machine->commit(11, set_entry->get_buf());
ASSERT_TRUE(storage.container.find(node_without_acl)->value.getData() == "modified");
reset_node_value(node_without_acl);
auto close_entry = getLogEntryFromZKRequest(term, session_without_auth, state_machine->getNextZxid(), close_req);
// Pre-commit close session
state_machine->pre_commit(12, close_entry->get_buf());
/// will be rejected because we don't have required auth
auto set_entry_with_acl = getLogEntryFromZKRequest(term, session_without_auth, state_machine->getNextZxid(), set_req_with_acl);
state_machine->pre_commit(13, set_entry_with_acl->get_buf());
/// will be accepted because no ACL
auto set_entry_without_acl = getLogEntryFromZKRequest(term, session_without_auth, state_machine->getNextZxid(), set_req_without_acl);
state_machine->pre_commit(14, set_entry_without_acl->get_buf());
ASSERT_TRUE(uncommitted_state.getNode(node_with_acl)->getData() == "notmodified");
ASSERT_TRUE(uncommitted_state.getNode(node_without_acl)->getData() == "modified");
state_machine->rollback(14, set_entry_without_acl->get_buf());
state_machine->rollback(13, set_entry_with_acl->get_buf());
// let's commit close and verify we get same outcome
state_machine->commit(12, close_entry->get_buf());
/// will be rejected because we don't have required auth
set_entry_with_acl = getLogEntryFromZKRequest(term, session_without_auth, state_machine->getNextZxid(), set_req_with_acl);
state_machine->pre_commit(13, set_entry_with_acl->get_buf());
/// will be accepted because no ACL
set_entry_without_acl = getLogEntryFromZKRequest(term, session_without_auth, state_machine->getNextZxid(), set_req_without_acl);
state_machine->pre_commit(14, set_entry_without_acl->get_buf());
ASSERT_TRUE(uncommitted_state.getNode(node_with_acl)->getData() == "notmodified");
ASSERT_TRUE(uncommitted_state.getNode(node_without_acl)->getData() == "modified");
state_machine->commit(13, set_entry_with_acl->get_buf());
state_machine->commit(14, set_entry_without_acl->get_buf());
ASSERT_TRUE(storage.container.find(node_with_acl)->value.getData() == "notmodified");
ASSERT_TRUE(storage.container.find(node_without_acl)->value.getData() == "modified");
reset_node_value(node_without_acl);
}
}
TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted)