Fix REPLICA timestamps (#1615)

* Fix up REPLICA GetInfo and CreateSnapshot

Subtle bug where these actions were using the incorrect transactional
access while in REPLICA role. This casued timestamp to be incorrectly
bumped, breaking REPLICA from doing replication.

* Delay DNS resolution

Rather than resolve at endpoint creation, we will instread resolve only
on Socket connect. This allows k8s deployments to change their IP during
pod restarts.

* Minor sonarsource fixes

---------
Co-authored-by: Andreja <andreja.tonev@memgraph.io>
Co-authored-by: DavIvek <david.ivekovic@memgraph.io>
This commit is contained in:
Gareth Andrew Lloyd 2024-01-05 16:42:54 +00:00 committed by GitHub
parent 7128e1cea8
commit 0fb8e4116f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
73 changed files with 1229 additions and 1121 deletions

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -25,7 +25,7 @@ namespace mg_exception {
template <typename FirstArg, typename... Args>
std::string StringSerialize(FirstArg &&firstArg, Args &&...args) {
std::stringstream stream;
stream << firstArg;
stream << std::forward<FirstArg>(firstArg);
((stream << " " << args), ...);
return stream.str();
}

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -12,6 +12,7 @@
#include "communication/client.hpp"
#include "communication/helpers.hpp"
#include "io/network/network_error.hpp"
#include "utils/logging.hpp"
namespace memgraph::communication {
@ -25,7 +26,9 @@ Client::~Client() {
bool Client::Connect(const io::network::Endpoint &endpoint) {
// Try to establish a socket connection.
if (!socket_.Connect(endpoint)) return false;
if (!socket_.Connect(endpoint)) {
return false;
}
// Enable TCP keep alive for all connections.
// Because we manually always set the `have_more` flag to the socket

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -107,7 +107,7 @@ class Session : public std::enable_shared_from_this<Session<TRequestHandler, TSe
void DoRead() {
req_ = {};
ExecuteForStream([this](auto &&stream) {
ExecuteForStream([this](auto &stream) {
boost::beast::get_lowest_layer(stream).expires_after(std::chrono::seconds(kSSLExpirySeconds));
boost::beast::http::async_read(
@ -129,7 +129,7 @@ class Session : public std::enable_shared_from_this<Session<TRequestHandler, TSe
}
auto async_write = [this](boost::beast::http::response<boost::beast::http::string_body> msg) {
ExecuteForStream([this, &msg](auto &&stream) {
ExecuteForStream([this, &msg](auto &stream) {
// The lifetime of the message has to extend
// for the duration of the async operation so
// we use a shared_ptr to manage it.
@ -171,7 +171,7 @@ class Session : public std::enable_shared_from_this<Session<TRequestHandler, TSe
}
auto GetExecutor() {
return std::visit(utils::Overloaded{[](auto &&stream) { return stream.get_executor(); }}, stream_);
return std::visit(utils::Overloaded{[](auto &stream) { return stream.get_executor(); }}, stream_);
}
template <typename F>

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -68,12 +68,12 @@ class Database {
*/
std::unique_ptr<storage::Storage::Accessor> Access(
std::optional<storage::IsolationLevel> override_isolation_level = {}) {
return storage_->Access(override_isolation_level, repl_state_->IsMain());
return storage_->Access(repl_state_->GetRole(), override_isolation_level);
}
std::unique_ptr<storage::Storage::Accessor> UniqueAccess(
std::optional<storage::IsolationLevel> override_isolation_level = {}) {
return storage_->UniqueAccess(override_isolation_level, repl_state_->IsMain());
return storage_->UniqueAccess(repl_state_->GetRole(), override_isolation_level);
}
/**
@ -103,9 +103,9 @@ class Database {
* @param force_directory Use the configured directory, do not try to decipher the multi-db version
* @return DatabaseInfo
*/
DatabaseInfo GetInfo(bool force_directory = false) const {
DatabaseInfo GetInfo(bool force_directory, replication::ReplicationRole replication_role) const {
DatabaseInfo info;
info.storage_info = storage_->GetInfo(force_directory);
info.storage_info = storage_->GetInfo(force_directory, replication_role);
info.triggers = trigger_store_.GetTriggerInfo().size();
info.streams = streams_.GetStreamInfo().size();
return info;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -232,6 +232,7 @@ class DbmsHandler {
* @return Statistics
*/
Statistics Stats() {
auto const replication_role = repl_state_.GetRole();
Statistics stats{};
// TODO: Handle overflow?
#ifdef MG_ENTERPRISE
@ -244,7 +245,7 @@ class DbmsHandler {
auto db_acc_opt = db_gk.access();
if (db_acc_opt) {
auto &db_acc = *db_acc_opt;
const auto &info = db_acc->GetInfo();
const auto &info = db_acc->GetInfo(false, replication_role);
const auto &storage_info = info.storage_info;
stats.num_vertex += storage_info.vertex_count;
stats.num_edges += storage_info.edge_count;
@ -268,6 +269,7 @@ class DbmsHandler {
* @return std::vector<DatabaseInfo>
*/
std::vector<DatabaseInfo> Info() {
auto const replication_role = repl_state_.GetRole();
std::vector<DatabaseInfo> res;
#ifdef MG_ENTERPRISE
std::shared_lock<LockT> rd(lock_);
@ -280,7 +282,7 @@ class DbmsHandler {
auto db_acc_opt = db_gk.access();
if (db_acc_opt) {
auto &db_acc = *db_acc_opt;
res.push_back(db_acc->GetInfo());
res.push_back(db_acc->GetInfo(false, replication_role));
}
}
return res;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -22,6 +22,7 @@
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/inmemory/unique_constraints.hpp"
using memgraph::replication::ReplicationRole;
using memgraph::storage::Delta;
using memgraph::storage::EdgeAccessor;
using memgraph::storage::EdgeRef;
@ -380,9 +381,9 @@ uint64_t InMemoryReplicationHandlers::ReadAndApplyDelta(storage::InMemoryStorage
if (!commit_timestamp_and_accessor) {
std::unique_ptr<storage::Storage::Accessor> acc = nullptr;
if (unique) {
acc = storage->UniqueAccess(std::nullopt, false /*not main*/);
acc = storage->UniqueAccess(ReplicationRole::REPLICA);
} else {
acc = storage->Access(std::nullopt, false /*not main*/);
acc = storage->Access(ReplicationRole::REPLICA);
}
auto inmem_acc = std::unique_ptr<storage::InMemoryStorage::InMemoryAccessor>(
static_cast<storage::InMemoryStorage::InMemoryAccessor *>(acc.release()));

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -31,10 +31,7 @@ inline std::unique_ptr<storage::Storage> CreateInMemoryStorage(storage::Config c
// Connect replication state and storage
storage->CreateSnapshotHandler(
[storage = storage.get(), &repl_state]() -> utils::BasicResult<storage::InMemoryStorage::CreateSnapshotError> {
if (repl_state.IsReplica()) {
return storage::InMemoryStorage::CreateSnapshotError::DisabledForReplica;
}
return storage->CreateSnapshot();
return storage->CreateSnapshot(repl_state.GetRole());
});
if (allow_mt_repl || name == dbms::kDefaultDB) {

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -139,18 +139,13 @@ Endpoint::Endpoint(std::string ip_address, uint16_t port) : address(std::move(ip
}
// NOLINTNEXTLINE
Endpoint::Endpoint(needs_resolving_t, std::string hostname, uint16_t port) : port(port) {
address = ResolveHostnameIntoIpAddress(hostname, port);
IpFamily ip_family = GetIpFamily(address);
if (ip_family == IpFamily::NONE) {
throw NetworkError("Not a valid IPv4 or IPv6 address: {}", address);
}
family = ip_family;
}
Endpoint::Endpoint(needs_resolving_t, std::string hostname, uint16_t port)
: address(std::move(hostname)), port(port), family{GetIpFamily(address)} {}
std::ostream &operator<<(std::ostream &os, const Endpoint &endpoint) {
// no need to cover the IpFamily::NONE case, as you can't even construct an
// Endpoint object if the IpFamily is NONE (i.e. the IP address is invalid)
// unless you use DNS hostname
if (endpoint.family == Endpoint::IpFamily::IP6) {
return os << "[" << endpoint.address << "]"
<< ":" << endpoint.port;
@ -166,7 +161,7 @@ bool Endpoint::IsResolvableAddress(const std::string &address, uint16_t port) {
};
addrinfo *info = nullptr;
auto status = getaddrinfo(address.c_str(), std::to_string(port).c_str(), &hints, &info);
freeaddrinfo(info);
if (info) freeaddrinfo(info);
return status == 0;
}
@ -189,34 +184,4 @@ std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrAddress(
return std::nullopt;
}
std::string Endpoint::ResolveHostnameIntoIpAddress(const std::string &address, uint16_t port) {
addrinfo hints{
.ai_flags = AI_PASSIVE,
.ai_family = AF_UNSPEC, // IPv4 and IPv6
.ai_socktype = SOCK_STREAM // TCP socket
};
addrinfo *info = nullptr;
auto status = getaddrinfo(address.c_str(), std::to_string(port).c_str(), &hints, &info);
if (status != 0) throw NetworkError(gai_strerror(status));
for (auto *result = info; result != nullptr; result = result->ai_next) {
if (result->ai_family == AF_INET) {
char ipstr[INET_ADDRSTRLEN];
auto *ipv4 = reinterpret_cast<struct sockaddr_in *>(result->ai_addr);
inet_ntop(AF_INET, &(ipv4->sin_addr), ipstr, sizeof(ipstr));
freeaddrinfo(info);
return ipstr;
}
if (result->ai_family == AF_INET6) {
char ipstr[INET6_ADDRSTRLEN];
auto *ipv6 = reinterpret_cast<struct sockaddr_in6 *>(result->ai_addr);
inet_ntop(AF_INET6, &(ipv6->sin6_addr), ipstr, sizeof(ipstr));
freeaddrinfo(info);
return ipstr;
}
}
freeaddrinfo(info);
throw NetworkError("Not a valid address: {}", address);
}
} // namespace memgraph::io::network

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -15,6 +15,7 @@
#include <poll.h>
#include "io/network/addrinfo.hpp"
#include "io/network/network_error.hpp"
#include "io/network/socket.hpp"
#include "utils/likely.hpp"
#include "utils/logging.hpp"
@ -55,17 +56,21 @@ bool Socket::IsOpen() const { return socket_ != -1; }
bool Socket::Connect(const Endpoint &endpoint) {
if (socket_ != -1) return false;
for (const auto &it : AddrInfo{endpoint}) {
int sfd = socket(it.ai_family, it.ai_socktype, it.ai_protocol);
if (sfd == -1) continue;
if (connect(sfd, it.ai_addr, it.ai_addrlen) == 0) {
socket_ = sfd;
endpoint_ = endpoint;
break;
try {
for (const auto &it : AddrInfo{endpoint}) {
int sfd = socket(it.ai_family, it.ai_socktype, it.ai_protocol);
if (sfd == -1) continue;
if (connect(sfd, it.ai_addr, it.ai_addrlen) == 0) {
socket_ = sfd;
endpoint_ = endpoint;
break;
}
// If the connect failed close the file descriptor to prevent file
// descriptors being leaked
close(sfd);
}
// If the connect failed close the file descriptor to prevent file
// descriptors being leaked
close(sfd);
} catch (const NetworkError &e) {
return false;
}
return !(socket_ == -1);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -32,6 +32,8 @@
#include "utils/timer.hpp"
#include "version.hpp"
using memgraph::replication::ReplicationRole;
bool ValidateControlCharacter(const char *flagname, const std::string &value) {
if (value.empty()) {
printf("The argument '%s' cannot be empty\n", flagname);
@ -425,7 +427,7 @@ void ProcessNodeRow(memgraph::storage::Storage *store, const std::vector<std::st
const std::vector<Field> &fields, const std::vector<std::string> &additional_labels,
std::unordered_map<NodeId, memgraph::storage::Gid> *node_id_map) {
std::optional<NodeId> id;
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto node = acc->CreateVertex();
for (size_t i = 0; i < row.size(); ++i) {
const auto &field = fields[i];
@ -571,7 +573,7 @@ void ProcessRelationshipsRow(memgraph::storage::Storage *store, const std::vecto
if (!end_id) throw LoadException("END_ID must be set");
if (!relationship_type) throw LoadException("Relationship TYPE must be set");
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto from_node = acc->FindVertex(*start_id, memgraph::storage::View::NEW);
if (!from_node) throw LoadException("From node must be in the storage");
auto to_node = acc->FindVertex(*end_id, memgraph::storage::View::NEW);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -2786,8 +2786,8 @@ PreparedQuery PrepareEdgeImportModeQuery(ParsedQuery parsed_query, CurrentDB &cu
RWType::NONE};
}
PreparedQuery PrepareCreateSnapshotQuery(ParsedQuery parsed_query, bool in_explicit_transaction,
CurrentDB &current_db) {
PreparedQuery PrepareCreateSnapshotQuery(ParsedQuery parsed_query, bool in_explicit_transaction, CurrentDB &current_db,
replication::ReplicationRole replication_role) {
if (in_explicit_transaction) {
throw CreateSnapshotInMulticommandTxException();
}
@ -2802,9 +2802,10 @@ PreparedQuery PrepareCreateSnapshotQuery(ParsedQuery parsed_query, bool in_expli
return PreparedQuery{
{},
std::move(parsed_query.required_privileges),
[storage](AnyStream * /*stream*/, std::optional<int> /*n*/) -> std::optional<QueryHandlerResult> {
[storage, replication_role](AnyStream * /*stream*/,
std::optional<int> /*n*/) -> std::optional<QueryHandlerResult> {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(storage);
if (auto maybe_error = mem_storage->CreateSnapshot(); maybe_error.HasError()) {
if (auto maybe_error = mem_storage->CreateSnapshot(replication_role); maybe_error.HasError()) {
switch (maybe_error.GetError()) {
case storage::InMemoryStorage::CreateSnapshotError::DisabledForReplica:
throw utils::BasicException(
@ -2977,15 +2978,16 @@ PreparedQuery PrepareDatabaseInfoQuery(ParsedQuery parsed_query, bool in_explici
auto *info_query = utils::Downcast<DatabaseInfoQuery>(parsed_query.query);
std::vector<std::string> header;
std::function<std::pair<std::vector<std::vector<TypedValue>>, QueryHandlerResult>()> handler;
auto *database = current_db.db_acc_->get();
switch (info_query->info_type_) {
case DatabaseInfoQuery::InfoType::INDEX: {
header = {"index type", "label", "property", "count"};
handler = [storage = current_db.db_acc_->get()->storage(), dba] {
handler = [database, dba] {
auto *storage = database->storage();
const std::string_view label_index_mark{"label"};
const std::string_view label_property_index_mark{"label+property"};
auto info = dba->ListAllIndices();
auto storage_acc = storage->Access();
auto storage_acc = database->Access();
std::vector<std::vector<TypedValue>> results;
results.reserve(info.label.size() + info.label_property.size());
for (const auto &item : info.label) {
@ -3823,7 +3825,9 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
} else if (utils::Downcast<IsolationLevelQuery>(parsed_query.query)) {
prepared_query = PrepareIsolationLevelQuery(std::move(parsed_query), in_explicit_transaction_, current_db_, this);
} else if (utils::Downcast<CreateSnapshotQuery>(parsed_query.query)) {
prepared_query = PrepareCreateSnapshotQuery(std::move(parsed_query), in_explicit_transaction_, current_db_);
auto const replication_role = interpreter_context_->repl_state->GetRole();
prepared_query =
PrepareCreateSnapshotQuery(std::move(parsed_query), in_explicit_transaction_, current_db_, replication_role);
} else if (utils::Downcast<SettingQuery>(parsed_query.query)) {
/// SYSTEM PURE
prepared_query = PrepareSettingQuery(std::move(parsed_query), in_explicit_transaction_);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -14,5 +14,6 @@
#include <cstdint>
namespace memgraph::replication {
// TODO: figure out a way of ensuring that usage of this type is never uninitialed/defaulted incorrectly to MAIN
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
}
} // namespace memgraph::replication

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -288,7 +288,7 @@ DiskStorage::~DiskStorage() {
DiskStorage::DiskAccessor::DiskAccessor(auto tag, DiskStorage *storage, IsolationLevel isolation_level,
StorageMode storage_mode)
: Accessor(tag, storage, isolation_level, storage_mode) {
: Accessor(tag, storage, isolation_level, storage_mode, memgraph::replication::ReplicationRole::MAIN) {
rocksdb::WriteOptions write_options;
auto txOptions = rocksdb::TransactionOptions{.set_snapshot = true};
transaction_.disk_transaction_ = storage->kvstore_->db_->BeginTransaction(write_options, txOptions);
@ -837,10 +837,10 @@ StorageInfo DiskStorage::GetBaseInfo(bool /* unused */) {
return info;
}
StorageInfo DiskStorage::GetInfo(bool force_dir) {
StorageInfo DiskStorage::GetInfo(bool force_dir, memgraph::replication::ReplicationRole replication_role) {
StorageInfo info = GetBaseInfo(force_dir);
{
auto access = Access(std::nullopt);
auto access = Access(replication_role);
const auto &lbl = access->ListAllIndices();
info.label_indices = lbl.label.size();
info.label_property_indices = lbl.label_property.size();
@ -2005,7 +2005,8 @@ UniqueConstraints::DeletionStatus DiskStorage::DiskAccessor::DropUniqueConstrain
return UniqueConstraints::DeletionStatus::SUCCESS;
}
Transaction DiskStorage::CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode, bool /*is_main*/) {
Transaction DiskStorage::CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode,
memgraph::replication::ReplicationRole /*is_main*/) {
/// We acquire the transaction engine lock here because we access (and
/// modify) the transaction engine variables (`transaction_id` and
/// `timestamp`) below.
@ -2030,8 +2031,8 @@ uint64_t DiskStorage::CommitTimestamp(const std::optional<uint64_t> desired_comm
return *desired_commit_timestamp;
}
std::unique_ptr<Storage::Accessor> DiskStorage::Access(std::optional<IsolationLevel> override_isolation_level,
bool /*is_main*/) {
std::unique_ptr<Storage::Accessor> DiskStorage::Access(memgraph::replication::ReplicationRole /*replication_role*/,
std::optional<IsolationLevel> override_isolation_level) {
auto isolation_level = override_isolation_level.value_or(isolation_level_);
if (isolation_level != IsolationLevel::SNAPSHOT_ISOLATION) {
throw utils::NotYetImplemented("Disk storage supports only SNAPSHOT isolation level.");
@ -2039,8 +2040,9 @@ std::unique_ptr<Storage::Accessor> DiskStorage::Access(std::optional<IsolationLe
return std::unique_ptr<DiskAccessor>(
new DiskAccessor{Storage::Accessor::shared_access, this, isolation_level, storage_mode_});
}
std::unique_ptr<Storage::Accessor> DiskStorage::UniqueAccess(std::optional<IsolationLevel> override_isolation_level,
bool /*is_main*/) {
std::unique_ptr<Storage::Accessor> DiskStorage::UniqueAccess(
memgraph::replication::ReplicationRole /*replication_role*/,
std::optional<IsolationLevel> override_isolation_level) {
auto isolation_level = override_isolation_level.value_or(isolation_level_);
if (isolation_level != IsolationLevel::SNAPSHOT_ISOLATION) {
throw utils::NotYetImplemented("Disk storage supports only SNAPSHOT isolation level.");

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -176,12 +176,12 @@ class DiskStorage final : public Storage {
};
using Storage::Access;
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level,
bool is_main) override;
std::unique_ptr<Accessor> Access(memgraph::replication::ReplicationRole replication_role,
std::optional<IsolationLevel> override_isolation_level) override;
using Storage::UniqueAccess;
std::unique_ptr<Storage::Accessor> UniqueAccess(std::optional<IsolationLevel> override_isolation_level,
bool is_main) override;
std::unique_ptr<Accessor> UniqueAccess(memgraph::replication::ReplicationRole replication_role,
std::optional<IsolationLevel> override_isolation_level) override;
/// Flushing methods
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> FlushIndexCache(Transaction *transaction);
@ -284,8 +284,8 @@ class DiskStorage final : public Storage {
RocksDBStorage *GetRocksDBStorage() const { return kvstore_.get(); }
using Storage::CreateTransaction;
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode, bool is_main) override;
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode,
memgraph::replication::ReplicationRole replication_role) override;
void SetEdgeImportMode(EdgeImportMode edge_import_status);
@ -308,7 +308,7 @@ class DiskStorage final : public Storage {
PropertyId property);
StorageInfo GetBaseInfo(bool force_directory) override;
StorageInfo GetInfo(bool force_directory) override;
StorageInfo GetInfo(bool force_directory, memgraph::replication::ReplicationRole replication_role) override;
void FreeMemory(std::unique_lock<utils::ResourceLock> /*lock*/) override {}

View File

@ -175,8 +175,9 @@ InMemoryStorage::~InMemoryStorage() {
}
InMemoryStorage::InMemoryAccessor::InMemoryAccessor(auto tag, InMemoryStorage *storage, IsolationLevel isolation_level,
StorageMode storage_mode, bool is_main)
: Accessor(tag, storage, isolation_level, storage_mode, is_main), config_(storage->config_.items) {}
StorageMode storage_mode,
memgraph::replication::ReplicationRole replication_role)
: Accessor(tag, storage, isolation_level, storage_mode, replication_role), config_(storage->config_.items) {}
InMemoryStorage::InMemoryAccessor::InMemoryAccessor(InMemoryAccessor &&other) noexcept
: Accessor(std::move(other)), config_(other.config_) {}
@ -1301,7 +1302,8 @@ VerticesIterable InMemoryStorage::InMemoryAccessor::Vertices(
mem_label_property_index->Vertices(label, property, lower_bound, upper_bound, view, storage_, &transaction_));
}
Transaction InMemoryStorage::CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode, bool is_main) {
Transaction InMemoryStorage::CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode,
memgraph::replication::ReplicationRole replication_role) {
// We acquire the transaction engine lock here because we access (and
// modify) the transaction engine variables (`transaction_id` and
// `timestamp`) below.
@ -1316,7 +1318,7 @@ Transaction InMemoryStorage::CreateTransaction(IsolationLevel isolation_level, S
// of any query on replica to the last commited transaction
// which is timestamp_ as only commit of transaction with writes
// can change the value of it.
if (is_main) {
if (replication_role == memgraph::replication::ReplicationRole::MAIN) {
start_timestamp = timestamp_++;
} else {
start_timestamp = timestamp_;
@ -1700,10 +1702,10 @@ StorageInfo InMemoryStorage::GetBaseInfo(bool force_directory) {
return info;
}
StorageInfo InMemoryStorage::GetInfo(bool force_directory) {
StorageInfo InMemoryStorage::GetInfo(bool force_directory, memgraph::replication::ReplicationRole replication_role) {
StorageInfo info = GetBaseInfo(force_directory);
{
auto access = Access(std::nullopt);
auto access = Access(replication_role); // TODO: override isolation level?
const auto &lbl = access->ListAllIndices();
info.label_indices = lbl.label.size();
info.label_property_indices = lbl.label_property.size();
@ -2030,11 +2032,16 @@ void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOpera
return AppendToWalDataDefinition(operation, label, {}, {}, final_commit_timestamp);
}
utils::BasicResult<InMemoryStorage::CreateSnapshotError> InMemoryStorage::CreateSnapshot() {
utils::BasicResult<InMemoryStorage::CreateSnapshotError> InMemoryStorage::CreateSnapshot(
memgraph::replication::ReplicationRole replication_role) {
if (replication_role == memgraph::replication::ReplicationRole::REPLICA) {
return InMemoryStorage::CreateSnapshotError::DisabledForReplica;
}
auto const &epoch = repl_storage_state_.epoch_;
auto snapshot_creator = [this, &epoch]() {
utils::Timer timer;
auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION, storage_mode_);
auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION, storage_mode_,
memgraph::replication::ReplicationRole::MAIN);
durability::CreateSnapshot(this, &transaction, recovery_.snapshot_directory_, recovery_.wal_directory_, &vertices_,
&edges_, uuid_, epoch, repl_storage_state_.history, &file_retainer_);
// Finalize snapshot transaction.
@ -2122,17 +2129,17 @@ utils::FileRetainer::FileLockerAccessor::ret_type InMemoryStorage::UnlockPath()
return true;
}
std::unique_ptr<Storage::Accessor> InMemoryStorage::Access(std::optional<IsolationLevel> override_isolation_level,
bool is_main) {
std::unique_ptr<Storage::Accessor> InMemoryStorage::Access(memgraph::replication::ReplicationRole replication_role,
std::optional<IsolationLevel> override_isolation_level) {
return std::unique_ptr<InMemoryAccessor>(new InMemoryAccessor{Storage::Accessor::shared_access, this,
override_isolation_level.value_or(isolation_level_),
storage_mode_, is_main});
storage_mode_, replication_role});
}
std::unique_ptr<Storage::Accessor> InMemoryStorage::UniqueAccess(std::optional<IsolationLevel> override_isolation_level,
bool is_main) {
std::unique_ptr<Storage::Accessor> InMemoryStorage::UniqueAccess(
memgraph::replication::ReplicationRole replication_role, std::optional<IsolationLevel> override_isolation_level) {
return std::unique_ptr<InMemoryAccessor>(new InMemoryAccessor{Storage::Accessor::unique_access, this,
override_isolation_level.value_or(isolation_level_),
storage_mode_, is_main});
storage_mode_, replication_role});
}
void InMemoryStorage::CreateSnapshotHandler(

View File

@ -73,7 +73,7 @@ class InMemoryStorage final : public Storage {
friend class InMemoryStorage;
explicit InMemoryAccessor(auto tag, InMemoryStorage *storage, IsolationLevel isolation_level,
StorageMode storage_mode, bool is_main = true);
StorageMode storage_mode, memgraph::replication::ReplicationRole replication_role);
public:
InMemoryAccessor(const InMemoryAccessor &) = delete;
@ -322,12 +322,11 @@ class InMemoryStorage final : public Storage {
};
using Storage::Access;
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level,
bool is_main) override;
std::unique_ptr<Accessor> Access(memgraph::replication::ReplicationRole replication_role,
std::optional<IsolationLevel> override_isolation_level) override;
using Storage::UniqueAccess;
std::unique_ptr<Storage::Accessor> UniqueAccess(std::optional<IsolationLevel> override_isolation_level,
bool is_main) override;
std::unique_ptr<Accessor> UniqueAccess(memgraph::replication::ReplicationRole replication_role,
std::optional<IsolationLevel> override_isolation_level) override;
void FreeMemory(std::unique_lock<utils::ResourceLock> main_guard) override;
@ -335,12 +334,13 @@ class InMemoryStorage final : public Storage {
utils::FileRetainer::FileLockerAccessor::ret_type LockPath();
utils::FileRetainer::FileLockerAccessor::ret_type UnlockPath();
utils::BasicResult<InMemoryStorage::CreateSnapshotError> CreateSnapshot();
utils::BasicResult<InMemoryStorage::CreateSnapshotError> CreateSnapshot(
memgraph::replication::ReplicationRole replication_role);
void CreateSnapshotHandler(std::function<utils::BasicResult<InMemoryStorage::CreateSnapshotError>()> cb);
using Storage::CreateTransaction;
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode, bool is_main) override;
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode,
memgraph::replication::ReplicationRole replication_role) override;
void SetStorageMode(StorageMode storage_mode);
@ -365,7 +365,7 @@ class InMemoryStorage final : public Storage {
void FinalizeWalFile();
StorageInfo GetBaseInfo(bool force_directory) override;
StorageInfo GetInfo(bool force_directory) override;
StorageInfo GetInfo(bool force_directory, memgraph::replication::ReplicationRole replication_role) override;
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataManipulation(const Transaction &transaction, uint64_t final_commit_timestamp);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -117,11 +117,14 @@ class ReplicationStorageClient {
if (State() != replication::ReplicaState::REPLICATING) {
return;
}
if (replica_stream_->IsDefunct()) return;
if (!replica_stream_ || replica_stream_->IsDefunct()) {
replica_state_.WithLock([](auto &state) { state = replication::ReplicaState::MAYBE_BEHIND; });
return;
}
try {
callback(*replica_stream_); // failure state what if not streaming (std::nullopt)
} catch (const rpc::RpcFailedException &) {
return replica_state_.WithLock([](auto &state) { state = replication::ReplicaState::MAYBE_BEHIND; });
replica_state_.WithLock([](auto &state) { state = replication::ReplicaState::MAYBE_BEHIND; });
LogRpcFailure();
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -50,26 +50,26 @@ Storage::Storage(Config config, StorageMode storage_mode)
}
Storage::Accessor::Accessor(SharedAccess /* tag */, Storage *storage, IsolationLevel isolation_level,
StorageMode storage_mode, bool is_main)
StorageMode storage_mode, memgraph::replication::ReplicationRole replication_role)
: storage_(storage),
// The lock must be acquired before creating the transaction object to
// prevent freshly created transactions from dangling in an active state
// during exclusive operations.
storage_guard_(storage_->main_lock_),
unique_guard_(storage_->main_lock_, std::defer_lock),
transaction_(storage->CreateTransaction(isolation_level, storage_mode, is_main)),
transaction_(storage->CreateTransaction(isolation_level, storage_mode, replication_role)),
is_transaction_active_(true),
creation_storage_mode_(storage_mode) {}
Storage::Accessor::Accessor(UniqueAccess /* tag */, Storage *storage, IsolationLevel isolation_level,
StorageMode storage_mode, bool is_main)
StorageMode storage_mode, memgraph::replication::ReplicationRole replication_role)
: storage_(storage),
// The lock must be acquired before creating the transaction object to
// prevent freshly created transactions from dangling in an active state
// during exclusive operations.
storage_guard_(storage_->main_lock_, std::defer_lock),
unique_guard_(storage_->main_lock_),
transaction_(storage->CreateTransaction(isolation_level, storage_mode, is_main)),
transaction_(storage->CreateTransaction(isolation_level, storage_mode, replication_role)),
is_transaction_active_(true),
creation_storage_mode_(storage_mode) {}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -132,9 +132,9 @@ class Storage {
} unique_access;
Accessor(SharedAccess /* tag */, Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode,
bool is_main = true);
memgraph::replication::ReplicationRole replication_role);
Accessor(UniqueAccess /* tag */, Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode,
bool is_main = true);
memgraph::replication::ReplicationRole replication_role);
Accessor(const Accessor &) = delete;
Accessor &operator=(const Accessor &) = delete;
Accessor &operator=(Accessor &&other) = delete;
@ -315,19 +315,17 @@ class Storage {
void FreeMemory() { FreeMemory({}); }
virtual std::unique_ptr<Accessor> Access(std::optional<IsolationLevel> override_isolation_level, bool is_main) = 0;
std::unique_ptr<Accessor> Access(bool is_main = true) { return Access(std::optional<IsolationLevel>{}, is_main); }
std::unique_ptr<Accessor> Access(std::optional<IsolationLevel> override_isolation_level) {
return Access(std::move(override_isolation_level), true);
virtual std::unique_ptr<Accessor> Access(memgraph::replication::ReplicationRole replication_role,
std::optional<IsolationLevel> override_isolation_level) = 0;
std::unique_ptr<Accessor> Access(memgraph::replication::ReplicationRole replication_role) {
return Access(replication_role, {});
}
virtual std::unique_ptr<Accessor> UniqueAccess(std::optional<IsolationLevel> override_isolation_level,
bool is_main) = 0;
std::unique_ptr<Accessor> UniqueAccess(bool is_main = true) {
return UniqueAccess(std::optional<IsolationLevel>{}, is_main);
}
std::unique_ptr<Accessor> UniqueAccess(std::optional<IsolationLevel> override_isolation_level) {
return UniqueAccess(std::move(override_isolation_level), true);
virtual std::unique_ptr<Accessor> UniqueAccess(memgraph::replication::ReplicationRole replication_role,
std::optional<IsolationLevel> override_isolation_level) = 0;
std::unique_ptr<Accessor> UniqueAccess(memgraph::replication::ReplicationRole replication_role) {
return UniqueAccess(replication_role, {});
}
enum class SetIsolationLevelError : uint8_t { DisabledForAnalyticalMode };
@ -345,21 +343,10 @@ class Storage {
return GetBaseInfo(force_dir);
}
virtual StorageInfo GetInfo(bool force_directory) = 0;
StorageInfo GetInfo() {
#if MG_ENTERPRISE
const bool force_dir = false;
#else
const bool force_dir = true; //!< Use the configured directory (multi-tenancy reroutes to another dir)
#endif
return GetInfo(force_dir);
}
virtual StorageInfo GetInfo(bool force_directory, memgraph::replication::ReplicationRole replication_role) = 0;
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode) {
return CreateTransaction(isolation_level, storage_mode, true);
}
virtual Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode, bool is_main) = 0;
virtual Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode,
memgraph::replication::ReplicationRole replication_role) = 0;
virtual void PrepareForNewEpoch() = 0;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -68,7 +68,7 @@ inline std::string IterableToString(const TIterable &iterable, const std::string
*/
template <typename TIterable>
inline std::string IterableToString(const TIterable &iterable, const std::string_view delim = ", ") {
return IterableToString(iterable, delim, [](const auto &item) { return item; });
return IterableToString(iterable, delim, std::identity{});
}
/**
@ -216,8 +216,7 @@ TCollection Reversed(const TCollection &collection, const TAllocator &alloc) {
template <typename TIterator>
class Iterable {
public:
Iterable(TIterator &&begin, TIterator &&end)
: begin_(std::forward<TIterator>(begin)), end_(std::forward<TIterator>(end)) {}
Iterable(TIterator &&begin, TIterator &&end) : begin_(std::move(begin)), end_(std::move(end)) {}
auto begin() { return begin_; };
auto end() { return end_; };

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -11,6 +11,7 @@
#pragma once
#include <concepts>
#include <functional>
namespace memgraph::utils {
@ -37,7 +38,9 @@ namespace memgraph::utils {
template <typename Callable>
class [[nodiscard]] OnScopeExit {
public:
explicit OnScopeExit(Callable &&function) : function_{std::forward<Callable>(function)}, doCall_{true} {}
template <typename U>
requires std::constructible_from<Callable, U>
explicit OnScopeExit(U &&function) : function_{std::forward<U>(function)}, doCall_{true} {}
OnScopeExit(OnScopeExit const &) = delete;
OnScopeExit(OnScopeExit &&) = delete;
OnScopeExit &operator=(OnScopeExit const &) = delete;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -17,6 +17,7 @@
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/storage.hpp"
using memgraph::replication::ReplicationRole;
// The following classes are wrappers for memgraph::utils::MemoryResource, so that we can
// use BENCHMARK_TEMPLATE
@ -40,7 +41,7 @@ static void MapLiteral(benchmark::State &state) {
TMemory memory;
memgraph::query::Frame frame(symbol_table.max_position(), memory.get());
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
std::unordered_map<memgraph::query::PropertyIx, memgraph::query::Expression *> elements;
for (int64_t i = 0; i < state.range(0); ++i) {
@ -71,7 +72,7 @@ static void AdditionOperator(benchmark::State &state) {
TMemory memory;
memgraph::query::Frame frame(symbol_table.max_position(), memory.get());
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
memgraph::query::Expression *expr = ast.Create<memgraph::query::PrimitiveLiteral>(0);
for (int64_t i = 0; i < state.range(0); ++i) {

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -32,6 +32,8 @@
#include "query/interpreter.hpp"
#include "storage/v2/inmemory/storage.hpp"
using memgraph::replication::ReplicationRole;
// The following classes are wrappers for memgraph::utils::MemoryResource, so that we can
// use BENCHMARK_TEMPLATE
@ -61,7 +63,7 @@ class PoolResource final {
};
static void AddVertices(memgraph::storage::Storage *db, int vertex_count) {
auto dba = db->Access();
auto dba = db->Access(ReplicationRole::MAIN);
for (int i = 0; i < vertex_count; i++) dba->CreateVertex();
MG_ASSERT(!dba->Commit().HasError());
}
@ -70,7 +72,7 @@ static const char *kStartLabel = "start";
static void AddStarGraph(memgraph::storage::Storage *db, int spoke_count, int depth) {
{
auto dba = db->Access();
auto dba = db->Access(ReplicationRole::MAIN);
auto center_vertex = dba->CreateVertex();
MG_ASSERT(center_vertex.AddLabel(dba->NameToLabel(kStartLabel)).HasValue());
for (int i = 0; i < spoke_count; ++i) {
@ -84,14 +86,14 @@ static void AddStarGraph(memgraph::storage::Storage *db, int spoke_count, int de
MG_ASSERT(!dba->Commit().HasError());
}
{
auto unique_acc = db->UniqueAccess();
auto unique_acc = db->UniqueAccess(ReplicationRole::MAIN);
MG_ASSERT(!unique_acc->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
}
}
static void AddTree(memgraph::storage::Storage *db, int vertex_count) {
{
auto dba = db->Access();
auto dba = db->Access(ReplicationRole::MAIN);
std::vector<memgraph::storage::VertexAccessor> vertices;
vertices.reserve(vertex_count);
auto root = dba->CreateVertex();
@ -109,7 +111,7 @@ static void AddTree(memgraph::storage::Storage *db, int vertex_count) {
MG_ASSERT(!dba->Commit().HasError());
}
{
auto unique_acc = db->UniqueAccess();
auto unique_acc = db->UniqueAccess(ReplicationRole::MAIN);
MG_ASSERT(!unique_acc->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
}
}
@ -132,7 +134,7 @@ static void Distinct(benchmark::State &state) {
memgraph::query::Parameters parameters;
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddVertices(db.get(), state.range(0));
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto query_string = "MATCH (s) RETURN DISTINCT s";
auto *cypher_query = ParseCypherQuery(query_string, &ast);
@ -185,7 +187,7 @@ static void ExpandVariable(benchmark::State &state) {
AddStarGraph(db.get(), state.range(0), state.range(1));
memgraph::query::SymbolTable symbol_table;
auto expand_variable = MakeExpandVariable(memgraph::query::EdgeAtom::Type::DEPTH_FIRST, &symbol_table);
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -225,7 +227,7 @@ static void ExpandBfs(benchmark::State &state) {
AddTree(db.get(), state.range(0));
memgraph::query::SymbolTable symbol_table;
auto expand_variable = MakeExpandVariable(memgraph::query::EdgeAtom::Type::BREADTH_FIRST, &symbol_table);
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -261,7 +263,7 @@ static void ExpandShortest(benchmark::State &state) {
auto expand_variable = MakeExpandVariable(memgraph::query::EdgeAtom::Type::BREADTH_FIRST, &symbol_table);
expand_variable.common_.existing_node = true;
auto dest_symbol = expand_variable.common_.node_symbol;
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -303,7 +305,7 @@ static void ExpandWeightedShortest(benchmark::State &state) {
symbol_table.CreateSymbol("edge", false), symbol_table.CreateSymbol("vertex", false),
ast.Create<memgraph::query::PrimitiveLiteral>(1)};
auto dest_symbol = expand_variable.common_.node_symbol;
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -349,7 +351,7 @@ static void Accumulate(benchmark::State &state) {
}
memgraph::query::plan::Accumulate accumulate(scan_all, symbols,
/* advance_command= */ false);
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -399,7 +401,7 @@ static void Aggregate(benchmark::State &state) {
symbol_table.CreateSymbol("out" + std::to_string(i), false)});
}
memgraph::query::plan::Aggregate aggregate(scan_all, aggregations, group_by, symbols);
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -450,7 +452,7 @@ static void OrderBy(benchmark::State &state) {
sort_items.push_back({memgraph::query::Ordering::ASC, ast.Create<memgraph::query::PrimitiveLiteral>(rand_value)});
}
memgraph::query::plan::OrderBy order_by(scan_all, sort_items, symbols);
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -489,7 +491,7 @@ static void Unwind(benchmark::State &state) {
auto *list_expr = ast.Create<memgraph::query::Identifier>("list")->MapTo(list_sym);
auto out_sym = symbol_table.CreateSymbol("out", false);
memgraph::query::plan::Unwind unwind(scan_all, list_expr, out_sym);
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
@ -527,7 +529,7 @@ static void Foreach(benchmark::State &state) {
std::make_shared<memgraph::query::plan::CreateNode>(nullptr, memgraph::query::plan::NodeCreationInfo{});
auto foreach = std::make_shared<memgraph::query::plan::Foreach>(nullptr, std::move(create_node), list_expr, out_sym);
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -19,6 +19,8 @@
#include "query/plan/vertex_count_cache.hpp"
#include "storage/v2/inmemory/storage.hpp"
using memgraph::replication::ReplicationRole;
// Add chained MATCH (node1) -- (node2), MATCH (node2) -- (node3) ... clauses.
static memgraph::query::CypherQuery *AddChainedMatches(int num_matches, memgraph::query::AstStorage &storage) {
auto *query = storage.Create<memgraph::query::CypherQuery>();
@ -44,7 +46,7 @@ static memgraph::query::CypherQuery *AddChainedMatches(int num_matches, memgraph
static void BM_PlanChainedMatches(benchmark::State &state) {
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
while (state.KeepRunning()) {
state.PauseTiming();
@ -95,10 +97,10 @@ static auto CreateIndexedVertices(int index_count, int vertex_count, memgraph::s
auto label = db->NameToLabel("label");
auto prop = db->NameToProperty("prop");
{
auto unique_acc = db->UniqueAccess();
auto unique_acc = db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
}
auto dba = db->Access();
auto dba = db->Access(ReplicationRole::MAIN);
for (int vi = 0; vi < vertex_count; ++vi) {
for (int index = 0; index < index_count; ++index) {
auto vertex = dba->CreateVertex();
@ -117,7 +119,7 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db.get());
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
memgraph::query::Parameters parameters;
while (state.KeepRunning()) {
@ -146,7 +148,7 @@ static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(benchmark::State &
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db.get());
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto vertex_counts = memgraph::query::plan::MakeVertexCountCache(&dba);
memgraph::query::Parameters parameters;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -141,9 +141,9 @@ struct ScopedProfile {
}
}
Context *context;
ProfilingStats *stats;
unsigned long long start_time;
Context *context = nullptr;
ProfilingStats *stats = nullptr;
unsigned long long start_time{};
};
//////////////////////////////////////////////////////////////////////////////

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -17,6 +17,8 @@
#include "storage/v2/storage.hpp"
#include "utils/timer.hpp"
using memgraph::replication::ReplicationRole;
// This benchmark should be run for a fixed amount of time that is
// large compared to GC interval to make the output relevant.
@ -42,7 +44,7 @@ void UpdateLabelFunc(int thread_id, memgraph::storage::Storage *storage,
memgraph::utils::Timer timer;
for (int iter = 0; iter < num_iterations; ++iter) {
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
memgraph::storage::Gid gid = vertices.at(vertex_dist(gen));
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
MG_ASSERT(vertex.has_value(), "Vertex with GID {} doesn't exist", gid.AsUint());
@ -61,7 +63,7 @@ int main(int argc, char *argv[]) {
std::unique_ptr<memgraph::storage::Storage> storage(new memgraph::storage::InMemoryStorage(config.second));
std::vector<memgraph::storage::Gid> vertices;
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < FLAGS_num_vertices; ++i) {
vertices.push_back(acc->CreateVertex().Gid());
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -17,6 +17,8 @@
#include "storage/v2/storage.hpp"
#include "utils/timer.hpp"
using memgraph::replication::ReplicationRole;
// This benchmark should be run for a fixed amount of time that is
// large compared to GC interval to make the output relevant.
@ -41,14 +43,14 @@ int main(int argc, char *argv[]) {
std::array<memgraph::storage::Gid, 1> vertices;
memgraph::storage::PropertyId pid;
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
vertices[0] = acc->CreateVertex().Gid();
pid = acc->NameToProperty("NEW_PROP");
MG_ASSERT(!acc->Commit().HasError());
}
for (int iter = 0; iter != FLAGS_num_iterations; ++iter) {
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(vertices[0], memgraph::storage::View::OLD);
for (auto i = 0; i != FLAGS_num_poperties; ++i) {
MG_ASSERT(!vertex1.value().SetProperty(pid, memgraph::storage::PropertyValue{i}).HasError());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -19,6 +19,8 @@
#include "storage/v2/storage_error.hpp"
#include "utils/thread.hpp"
using memgraph::replication::ReplicationRole;
const uint64_t kNumVerifiers = 5;
const uint64_t kNumMutators = 1;
@ -31,7 +33,7 @@ TEST(Storage, LabelIndex) {
auto label = store->NameToLabel("label");
{
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label).HasError());
}
@ -44,7 +46,7 @@ TEST(Storage, LabelIndex) {
gids.reserve(kNumIterations * kVerifierBatchSize);
for (uint64_t i = 0; i < kNumIterations; ++i) {
for (uint64_t j = 0; j < kVerifierBatchSize; ++j) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gids.emplace(vertex.Gid(), false);
auto ret = vertex.AddLabel(label);
@ -53,7 +55,7 @@ TEST(Storage, LabelIndex) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertices = acc->Vertices(label, memgraph::storage::View::OLD);
for (auto vertex : vertices) {
auto it = gids.find(vertex.Gid());
@ -81,7 +83,7 @@ TEST(Storage, LabelIndex) {
gids.resize(kMutatorBatchSize);
while (mutators_run.load(std::memory_order_acquire)) {
for (uint64_t i = 0; i < kMutatorBatchSize; ++i) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gids[i] = vertex.Gid();
auto ret = vertex.AddLabel(label);
@ -90,7 +92,7 @@ TEST(Storage, LabelIndex) {
ASSERT_FALSE(acc->Commit().HasError());
}
for (uint64_t i = 0; i < kMutatorBatchSize; ++i) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
ASSERT_TRUE(acc->DeleteVertex(&*vertex).HasValue());
@ -116,7 +118,7 @@ TEST(Storage, LabelPropertyIndex) {
auto label = store->NameToLabel("label");
auto prop = store->NameToProperty("prop");
{
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label, prop).HasError());
}
@ -129,7 +131,7 @@ TEST(Storage, LabelPropertyIndex) {
gids.reserve(kNumIterations * kVerifierBatchSize);
for (uint64_t i = 0; i < kNumIterations; ++i) {
for (uint64_t j = 0; j < kVerifierBatchSize; ++j) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gids.emplace(vertex.Gid(), false);
{
@ -145,7 +147,7 @@ TEST(Storage, LabelPropertyIndex) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertices = acc->Vertices(label, prop, memgraph::storage::View::OLD);
for (auto vertex : vertices) {
auto it = gids.find(vertex.Gid());
@ -173,7 +175,7 @@ TEST(Storage, LabelPropertyIndex) {
gids.resize(kMutatorBatchSize);
while (mutators_run.load(std::memory_order_acquire)) {
for (uint64_t i = 0; i < kMutatorBatchSize; ++i) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gids[i] = vertex.Gid();
{
@ -189,7 +191,7 @@ TEST(Storage, LabelPropertyIndex) {
ASSERT_FALSE(acc->Commit().HasError());
}
for (uint64_t i = 0; i < kMutatorBatchSize; ++i) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
ASSERT_TRUE(acc->DeleteVertex(&*vertex).HasValue());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -16,6 +16,8 @@
#include "storage/v2/constraints/constraints.hpp"
#include "storage/v2/inmemory/storage.hpp"
using memgraph::replication::ReplicationRole;
const int kNumThreads = 8;
#define ASSERT_OK(x) ASSERT_FALSE((x).HasError())
@ -34,7 +36,7 @@ class StorageUniqueConstraints : public ::testing::Test {
void SetUp() override {
// Create initial vertices.
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
// NOLINTNEXTLINE(modernize-loop-convert)
for (int i = 0; i < kNumThreads; ++i) {
auto vertex = acc->CreateVertex();
@ -55,7 +57,7 @@ void SetProperties(memgraph::storage::Storage *storage, memgraph::storage::Gid g
const std::vector<PropertyId> &properties, const std::vector<PropertyValue> &values,
bool *commit_status) {
ASSERT_EQ(properties.size(), values.size());
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
int value = 0;
@ -71,7 +73,7 @@ void SetProperties(memgraph::storage::Storage *storage, memgraph::storage::Gid g
}
void AddLabel(memgraph::storage::Storage *storage, memgraph::storage::Gid gid, LabelId label, bool *commit_status) {
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
for (int iter = 0; iter < 40000; ++iter) {
@ -84,7 +86,7 @@ void AddLabel(memgraph::storage::Storage *storage, memgraph::storage::Gid gid, L
TEST_F(StorageUniqueConstraints, ChangeProperties) {
{
auto unique_acc = storage->UniqueAccess();
auto unique_acc = storage->UniqueAccess(ReplicationRole::MAIN);
auto res = unique_acc->CreateUniqueConstraint(label, {prop1, prop2, prop3});
ASSERT_TRUE(res.HasValue());
ASSERT_EQ(res.GetValue(), memgraph::storage::UniqueConstraints::CreationStatus::SUCCESS);
@ -92,7 +94,7 @@ TEST_F(StorageUniqueConstraints, ChangeProperties) {
}
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
// NOLINTNEXTLINE(modernize-loop-convert)
for (int i = 0; i < kNumThreads; ++i) {
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
@ -168,7 +170,7 @@ TEST_F(StorageUniqueConstraints, ChangeProperties) {
TEST_F(StorageUniqueConstraints, ChangeLabels) {
{
auto unique_acc = storage->UniqueAccess();
auto unique_acc = storage->UniqueAccess(ReplicationRole::MAIN);
auto res = unique_acc->CreateUniqueConstraint(label, {prop1, prop2, prop3});
ASSERT_TRUE(res.HasValue());
ASSERT_EQ(res.GetValue(), memgraph::storage::UniqueConstraints::CreationStatus::SUCCESS);
@ -181,7 +183,7 @@ TEST_F(StorageUniqueConstraints, ChangeLabels) {
// succeed, as the others should result with constraint violation.
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
// NOLINTNEXTLINE(modernize-loop-convert)
for (int i = 0; i < kNumThreads; ++i) {
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
@ -196,7 +198,7 @@ TEST_F(StorageUniqueConstraints, ChangeLabels) {
for (int iter = 0; iter < 20; ++iter) {
// Clear labels.
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
// NOLINTNEXTLINE(modernize-loop-convert)
for (int i = 0; i < kNumThreads; ++i) {
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
@ -227,7 +229,7 @@ TEST_F(StorageUniqueConstraints, ChangeLabels) {
// should succeed.
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
// NOLINTNEXTLINE(modernize-loop-convert)
for (int i = 0; i < kNumThreads; ++i) {
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
@ -242,7 +244,7 @@ TEST_F(StorageUniqueConstraints, ChangeLabels) {
for (int iter = 0; iter < 20; ++iter) {
// Clear labels.
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
// NOLINTNEXTLINE(modernize-loop-convert)
for (int i = 0; i < kNumThreads; ++i) {
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);

View File

@ -1754,5 +1754,74 @@ def test_triggers_on_create_before_commit_with_offline_sync_replica():
assert res_from_main == interactive_mg_runner.MEMGRAPH_INSTANCES["sync_replica2"].query(QUERY_TO_CHECK)
def test_replication_not_messed_up_by_CreateSnapshot(connection):
# Goal of this test is to check the replica can not run CreateSnapshot
# 1/ CREATE SNAPSHOT should raise a DatabaseError
interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION)
cursor = connection(7688, "replica_1").cursor()
# 1/
with pytest.raises(mgclient.DatabaseError):
execute_and_fetch_all(cursor, "CREATE SNAPSHOT;")
def test_replication_not_messed_up_by_ShowIndexInfo(connection):
# Goal of this test is to check the replicas timestamp and hence ability to recieve MAINs writes
# is uneffected by SHOW INDEX INFO
# 1/ Run SHOW INDEX INFO; multiple times on REPLICA
# 2/ Send a write from MAIN
# 3/ Check REPLICA processed the write
BASIC_MEMGRAPH_INSTANCES_DESCRIPTION = {
"replica_1": {
"args": ["--bolt-port", "7688", "--log-level=TRACE"],
"log_file": "replica1.log",
"setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10001;"],
},
"main": {
"args": ["--bolt-port", "7687", "--log-level=TRACE"],
"log_file": "main.log",
"setup_queries": [
"REGISTER REPLICA replica_1 ASYNC TO '127.0.0.1:10001';",
],
},
}
interactive_mg_runner.start_all(BASIC_MEMGRAPH_INSTANCES_DESCRIPTION)
cursor = connection(7688, "replica_1").cursor()
# 1/
# This query use to incorrectly change REPLICA storage timestamp
# run this multiple times to try and get into error case of MAIN timestamp < REPLICA timestamp
for _ in range(20):
execute_and_fetch_all(cursor, "SHOW INDEX INFO;")
cursor = connection(7687, "main").cursor()
# 2/
execute_and_fetch_all(cursor, "CREATE ();")
def retrieve_data():
replicas = interactive_mg_runner.MEMGRAPH_INSTANCES["main"].query("SHOW REPLICAS;")
return replicas
expected_data = [
("replica_1", "127.0.0.1:10001", "async", 2, 0, "ready"),
]
actual_data = mg_sleep_and_assert(expected_data, retrieve_data)
assert actual_data == expected_data
# 3/
cursor = connection(7688, "replica_1").cursor()
result = execute_and_fetch_all(cursor, "MATCH () RETURN count(*);")
assert len(result) == 1
assert result[0][0] == 1 # The one node was replicated from MAIN to REPLICA
if __name__ == "__main__":
sys.exit(pytest.main([__file__, "-rA"]))

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -14,14 +14,14 @@
#include <gflags/gflags.h>
#include "storage/v2/inmemory/storage.hpp"
using memgraph::replication::ReplicationRole;
DECLARE_int32(min_log_level);
int main(int argc, char *argv[]) {
gflags::ParseCommandLineFlags(&argc, &argv, true);
spdlog::set_level(spdlog::level::err);
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
auto storage_dba = db->Access();
auto storage_dba = db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
RunInteractivePlanning(&dba);
return 0;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -23,7 +23,7 @@
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/vertex_accessor.hpp"
using memgraph::replication::ReplicationRole;
/**
* It is possible to run test with custom seed with:
* RC_PARAMS="seed=1" ./random_graph
@ -40,7 +40,7 @@ RC_GTEST_PROP(RandomGraph, RandomGraph, (std::vector<std::string> vertex_labels,
std::unordered_map<memgraph::storage::VertexAccessor, std::string> vertex_label_map;
std::unordered_map<memgraph::storage::EdgeAccessor, std::string> edge_type_map;
auto dba = db->Access();
auto dba = db->Access(ReplicationRole::MAIN);
for (auto label : vertex_labels) {
auto vertex_accessor = dba->CreateVertex();

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -22,7 +22,7 @@
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/view.hpp"
using memgraph::replication::ReplicationRole;
#ifdef MG_ENTERPRISE
template <typename StorageType>
class FineGrainedAuthCheckerFixture : public testing::Test {
@ -31,7 +31,7 @@ class FineGrainedAuthCheckerFixture : public testing::Test {
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
// make a V-graph (v3)<-[r2]-(v1)-[r1]->(v2)

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -42,7 +42,9 @@ class VertexDb : public Database {
}
}
std::unique_ptr<memgraph::storage::Storage::Accessor> Access() override { return db_->Access(); }
std::unique_ptr<memgraph::storage::Storage::Accessor> Access() override {
return db_->Access(memgraph::replication::ReplicationRole::MAIN);
}
std::unique_ptr<LogicalOperator> MakeBfsOperator(Symbol source_sym, Symbol sink_sym, Symbol edge_sym,
EdgeAtom::Direction direction,

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -31,7 +31,9 @@ class SingleNodeDb : public Database {
}
}
std::unique_ptr<memgraph::storage::Storage::Accessor> Access() override { return db_->Access(); }
std::unique_ptr<memgraph::storage::Storage::Accessor> Access() override {
return db_->Access(memgraph::replication::ReplicationRole::MAIN);
}
std::unique_ptr<LogicalOperator> MakeBfsOperator(Symbol source_sym, Symbol sink_sym, Symbol edge_sym,
EdgeAtom::Direction direction,

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -182,7 +182,7 @@ void TestVertexAndEdgeWithDifferentStorages(std::unique_ptr<memgraph::storage::S
output.clear();
// create vertex
auto dba = db->Access();
auto dba = db->Access(memgraph::replication::ReplicationRole::MAIN);
auto va1 = dba->CreateVertex();
auto va2 = dba->CreateVertex();
auto l1 = dba->NameToLabel("label1");

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -20,6 +20,8 @@
#include "storage/v2/property_value.hpp"
#include "storage/v2/view.hpp"
using memgraph::replication::ReplicationRole;
class ClearingOldDiskDataTest : public ::testing::Test {
public:
const std::string testSuite = "clearing_old_disk_data";
@ -33,7 +35,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithVertexTimestampUpdate) {
auto *tx_db = disk_storage->GetRocksDBStorage()->db_;
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 0);
auto acc1 = disk_storage->Access(std::nullopt);
auto acc1 = disk_storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto label1 = acc1->NameToLabel("DiskLabel");
auto property1 = acc1->NameToProperty("DiskProperty");
@ -43,7 +45,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithVertexTimestampUpdate) {
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc2 = disk_storage->Access(std::nullopt);
auto acc2 = disk_storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW).value();
/// This is the same property as in the first transaction, we just want to test
/// the number of entries inside RocksDB when the timestamp changes
@ -58,7 +60,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithVertexValueUpdate) {
auto *tx_db = disk_storage->GetRocksDBStorage()->db_;
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 0);
auto acc1 = disk_storage->Access(std::nullopt);
auto acc1 = disk_storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto label1 = acc1->NameToLabel("DiskLabel");
auto property1 = acc1->NameToProperty("DiskProperty");
@ -68,7 +70,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithVertexValueUpdate) {
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc2 = disk_storage->Access(std::nullopt);
auto acc2 = disk_storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW).value();
/// This is the same property as in the first transaction, we just want to test
/// the number of entries inside RocksDB when the timestamp changes
@ -83,7 +85,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithVertexKeyUpdate) {
auto *tx_db = disk_storage->GetRocksDBStorage()->db_;
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 0);
auto acc1 = disk_storage->Access(std::nullopt);
auto acc1 = disk_storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto label1 = acc1->NameToLabel("DiskLabel");
auto property1 = acc1->NameToProperty("DiskProperty");
@ -93,7 +95,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithVertexKeyUpdate) {
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc2 = disk_storage->Access(std::nullopt);
auto acc2 = disk_storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW).value();
auto label2 = acc2->NameToLabel("DiskLabel2");
ASSERT_TRUE(vertex2.AddLabel(label2).HasValue());
@ -106,7 +108,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithEdgeTimestampUpdate) {
auto *tx_db = disk_storage->GetRocksDBStorage()->db_;
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 0);
auto acc1 = disk_storage->Access(std::nullopt);
auto acc1 = disk_storage->Access(ReplicationRole::MAIN);
auto label1 = acc1->NameToLabel("DiskLabel");
auto property1 = acc1->NameToProperty("DiskProperty");
@ -126,7 +128,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithEdgeTimestampUpdate) {
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 5);
auto acc2 = disk_storage->Access(std::nullopt);
auto acc2 = disk_storage->Access(ReplicationRole::MAIN);
auto from_vertex = acc2->FindVertex(from.Gid(), memgraph::storage::View::NEW).value();
auto ret = from_vertex.OutEdges(memgraph::storage::View::NEW);
@ -145,7 +147,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithEdgeValueUpdate) {
auto *tx_db = disk_storage->GetRocksDBStorage()->db_;
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 0);
auto acc1 = disk_storage->Access(std::nullopt);
auto acc1 = disk_storage->Access(ReplicationRole::MAIN);
auto label1 = acc1->NameToLabel("DiskLabel");
auto property1 = acc1->NameToProperty("DiskProperty");
@ -165,7 +167,7 @@ TEST_F(ClearingOldDiskDataTest, TestNumOfEntriesWithEdgeValueUpdate) {
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 5);
auto acc2 = disk_storage->Access(std::nullopt);
auto acc2 = disk_storage->Access(ReplicationRole::MAIN);
auto from_vertex = acc2->FindVertex(from.Gid(), memgraph::storage::View::NEW).value();
auto ret = from_vertex.OutEdges(memgraph::storage::View::NEW);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -43,7 +43,7 @@ struct CppApiTestFixture : public ::testing::Test {
}
memgraph::query::DbAccessor &CreateDbAccessor(const memgraph::storage::IsolationLevel isolationLevel) {
accessors_.push_back(storage->Access(isolationLevel));
accessors_.push_back(storage->Access(memgraph::replication::ReplicationRole::MAIN, isolationLevel));
db_accessors_.emplace_back(accessors_.back().get());
return db_accessors_.back();
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -93,12 +93,12 @@ TYPED_TEST(InfoTest, InfoCheck) {
{
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateExistenceConstraint(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->DropExistenceConstraint(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
@ -123,49 +123,50 @@ TYPED_TEST(InfoTest, InfoCheck) {
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateIndex(lbl).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateIndex(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateIndex(lbl, prop2).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->DropIndex(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(lbl, {prop2}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(lbl2, {prop}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(lbl3, {prop}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db_acc->storage()->UniqueAccess();
auto unique_acc = db_acc->UniqueAccess();
ASSERT_EQ(unique_acc->DropUniqueConstraint(lbl, {prop2}),
memgraph::storage::UniqueConstraints::DeletionStatus::SUCCESS);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
const auto &info = db_acc->GetInfo(true); // force to use configured directory
const auto &info =
db_acc->GetInfo(true, memgraph::replication::ReplicationRole::MAIN); // force to use configured directory
ASSERT_EQ(info.storage_info.vertex_count, 5);
ASSERT_EQ(info.storage_info.edge_count, 2);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -43,7 +43,7 @@ class PrintToJsonTest : public ::testing::Test {
PrintToJsonTest()
: config(disk_test_utils::GenerateOnDiskConfig(testSuite)),
db(new StorageType(config)),
dba_storage(db->Access()),
dba_storage(db->Access(memgraph::replication::ReplicationRole::MAIN)),
dba(dba_storage.get()) {}
~PrintToJsonTest() override {

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -22,7 +22,7 @@
using namespace memgraph::query;
using namespace memgraph::query::plan;
using memgraph::replication::ReplicationRole;
using CardParam = CostEstimator<memgraph::query::DbAccessor>::CardParam;
using CostParam = CostEstimator<memgraph::query::DbAccessor>::CostParam;
using MiscParam = CostEstimator<memgraph::query::DbAccessor>::MiscParam;
@ -51,16 +51,16 @@ class QueryCostEstimator : public ::testing::Test {
void SetUp() override {
{
auto unique_acc = db->UniqueAccess();
auto unique_acc = db->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = db->UniqueAccess();
auto unique_acc = db->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label, property).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
storage_dba.emplace(db->Access());
storage_dba.emplace(db->Access(ReplicationRole::MAIN));
dba.emplace(storage_dba->get());
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -137,7 +137,7 @@ DatabaseState GetState(memgraph::storage::Storage *db) {
// Capture all vertices
std::map<memgraph::storage::Gid, int64_t> gid_mapping;
std::set<DatabaseState::Vertex> vertices;
auto dba = db->Access();
auto dba = db->Access(memgraph::replication::ReplicationRole::MAIN);
for (const auto &vertex : dba->Vertices(memgraph::storage::View::NEW)) {
std::set<std::string, std::less<>> labels;
auto maybe_labels = vertex.Labels(memgraph::storage::View::NEW);
@ -329,7 +329,7 @@ TYPED_TEST(DumpTest, EmptyGraph) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -339,7 +339,7 @@ TYPED_TEST(DumpTest, EmptyGraph) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, SingleVertex) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {}, {}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -348,7 +348,7 @@ TYPED_TEST(DumpTest, SingleVertex) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -360,7 +360,7 @@ TYPED_TEST(DumpTest, SingleVertex) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, VertexWithSingleLabel) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {"Label1"}, {}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -369,7 +369,7 @@ TYPED_TEST(DumpTest, VertexWithSingleLabel) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -381,7 +381,7 @@ TYPED_TEST(DumpTest, VertexWithSingleLabel) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, VertexWithMultipleLabels) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {"Label1", "Label 2"}, {}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -390,7 +390,7 @@ TYPED_TEST(DumpTest, VertexWithMultipleLabels) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -403,7 +403,7 @@ TYPED_TEST(DumpTest, VertexWithMultipleLabels) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, VertexWithSingleProperty) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {}, {{"prop", memgraph::storage::PropertyValue(42)}}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -412,7 +412,7 @@ TYPED_TEST(DumpTest, VertexWithSingleProperty) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -424,7 +424,7 @@ TYPED_TEST(DumpTest, VertexWithSingleProperty) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, MultipleVertices) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {}, {}, false);
CreateVertex(dba.get(), {}, {}, false);
CreateVertex(dba.get(), {}, {}, false);
@ -435,7 +435,7 @@ TYPED_TEST(DumpTest, MultipleVertices) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -447,7 +447,7 @@ TYPED_TEST(DumpTest, MultipleVertices) {
TYPED_TEST(DumpTest, PropertyValue) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
auto null_value = memgraph::storage::PropertyValue();
auto int_value = memgraph::storage::PropertyValue(13);
auto bool_value = memgraph::storage::PropertyValue(true);
@ -473,7 +473,7 @@ TYPED_TEST(DumpTest, PropertyValue) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -489,7 +489,7 @@ TYPED_TEST(DumpTest, PropertyValue) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, SingleEdge) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
auto u = CreateVertex(dba.get(), {}, {}, false);
auto v = CreateVertex(dba.get(), {}, {}, false);
CreateEdge(dba.get(), &u, &v, "EdgeType", {}, false);
@ -500,7 +500,7 @@ TYPED_TEST(DumpTest, SingleEdge) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -515,7 +515,7 @@ TYPED_TEST(DumpTest, SingleEdge) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, MultipleEdges) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
auto u = CreateVertex(dba.get(), {}, {}, false);
auto v = CreateVertex(dba.get(), {}, {}, false);
auto w = CreateVertex(dba.get(), {}, {}, false);
@ -529,7 +529,7 @@ TYPED_TEST(DumpTest, MultipleEdges) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -548,7 +548,7 @@ TYPED_TEST(DumpTest, MultipleEdges) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, EdgeWithProperties) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
auto u = CreateVertex(dba.get(), {}, {}, false);
auto v = CreateVertex(dba.get(), {}, {}, false);
CreateEdge(dba.get(), &u, &v, "EdgeType", {{"prop", memgraph::storage::PropertyValue(13)}}, false);
@ -559,7 +559,7 @@ TYPED_TEST(DumpTest, EdgeWithProperties) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -574,7 +574,7 @@ TYPED_TEST(DumpTest, EdgeWithProperties) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, IndicesKeys) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {"Label1", "Label 2"}, {{"p", memgraph::storage::PropertyValue(1)}}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -599,7 +599,7 @@ TYPED_TEST(DumpTest, IndicesKeys) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -612,7 +612,7 @@ TYPED_TEST(DumpTest, IndicesKeys) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, ExistenceConstraints) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {"L`abel 1"}, {{"prop", memgraph::storage::PropertyValue(1)}}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -628,7 +628,7 @@ TYPED_TEST(DumpTest, ExistenceConstraints) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -640,7 +640,7 @@ TYPED_TEST(DumpTest, ExistenceConstraints) {
TYPED_TEST(DumpTest, UniqueConstraints) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {"Label"},
{{"prop", memgraph::storage::PropertyValue(1)}, {"prop2", memgraph::storage::PropertyValue(2)}},
false);
@ -663,7 +663,7 @@ TYPED_TEST(DumpTest, UniqueConstraints) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -682,7 +682,7 @@ TYPED_TEST(DumpTest, UniqueConstraints) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, CheckStateVertexWithMultipleProperties) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
std::map<std::string, memgraph::storage::PropertyValue> prop1 = {
{"nested1", memgraph::storage::PropertyValue(1337)}, {"nested2", memgraph::storage::PropertyValue(3.14)}};
@ -722,7 +722,7 @@ TYPED_TEST(DumpTest, CheckStateVertexWithMultipleProperties) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -739,7 +739,7 @@ TYPED_TEST(DumpTest, CheckStateVertexWithMultipleProperties) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, CheckStateSimpleGraph) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
auto u = CreateVertex(dba.get(), {"Person"}, {{"name", memgraph::storage::PropertyValue("Ivan")}});
auto v = CreateVertex(dba.get(), {"Person"}, {{"name", memgraph::storage::PropertyValue("Josko")}});
auto w = CreateVertex(
@ -839,7 +839,7 @@ TYPED_TEST(DumpTest, CheckStateSimpleGraph) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
{
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::DumpDatabaseToCypherQueries(&dba, &query_stream);
}
@ -862,7 +862,7 @@ TYPED_TEST(DumpTest, CheckStateSimpleGraph) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(DumpTest, ExecuteDumpDatabase) {
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {}, {}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -925,7 +925,7 @@ TYPED_TEST(DumpTest, ExecuteDumpDatabaseInMulticommandTransaction) {
// Create the vertex.
{
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
CreateVertex(dba.get(), {}, {}, false);
ASSERT_FALSE(dba->Commit().HasError());
}
@ -1023,7 +1023,7 @@ TYPED_TEST(DumpTest, MultiplePartialPulls) {
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto dba = this->db->storage()->Access();
auto dba = this->db->Access();
auto p1 = CreateVertex(dba.get(), {"PERSON"},
{{"name", memgraph::storage::PropertyValue("Person1")},
{"surname", memgraph::storage::PropertyValue("Unique1")}},
@ -1053,7 +1053,7 @@ TYPED_TEST(DumpTest, MultiplePartialPulls) {
ResultStreamFaker stream(this->db->storage());
memgraph::query::AnyStream query_stream(&stream, memgraph::utils::NewDeleteResource());
auto acc = this->db->storage()->Access();
auto acc = this->db->Access();
memgraph::query::DbAccessor dba(acc.get());
memgraph::query::PullPlanDump pullPlan{&dba};

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -67,7 +67,7 @@ class ExpressionEvaluatorTest : public ::testing::Test {
ExpressionEvaluatorTest()
: config(disk_test_utils::GenerateOnDiskConfig(testSuite)),
db(new StorageType(config)),
storage_dba(db->Access()),
storage_dba(db->Access(memgraph::replication::ReplicationRole::MAIN)),
dba(storage_dba.get()) {}
~ExpressionEvaluatorTest() override {

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -39,7 +39,7 @@ class HintProviderSuite : public ::testing::Test {
int symbol_count = 0;
void SetUp() {
storage_dba.emplace(db->Access());
storage_dba.emplace(db->Access(memgraph::replication::ReplicationRole::MAIN));
dba.emplace(storage_dba->get());
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -25,6 +25,8 @@
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/inmemory/storage.hpp"
using memgraph::replication::ReplicationRole;
using namespace memgraph::query;
using namespace memgraph::query::plan;
using memgraph::query::test_common::ToIntList;
@ -95,7 +97,7 @@ TYPED_TEST(QueryPlanTest, Accumulate) {
this->db.reset(nullptr);
this->CleanStorageDirs();
this->db = std::make_unique<TypeParam>(this->config);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto prop = dba.NameToProperty("x");
@ -148,7 +150,7 @@ TYPED_TEST(QueryPlanTest, AccumulateAdvance) {
this->db.reset();
this->CleanStorageDirs();
this->db = std::make_unique<TypeParam>(this->config);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
NodeCreationInfo node;
@ -167,7 +169,7 @@ TYPED_TEST(QueryPlanTest, AccumulateAdvance) {
template <typename StorageType>
class QueryPlanAggregateOps : public QueryPlanTest<StorageType> {
protected:
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
memgraph::storage::PropertyId prop = this->db->NameToProperty("prop");
@ -308,7 +310,7 @@ TYPED_TEST(QueryPlanTest, AggregateGroupByValues) {
// Tests that distinct groups are aggregated properly for values of all types.
// Also test the "remember" part of the Aggregation API as final results are
// obtained via a property lookup of a remembered node.
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// a vector of memgraph::storage::PropertyValue to be set as property values on vertices
@ -370,7 +372,7 @@ TYPED_TEST(QueryPlanTest, AggregateMultipleGroupBy) {
// in this test we have 3 different properties that have different values
// for different records and assert that we get the correct combination
// of values in our groups
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto prop1 = dba.NameToProperty("prop1");
@ -401,7 +403,7 @@ TYPED_TEST(QueryPlanTest, AggregateMultipleGroupBy) {
}
TYPED_TEST(QueryPlanTest, AggregateNoInput) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -424,7 +426,7 @@ TYPED_TEST(QueryPlanTest, AggregateCountEdgeCases) {
// - 2 vertices in database, property set on one
// - 2 vertices in database, property set on both
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto prop = dba.NameToProperty("prop");
@ -476,7 +478,7 @@ TYPED_TEST(QueryPlanTest, AggregateFirstValueTypes) {
// testing exceptions that get emitted by the first-value
// type check
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
@ -528,7 +530,7 @@ TYPED_TEST(QueryPlanTest, AggregateTypes) {
// does not check all combinations that can result in an exception
// (that logic is defined and tested by TypedValue)
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto p1 = dba.NameToProperty("p1"); // has only string props
@ -581,7 +583,7 @@ TYPED_TEST(QueryPlanTest, AggregateTypes) {
}
TYPED_TEST(QueryPlanTest, Unwind) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -717,7 +719,7 @@ TYPED_TEST(QueryPlanTest, AggregateGroupByValuesWithDistinct) {
// Tests that distinct groups are aggregated properly for values of all types.
// Also test the "remember" part of the Aggregation API as final results are
// obtained via a property lookup of a remembered node.
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// a vector of memgraph::storage::PropertyValue to be set as property values on vertices
@ -782,7 +784,7 @@ TYPED_TEST(QueryPlanTest, AggregateMultipleGroupByWithDistinct) {
// in this test we have 3 different properties that have different values
// for different records and assert that we get the correct combination
// of values in our groups
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto prop1 = dba.NameToProperty("prop1");
@ -814,7 +816,7 @@ TYPED_TEST(QueryPlanTest, AggregateMultipleGroupByWithDistinct) {
}
TYPED_TEST(QueryPlanTest, AggregateNoInputWithDistinct) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -837,7 +839,7 @@ TYPED_TEST(QueryPlanTest, AggregateCountEdgeCasesWithDistinct) {
// - 2 vertices in database, property set on one
// - 2 vertices in database, property set on both
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto prop = dba.NameToProperty("prop");
@ -889,7 +891,7 @@ TYPED_TEST(QueryPlanTest, AggregateFirstValueTypesWithDistinct) {
// testing exceptions that get emitted by the first-value
// type check
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
@ -941,7 +943,7 @@ TYPED_TEST(QueryPlanTest, AggregateTypesWithDistinct) {
// does not check all combinations that can result in an exception
// (that logic is defined and tested by TypedValue)
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto p1 = dba.NameToProperty("p1"); // has only string props

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -31,6 +31,7 @@
#include "query_plan_common.hpp"
using memgraph::replication::ReplicationRole;
using namespace memgraph::query;
using namespace memgraph::query::plan;
@ -53,7 +54,7 @@ using StorageTypes = ::testing::Types<memgraph::storage::InMemoryStorage, memgra
TYPED_TEST_CASE(QueryPlanTest, StorageTypes);
TYPED_TEST(QueryPlanTest, Skip) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -81,7 +82,7 @@ TYPED_TEST(QueryPlanTest, Skip) {
}
TYPED_TEST(QueryPlanTest, Limit) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -112,7 +113,7 @@ TYPED_TEST(QueryPlanTest, CreateLimit) {
// CREATE (n), (m)
// MATCH (n) CREATE (m) LIMIT 1
// in the end we need to have 3 vertices in the db
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
dba.InsertVertex();
dba.InsertVertex();
@ -133,7 +134,7 @@ TYPED_TEST(QueryPlanTest, CreateLimit) {
}
TYPED_TEST(QueryPlanTest, OrderBy) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
auto prop = dba.NameToProperty("prop");
@ -204,7 +205,7 @@ TYPED_TEST(QueryPlanTest, OrderBy) {
}
TYPED_TEST(QueryPlanTest, OrderByMultiple) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -256,7 +257,7 @@ TYPED_TEST(QueryPlanTest, OrderByMultiple) {
}
TYPED_TEST(QueryPlanTest, OrderByExceptions) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
auto prop = dba.NameToProperty("prop");

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -38,6 +38,7 @@
using namespace memgraph::query;
using namespace memgraph::query::plan;
using memgraph::replication::ReplicationRole;
template <typename StorageType>
class QueryPlanTest : public testing::Test {
@ -58,7 +59,7 @@ using StorageTypes = ::testing::Types<memgraph::storage::InMemoryStorage, memgra
TYPED_TEST_CASE(QueryPlanTest, StorageTypes);
TYPED_TEST(QueryPlanTest, CreateNodeWithAttributes) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
memgraph::storage::LabelId label = dba.NameToLabel("Person");
@ -102,7 +103,7 @@ TYPED_TEST(QueryPlanTest, CreateNodeWithAttributes) {
TYPED_TEST(QueryPlanTest, FineGrainedCreateNodeWithAttributes) {
memgraph::license::global_license_checker.EnableTesting();
memgraph::query::SymbolTable symbol_table;
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
DbAccessor execution_dba(dba.get());
const auto label = dba->NameToLabel("label1");
const auto property = memgraph::storage::PropertyId::FromInt(1);
@ -142,7 +143,7 @@ TYPED_TEST(QueryPlanTest, FineGrainedCreateNodeWithAttributes) {
TYPED_TEST(QueryPlanTest, CreateReturn) {
// test CREATE (n:Person {age: 42}) RETURN n, n.age
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
memgraph::storage::LabelId label = dba.NameToLabel("Person");
@ -182,7 +183,7 @@ TYPED_TEST(QueryPlanTest, FineGrainedCreateReturn) {
memgraph::license::global_license_checker.EnableTesting();
// test CREATE (n:Person {age: 42}) RETURN n, n.age
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
const auto label = dba.NameToLabel("label");
@ -237,7 +238,7 @@ TYPED_TEST(QueryPlanTest, FineGrainedCreateReturn) {
#endif
TYPED_TEST(QueryPlanTest, CreateExpand) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
memgraph::storage::LabelId label_node_1 = dba.NameToLabel("Node1");
@ -315,7 +316,7 @@ TYPED_TEST(QueryPlanTest, CreateExpand) {
template <typename StorageType>
class CreateExpandWithAuthFixture : public QueryPlanTest<StorageType> {
protected:
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
SymbolTable symbol_table;
@ -445,7 +446,7 @@ TYPED_TEST(CreateExpandWithAuthFixture, CreateExpandWithCycleWithEverythingGrant
}
TYPED_TEST(QueryPlanTest, MatchCreateNode) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// add three nodes we'll match and expand-create from
@ -473,7 +474,7 @@ TYPED_TEST(QueryPlanTest, MatchCreateNode) {
template <typename StorageType>
class MatchCreateNodeWithAuthFixture : public QueryPlanTest<StorageType> {
protected:
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
SymbolTable symbol_table;
@ -553,7 +554,7 @@ TYPED_TEST(MatchCreateNodeWithAuthFixture, MatchCreateWithOneLabelDeniedThrows)
#endif
TYPED_TEST(QueryPlanTest, MatchCreateExpand) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// add three nodes we'll match and expand-create from
@ -601,7 +602,7 @@ TYPED_TEST(QueryPlanTest, MatchCreateExpand) {
template <typename StorageType>
class MatchCreateExpandWithAuthFixture : public QueryPlanTest<StorageType> {
protected:
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
SymbolTable symbol_table;
@ -746,7 +747,7 @@ TYPED_TEST(MatchCreateExpandWithAuthFixture, MatchCreateExpandWithCycleExecutesW
#endif
TYPED_TEST(QueryPlanTest, Delete) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// make a fully-connected (one-direction, no cycles) with 4 nodes
@ -818,7 +819,7 @@ TYPED_TEST(QueryPlanTest, Delete) {
template <typename StorageType>
class DeleteOperatorWithAuthFixture : public QueryPlanTest<StorageType> {
protected:
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
SymbolTable symbol_table;
@ -978,7 +979,7 @@ TYPED_TEST(QueryPlanTest, DeleteTwiceDeleteBlockingEdge) {
// MATCH (n)-[r]-(m) [DETACH] DELETE n, r, m
auto test_delete = [this](bool detach) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
@ -1013,7 +1014,7 @@ TYPED_TEST(QueryPlanTest, DeleteTwiceDeleteBlockingEdge) {
TYPED_TEST(QueryPlanTest, DeleteReturn) {
// MATCH (n) DETACH DELETE n RETURN n
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// graph with 4 vertices
@ -1047,7 +1048,7 @@ TYPED_TEST(QueryPlanTest, DeleteReturn) {
TYPED_TEST(QueryPlanTest, DeleteNull) {
// test (simplified) WITH Null as x delete x
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -1071,7 +1072,7 @@ TYPED_TEST(QueryPlanTest, DeleteAdvance) {
auto advance = std::make_shared<Accumulate>(delete_op, std::vector<Symbol>{n.sym_}, true);
auto res_sym = symbol_table.CreateSymbol("res", true);
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
dba.InsertVertex();
dba.AdvanceCommand();
@ -1080,7 +1081,7 @@ TYPED_TEST(QueryPlanTest, DeleteAdvance) {
EXPECT_EQ(1, PullAll(*produce, &context));
}
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
dba.InsertVertex();
dba.AdvanceCommand();
@ -1092,7 +1093,7 @@ TYPED_TEST(QueryPlanTest, DeleteAdvance) {
}
TYPED_TEST(QueryPlanTest, SetProperty) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// graph with 4 vertices in connected pairs
@ -1148,7 +1149,7 @@ TYPED_TEST(QueryPlanTest, SetProperty) {
TYPED_TEST(QueryPlanTest, SetProperties) {
auto test_set_properties = [this](bool update) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// graph: ({a: 0})-[:R {b:1}]->({c:2})
@ -1221,7 +1222,7 @@ TYPED_TEST(QueryPlanTest, SetProperties) {
}
TYPED_TEST(QueryPlanTest, SetLabels) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
@ -1271,7 +1272,7 @@ TYPED_TEST(QueryPlanTest, SetLabelsWithFineGrained) {
memgraph::auth::User user{"test"};
user.fine_grained_access_handler().label_permissions().Grant("*",
memgraph::auth::FineGrainedPermission::CREATE_DELETE);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
auto label2 = dba.NameToLabel("label2");
@ -1288,7 +1289,7 @@ TYPED_TEST(QueryPlanTest, SetLabelsWithFineGrained) {
{
memgraph::auth::User user{"test"};
user.fine_grained_access_handler().label_permissions().Grant("*", memgraph::auth::FineGrainedPermission::UPDATE);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
auto label2 = dba.NameToLabel("label2");
@ -1307,7 +1308,7 @@ TYPED_TEST(QueryPlanTest, SetLabelsWithFineGrained) {
user.fine_grained_access_handler().label_permissions().Grant("label3",
memgraph::auth::FineGrainedPermission::CREATE_DELETE);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
auto label2 = dba.NameToLabel("label2");
@ -1319,7 +1320,7 @@ TYPED_TEST(QueryPlanTest, SetLabelsWithFineGrained) {
#endif
TYPED_TEST(QueryPlanTest, RemoveProperty) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// graph with 4 vertices in connected pairs
@ -1381,7 +1382,7 @@ TYPED_TEST(QueryPlanTest, RemoveProperty) {
}
TYPED_TEST(QueryPlanTest, RemoveLabels) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
@ -1442,7 +1443,7 @@ TYPED_TEST(QueryPlanTest, RemoveLabelsFineGrainedFiltering) {
memgraph::auth::User user{"test"};
user.fine_grained_access_handler().label_permissions().Grant("*",
memgraph::auth::FineGrainedPermission::CREATE_DELETE);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
auto label2 = dba.NameToLabel("label2");
@ -1459,7 +1460,7 @@ TYPED_TEST(QueryPlanTest, RemoveLabelsFineGrainedFiltering) {
{
memgraph::auth::User user{"test"};
user.fine_grained_access_handler().label_permissions().Grant("*", memgraph::auth::FineGrainedPermission::UPDATE);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
auto label2 = dba.NameToLabel("label2");
@ -1478,7 +1479,7 @@ TYPED_TEST(QueryPlanTest, RemoveLabelsFineGrainedFiltering) {
user.fine_grained_access_handler().label_permissions().Grant("label3",
memgraph::auth::FineGrainedPermission::CREATE_DELETE);
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label1 = dba.NameToLabel("label1");
auto label2 = dba.NameToLabel("label2");
@ -1490,7 +1491,7 @@ TYPED_TEST(QueryPlanTest, RemoveLabelsFineGrainedFiltering) {
#endif
TYPED_TEST(QueryPlanTest, NodeFilterSet) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Create a graph such that (v1 {prop: 42}) is connected to v2 and v3.
auto v1 = dba.InsertVertex();
@ -1528,7 +1529,7 @@ TYPED_TEST(QueryPlanTest, NodeFilterSet) {
}
TYPED_TEST(QueryPlanTest, FilterRemove) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Create a graph such that (v1 {prop: 42}) is connected to v2 and v3.
auto v1 = dba.InsertVertex();
@ -1562,7 +1563,7 @@ TYPED_TEST(QueryPlanTest, FilterRemove) {
}
TYPED_TEST(QueryPlanTest, SetRemove) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v = dba.InsertVertex();
auto label1 = dba.NameToLabel("label1");
@ -1591,7 +1592,7 @@ TYPED_TEST(QueryPlanTest, Merge) {
// - merge_match branch looks for an expansion (any direction)
// and sets some property (for result validation)
// - merge_create branch just sets some other property
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -1633,7 +1634,7 @@ TYPED_TEST(QueryPlanTest, Merge) {
TYPED_TEST(QueryPlanTest, MergeNoInput) {
// merge with no input, creates a single node
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -1651,7 +1652,7 @@ TYPED_TEST(QueryPlanTest, MergeNoInput) {
TYPED_TEST(QueryPlanTest, SetPropertyWithCaching) {
// SET (Null).prop = 42
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
auto prop = PROPERTY_PAIR(dba, "property");
@ -1666,7 +1667,7 @@ TYPED_TEST(QueryPlanTest, SetPropertyWithCaching) {
TYPED_TEST(QueryPlanTest, SetPropertyOnNull) {
// SET (Null).prop = 42
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
auto prop = PROPERTY_PAIR(dba, "property");
@ -1681,7 +1682,7 @@ TYPED_TEST(QueryPlanTest, SetPropertyOnNull) {
TYPED_TEST(QueryPlanTest, SetPropertiesOnNull) {
// OPTIONAL MATCH (n) SET n = n
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
auto n = MakeScanAll(this->storage, symbol_table, "n");
@ -1694,7 +1695,7 @@ TYPED_TEST(QueryPlanTest, SetPropertiesOnNull) {
}
TYPED_TEST(QueryPlanTest, UpdateSetPropertiesFromMap) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a single vertex. ( {property: 43})
auto vertex_accessor = dba.InsertVertex();
@ -1729,7 +1730,7 @@ TYPED_TEST(QueryPlanTest, UpdateSetPropertiesFromMap) {
}
TYPED_TEST(QueryPlanTest, SetPropertiesFromMapWithCaching) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a single vertex. ({prop1: 43, prop2: 44})
@ -1769,7 +1770,7 @@ TYPED_TEST(QueryPlanTest, SetPropertiesFromMapWithCaching) {
TYPED_TEST(QueryPlanTest, SetLabelsOnNull) {
// OPTIONAL MATCH (n) SET n :label
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label = dba.NameToLabel("label");
SymbolTable symbol_table;
@ -1783,7 +1784,7 @@ TYPED_TEST(QueryPlanTest, SetLabelsOnNull) {
TYPED_TEST(QueryPlanTest, RemovePropertyOnNull) {
// REMOVE (Null).prop
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
auto prop = PROPERTY_PAIR(dba, "property");
@ -1797,7 +1798,7 @@ TYPED_TEST(QueryPlanTest, RemovePropertyOnNull) {
TYPED_TEST(QueryPlanTest, RemoveLabelsOnNull) {
// OPTIONAL MATCH (n) REMOVE n :label
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto label = dba.NameToLabel("label");
SymbolTable symbol_table;
@ -1812,7 +1813,7 @@ TYPED_TEST(QueryPlanTest, RemoveLabelsOnNull) {
TYPED_TEST(QueryPlanTest, DeleteSetProperty) {
// MATCH (n) DELETE n SET n.property = 42 RETURN n
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a single vertex.
dba.InsertVertex();
@ -1838,7 +1839,7 @@ TYPED_TEST(QueryPlanTest, DeleteSetProperty) {
TYPED_TEST(QueryPlanTest, DeleteSetPropertiesFromMap) {
// MATCH (n) DELETE n SET n = {property: 42} return n
// MATCH (n) DELETE n SET n += {property: 42} return n
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a single vertex.
dba.InsertVertex();
@ -1866,7 +1867,7 @@ TYPED_TEST(QueryPlanTest, DeleteSetPropertiesFromMap) {
TYPED_TEST(QueryPlanTest, DeleteSetPropertiesFrom) {
// MATCH (n) DELETE n SET n = n RETURN n
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a single vertex.
{
@ -1895,7 +1896,7 @@ TYPED_TEST(QueryPlanTest, DeleteSetPropertiesFrom) {
TYPED_TEST(QueryPlanTest, DeleteRemoveLabels) {
// MATCH (n) DELETE n REMOVE n :label return n
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a single vertex.
dba.InsertVertex();
@ -1920,7 +1921,7 @@ TYPED_TEST(QueryPlanTest, DeleteRemoveLabels) {
TYPED_TEST(QueryPlanTest, DeleteRemoveProperty) {
// MATCH (n) DELETE n REMOVE n.property RETURN n
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a single vertex.
dba.InsertVertex();
@ -1950,7 +1951,7 @@ TYPED_TEST(QueryPlanTest, DeleteRemoveProperty) {
template <typename StorageType>
class UpdatePropertiesWithAuthFixture : public QueryPlanTest<StorageType> {
protected:
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{this->db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
SymbolTable symbol_table;
@ -2587,7 +2588,7 @@ class DynamicExpandFixture : public testing::Test {
const std::string testSuite = "query_plan_create_set_remove_delete_dynamic_expand";
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
SymbolTable symbol_table;
AstStorage storage;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -42,6 +42,7 @@
using namespace memgraph::query;
using namespace memgraph::query::plan;
using memgraph::replication::ReplicationRole;
const std::string testSuite = "query_plan_match_filter_return";
@ -50,7 +51,7 @@ class MatchReturnFixture : public testing::Test {
protected:
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
AstStorage storage;
SymbolTable symbol_table;
@ -239,7 +240,7 @@ class QueryPlan : public testing::Test {
TYPED_TEST_CASE(QueryPlan, StorageTypes);
TYPED_TEST(QueryPlan, MatchReturnCartesian) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
ASSERT_TRUE(dba.InsertVertex().AddLabel(dba.NameToLabel("l1")).HasValue());
@ -263,7 +264,7 @@ TYPED_TEST(QueryPlan, MatchReturnCartesian) {
}
TYPED_TEST(QueryPlan, StandaloneReturn) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// add a few nodes to the database
@ -285,7 +286,7 @@ TYPED_TEST(QueryPlan, StandaloneReturn) {
}
TYPED_TEST(QueryPlan, NodeFilterLabelsAndProperties) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// add a few nodes to the database
@ -337,7 +338,7 @@ TYPED_TEST(QueryPlan, NodeFilterLabelsAndProperties) {
}
TYPED_TEST(QueryPlan, NodeFilterMultipleLabels) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// add a few nodes to the database
@ -382,7 +383,7 @@ TYPED_TEST(QueryPlan, NodeFilterMultipleLabels) {
}
TYPED_TEST(QueryPlan, Cartesian) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto add_vertex = [&dba](std::string label) {
@ -419,7 +420,7 @@ TYPED_TEST(QueryPlan, Cartesian) {
}
TYPED_TEST(QueryPlan, CartesianEmptySet) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -439,7 +440,7 @@ TYPED_TEST(QueryPlan, CartesianEmptySet) {
}
TYPED_TEST(QueryPlan, CartesianThreeWay) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto add_vertex = [&dba](std::string label) {
auto vertex = dba.InsertVertex();
@ -489,7 +490,7 @@ class ExpandFixture : public testing::Test {
protected:
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
SymbolTable symbol_table;
AstStorage storage;
@ -649,7 +650,7 @@ class QueryPlanExpandVariable : public testing::Test {
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
// labels for layers in the double chain
std::vector<memgraph::storage::LabelId> labels;
@ -1804,7 +1805,7 @@ class QueryPlanExpandWeightedShortestPath : public testing::Test {
protected:
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
std::pair<std::string, memgraph::storage::PropertyId> prop = PROPERTY_PAIR(dba, "property");
memgraph::storage::EdgeTypeId edge_type = dba.NameToEdgeType("edge_type");
@ -2248,7 +2249,7 @@ class QueryPlanExpandAllShortestPaths : public testing::Test {
protected:
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
std::pair<std::string, memgraph::storage::PropertyId> prop = PROPERTY_PAIR(dba, "property");
memgraph::storage::EdgeTypeId edge_type = dba.NameToEdgeType("edge_type");
@ -2699,7 +2700,7 @@ TYPED_TEST(QueryPlanExpandAllShortestPaths, BasicWithFineGrainedFiltering) {
#endif
TYPED_TEST(QueryPlan, ExpandOptional) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -2750,7 +2751,7 @@ TYPED_TEST(QueryPlan, ExpandOptional) {
}
TYPED_TEST(QueryPlan, OptionalMatchEmptyDB) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -2767,7 +2768,7 @@ TYPED_TEST(QueryPlan, OptionalMatchEmptyDB) {
}
TYPED_TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
// OPTIONAL MATCH (n)
@ -2790,7 +2791,7 @@ TYPED_TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
}
TYPED_TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Make a graph with 2 connected, unlabeled nodes.
auto v1 = dba.InsertVertex();
@ -2834,7 +2835,7 @@ TYPED_TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
}
TYPED_TEST(QueryPlan, ExpandExistingNode) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// make a graph (v1)->(v2) that
@ -2872,7 +2873,7 @@ TYPED_TEST(QueryPlan, ExpandExistingNode) {
TYPED_TEST(QueryPlan, ExpandBothCycleEdgeCase) {
// we're testing that expanding on BOTH
// does only one expansion for a cycle
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v = dba.InsertVertex();
@ -2889,7 +2890,7 @@ TYPED_TEST(QueryPlan, ExpandBothCycleEdgeCase) {
}
TYPED_TEST(QueryPlan, EdgeFilter) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// make an N-star expanding from (v1)
@ -2949,7 +2950,7 @@ TYPED_TEST(QueryPlan, EdgeFilter) {
}
TYPED_TEST(QueryPlan, EdgeFilterMultipleTypes) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
@ -2979,7 +2980,7 @@ TYPED_TEST(QueryPlan, EdgeFilterMultipleTypes) {
}
TYPED_TEST(QueryPlan, Filter) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// add a 6 nodes with property 'prop', 2 have true as value
@ -3003,7 +3004,7 @@ TYPED_TEST(QueryPlan, Filter) {
}
TYPED_TEST(QueryPlan, EdgeUniquenessFilter) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// make a graph that has (v1)->(v2) and a recursive edge (v1)->(v1)
@ -3038,7 +3039,7 @@ TYPED_TEST(QueryPlan, Distinct) {
// test queries like
// UNWIND [1, 2, 3, 3] AS x RETURN DISTINCT x
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
SymbolTable symbol_table;
@ -3082,11 +3083,11 @@ TYPED_TEST(QueryPlan, Distinct) {
TYPED_TEST(QueryPlan, ScanAllByLabel) {
auto label = this->db->NameToLabel("label");
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Add a vertex with a label and one without.
auto labeled_vertex = dba.InsertVertex();
@ -3133,7 +3134,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabelProperty) {
memgraph::storage::PropertyValue(
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(2)})};
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
for (const auto &value : values) {
auto vertex = dba.InsertVertex();
@ -3144,12 +3145,12 @@ TYPED_TEST(QueryPlan, ScanAllByLabelProperty) {
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
ASSERT_EQ(14, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
@ -3243,7 +3244,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
auto label = this->db->NameToLabel("label");
auto prop = this->db->NameToProperty("prop");
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto number_vertex = dba.InsertVertex();
ASSERT_TRUE(number_vertex.AddLabel(label).HasValue());
@ -3254,12 +3255,12 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
ASSERT_FALSE(dba.Commit().HasError());
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
EXPECT_EQ(2, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
// MATCH (n :label {prop: 42})
@ -3283,7 +3284,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyValueError) {
auto label = this->db->NameToLabel("label");
auto prop = this->db->NameToProperty("prop");
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
for (int i = 0; i < 2; ++i) {
auto vertex = dba.InsertVertex();
@ -3293,12 +3294,12 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyValueError) {
ASSERT_FALSE(dba.Commit().HasError());
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
EXPECT_EQ(2, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
// MATCH (m), (n :label {prop: m})
@ -3316,7 +3317,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
auto label = this->db->NameToLabel("label");
auto prop = this->db->NameToProperty("prop");
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
for (int i = 0; i < 2; ++i) {
auto vertex = dba.InsertVertex();
@ -3326,12 +3327,12 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
ASSERT_FALSE(dba.Commit().HasError());
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
EXPECT_EQ(2, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
// MATCH (m), (n :label {prop: m})
@ -3372,7 +3373,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) {
auto label = this->db->NameToLabel("label");
auto prop = this->db->NameToProperty("prop");
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto vertex = dba.InsertVertex();
ASSERT_TRUE(vertex.AddLabel(label).HasValue());
@ -3382,12 +3383,12 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) {
ASSERT_FALSE(dba.Commit().HasError());
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
EXPECT_EQ(2, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
// MATCH (n :label {prop: 42})
@ -3409,7 +3410,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
auto label = this->db->NameToLabel("label");
auto prop = this->db->NameToProperty("prop");
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto vertex = dba.InsertVertex();
ASSERT_TRUE(vertex.AddLabel(label).HasValue());
@ -3419,12 +3420,12 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
ASSERT_FALSE(dba.Commit().HasError());
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
EXPECT_EQ(2, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
// MATCH (n :label) WHERE null <= n.prop < null
@ -3444,7 +3445,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) {
auto label = this->db->NameToLabel("label");
auto prop = this->db->NameToProperty("prop");
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v = dba.InsertVertex();
ASSERT_TRUE(v.AddLabel(label).HasValue());
@ -3452,12 +3453,12 @@ TYPED_TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) {
ASSERT_FALSE(dba.Commit().HasError());
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
EXPECT_EQ(1, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
@ -3487,7 +3488,7 @@ TYPED_TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
const int prop_value1 = 42, prop_value2 = 69;
for (int i = 0; i < vertex_count; ++i) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v = dba.InsertVertex();
ASSERT_TRUE(v.AddLabel(label).HasValue());
@ -3497,14 +3498,14 @@ TYPED_TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
}
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
// Make sure there are `vertex_count` vertices
{
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
EXPECT_EQ(vertex_count, CountIterable(dba.Vertices(memgraph::storage::View::OLD)));
}
@ -3512,7 +3513,7 @@ TYPED_TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
// Make sure there are `vertex_prop_count` results when using index
auto count_with_index = [this, &label, &prop](int prop_value, int prop_count) {
SymbolTable symbol_table;
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto scan_all_by_label_property_value =
MakeScanAllByLabelPropertyValue(this->storage, symbol_table, "n", label, prop, "prop", LITERAL(prop_value));
@ -3526,7 +3527,7 @@ TYPED_TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
// Make sure there are `vertex_count` results when using scan all
auto count_with_scan_all = [this, &prop](int prop_value, int prop_count) {
SymbolTable symbol_table;
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto scan_all = MakeScanAll(this->storage, symbol_table, "n");
auto e = PROPERTY_LOOKUP(dba, IDENT("n")->MapTo(scan_all.sym_), std::make_pair("prop", prop));
@ -3551,7 +3552,7 @@ class ExistsFixture : public testing::Test {
protected:
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
AstStorage storage;
SymbolTable symbol_table;
@ -3779,7 +3780,7 @@ class SubqueriesFeature : public testing::Test {
protected:
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access(ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
AstStorage storage;
SymbolTable symbol_table;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -37,7 +37,7 @@ class OperatorToStringTest : public ::testing::Test {
OperatorToStringTest()
: config(disk_test_utils::GenerateOnDiskConfig(testSuite)),
db(new StorageType(config)),
dba_storage(db->Access()),
dba_storage(db->Access(memgraph::replication::ReplicationRole::MAIN)),
dba(dba_storage.get()) {}
~OperatorToStringTest() override {

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -36,7 +36,8 @@ class ReadWriteTypeCheckTest : public ::testing::Test {
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> dba_storage{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> dba_storage{
db->Access(memgraph::replication::ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{dba_storage.get()};
void TearDown() override {

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -18,6 +18,7 @@
#include "query/plan/operator.hpp"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/inmemory/storage.hpp"
using memgraph::replication::ReplicationRole;
template <typename StorageType>
class QueryPlan : public testing::Test {
@ -37,7 +38,7 @@ using StorageTypes = ::testing::Types<memgraph::storage::InMemoryStorage, memgra
TYPED_TEST_CASE(QueryPlan, StorageTypes);
TYPED_TEST(QueryPlan, CreateNodeWithAttributes) {
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
auto label = memgraph::storage::LabelId::FromInt(42);
auto property = memgraph::storage::PropertyId::FromInt(1);
@ -75,7 +76,7 @@ TYPED_TEST(QueryPlan, CreateNodeWithAttributes) {
TYPED_TEST(QueryPlan, ScanAllEmpty) {
memgraph::query::AstStorage ast;
memgraph::query::SymbolTable symbol_table;
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
DbAccessor execution_dba(dba.get());
auto node_symbol = symbol_table.CreateSymbol("n", true);
{
@ -100,13 +101,13 @@ TYPED_TEST(QueryPlan, ScanAllEmpty) {
TYPED_TEST(QueryPlan, ScanAll) {
{
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
for (int i = 0; i < 42; ++i) dba->CreateVertex();
EXPECT_FALSE(dba->Commit().HasError());
}
memgraph::query::AstStorage ast;
memgraph::query::SymbolTable symbol_table;
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
DbAccessor execution_dba(dba.get());
auto node_symbol = symbol_table.CreateSymbol("n", true);
memgraph::query::plan::ScanAll scan_all(nullptr, node_symbol);
@ -121,12 +122,12 @@ TYPED_TEST(QueryPlan, ScanAll) {
TYPED_TEST(QueryPlan, ScanAllByLabel) {
auto label = this->db->NameToLabel("label");
{
auto unique_acc = this->db->UniqueAccess();
auto unique_acc = this->db->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
// Add some unlabeled vertices
for (int i = 0; i < 12; ++i) dba->CreateVertex();
// Add labeled vertices
@ -136,7 +137,7 @@ TYPED_TEST(QueryPlan, ScanAllByLabel) {
}
EXPECT_FALSE(dba->Commit().HasError());
}
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::AstStorage ast;
memgraph::query::SymbolTable symbol_table;
auto node_symbol = symbol_table.CreateSymbol("n", true);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -36,7 +36,8 @@ class ExpressionPrettyPrinterTest : public ::testing::Test {
const std::string testSuite = "query_pretty_print";
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{
db->Access(memgraph::replication::ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
AstStorage storage;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -23,6 +23,8 @@
#include "disk_test_utils.hpp"
#include "test_utils.hpp"
using memgraph::replication::ReplicationRole;
template <typename StorageType>
class CypherType : public testing::Test {
public:
@ -244,7 +246,7 @@ TYPED_TEST(CypherType, MapSatisfiesType) {
}
TYPED_TEST(CypherType, VertexSatisfiesType) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto vertex = dba.InsertVertex();
mgp_memory memory{memgraph::utils::NewDeleteResource()};
@ -267,7 +269,7 @@ TYPED_TEST(CypherType, VertexSatisfiesType) {
}
TYPED_TEST(CypherType, EdgeSatisfiesType) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -291,7 +293,7 @@ TYPED_TEST(CypherType, EdgeSatisfiesType) {
}
TYPED_TEST(CypherType, PathSatisfiesType) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -21,6 +21,8 @@
#include "storage/v2/inmemory/storage.hpp"
#include "test_utils.hpp"
using memgraph::replication::ReplicationRole;
template <typename StorageType>
class PyModule : public testing::Test {
public:
@ -116,7 +118,7 @@ static void AssertPickleAndCopyAreNotSupported(PyObject *py_obj) {
TYPED_TEST(PyModule, PyVertex) {
// Initialize the database with 2 vertices and 1 edge.
{
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
auto v1 = dba->CreateVertex();
auto v2 = dba->CreateVertex();
@ -129,7 +131,7 @@ TYPED_TEST(PyModule, PyVertex) {
ASSERT_FALSE(dba->Commit().HasError());
}
// Get the first vertex as an mgp_value.
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
mgp_memory memory{memgraph::utils::NewDeleteResource()};
mgp_graph graph{&dba, memgraph::storage::View::OLD, nullptr, dba.GetStorageMode()};
@ -165,7 +167,7 @@ TYPED_TEST(PyModule, PyVertex) {
TYPED_TEST(PyModule, PyEdge) {
// Initialize the database with 2 vertices and 1 edge.
{
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
auto v1 = dba->CreateVertex();
auto v2 = dba->CreateVertex();
@ -179,7 +181,7 @@ TYPED_TEST(PyModule, PyEdge) {
ASSERT_FALSE(dba->Commit().HasError());
}
// Get the edge as an mgp_value.
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
mgp_memory memory{memgraph::utils::NewDeleteResource()};
mgp_graph graph{&dba, memgraph::storage::View::OLD, nullptr, dba.GetStorageMode()};
@ -219,13 +221,13 @@ TYPED_TEST(PyModule, PyEdge) {
TYPED_TEST(PyModule, PyPath) {
{
auto dba = this->db->Access();
auto dba = this->db->Access(ReplicationRole::MAIN);
auto v1 = dba->CreateVertex();
auto v2 = dba->CreateVertex();
ASSERT_TRUE(dba->CreateEdge(&v1, &v2, dba->NameToEdgeType("type")).HasValue());
ASSERT_FALSE(dba->Commit().HasError());
}
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
mgp_memory memory{memgraph::utils::NewDeleteResource()};
mgp_graph graph{&dba, memgraph::storage::View::OLD, nullptr, dba.GetStorageMode()};

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -34,6 +34,8 @@
#include "utils/memory.hpp"
#include "utils/variant_helpers.hpp"
using memgraph::replication::ReplicationRole;
#define EXPECT_SUCCESS(...) EXPECT_EQ(__VA_ARGS__, mgp_error::MGP_ERROR_NO_ERROR)
namespace {
@ -155,7 +157,7 @@ class MgpGraphTest : public ::testing::Test {
}
memgraph::query::DbAccessor &CreateDbAccessor(const memgraph::storage::IsolationLevel isolationLevel) {
accessors_.push_back(storage->Access(isolationLevel));
accessors_.push_back(storage->Access(ReplicationRole::MAIN, isolationLevel));
db_accessors_.emplace_back(accessors_.back().get());
return db_accessors_.back();
}
@ -194,7 +196,8 @@ TYPED_TEST(MgpGraphTest, CreateVertex) {
return;
}
mgp_graph graph = this->CreateGraph();
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
EXPECT_EQ(CountVertices(*read_uncommited_accessor, memgraph::storage::View::NEW), 0);
MgpVertexPtr vertex{EXPECT_MGP_NO_ERROR(mgp_vertex *, mgp_graph_create_vertex, &graph, &this->memory)};
EXPECT_NE(vertex, nullptr);
@ -217,7 +220,8 @@ TYPED_TEST(MgpGraphTest, DeleteVertex) {
ASSERT_FALSE(accessor.Commit().HasError());
}
mgp_graph graph = this->CreateGraph();
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
EXPECT_EQ(CountVertices(*read_uncommited_accessor, memgraph::storage::View::NEW), 1);
MgpVertexPtr vertex{EXPECT_MGP_NO_ERROR(mgp_vertex *, mgp_graph_get_vertex_by_id, &graph,
mgp_vertex_id{vertex_id.AsInt()}, &this->memory)};
@ -233,7 +237,8 @@ TYPED_TEST(MgpGraphTest, DetachDeleteVertex) {
}
const auto vertex_ids = this->CreateEdge();
auto graph = this->CreateGraph();
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
EXPECT_EQ(CountVertices(*read_uncommited_accessor, memgraph::storage::View::NEW), 2);
MgpVertexPtr vertex{EXPECT_MGP_NO_ERROR(mgp_vertex *, mgp_graph_get_vertex_by_id, &graph,
mgp_vertex_id{vertex_ids.front().AsInt()}, &this->memory)};
@ -255,7 +260,8 @@ TYPED_TEST(MgpGraphTest, CreateDeleteWithImmutableGraph) {
vertex_id = vertex.Gid();
ASSERT_FALSE(accessor.Commit().HasError());
}
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
EXPECT_EQ(CountVertices(*read_uncommited_accessor, memgraph::storage::View::NEW), 1);
mgp_graph immutable_graph = this->CreateGraph(memgraph::storage::View::OLD);
@ -326,7 +332,8 @@ TYPED_TEST(MgpGraphTest, VertexSetProperty) {
ASSERT_TRUE(result.HasValue());
ASSERT_FALSE(accessor.Commit().HasError());
}
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
EXPECT_EQ(CountVertices(*read_uncommited_accessor, memgraph::storage::View::NEW), 1);
mgp_graph graph = this->CreateGraph(memgraph::storage::View::NEW);
@ -396,7 +403,8 @@ TYPED_TEST(MgpGraphTest, VertexAddLabel) {
auto check_label = [&]() {
EXPECT_NE(EXPECT_MGP_NO_ERROR(int, mgp_vertex_has_label_named, vertex.get(), label.data()), 0);
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
const auto maybe_vertex = read_uncommited_accessor->FindVertex(vertex_id, memgraph::storage::View::NEW);
ASSERT_TRUE(maybe_vertex);
const auto label_ids = maybe_vertex->Labels(memgraph::storage::View::NEW);
@ -433,7 +441,8 @@ TYPED_TEST(MgpGraphTest, VertexRemoveLabel) {
auto check_label = [&]() {
EXPECT_EQ(EXPECT_MGP_NO_ERROR(int, mgp_vertex_has_label_named, vertex.get(), label.data()), 0);
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
const auto maybe_vertex = read_uncommited_accessor->FindVertex(vertex_id, memgraph::storage::View::NEW);
ASSERT_TRUE(maybe_vertex);
const auto label_ids = maybe_vertex->Labels(memgraph::storage::View::NEW);
@ -622,14 +631,15 @@ TYPED_TEST(MgpGraphTest, EdgeSetProperty) {
{
const auto vertex_ids = this->CreateEdge();
from_vertex_id = vertex_ids[0];
auto accessor = this->storage->Access(memgraph::storage::IsolationLevel::SNAPSHOT_ISOLATION);
auto accessor = this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::SNAPSHOT_ISOLATION);
auto edge = get_edge(accessor.get());
const auto result =
edge.SetProperty(accessor->NameToProperty(property_to_update), memgraph::storage::PropertyValue(42));
ASSERT_TRUE(result.HasValue());
ASSERT_FALSE(accessor->Commit().HasError());
}
auto read_uncommited_accessor = this->storage->Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
auto read_uncommited_accessor =
this->storage->Access(ReplicationRole::MAIN, memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
mgp_graph graph = this->CreateGraph(memgraph::storage::View::NEW);
MgpEdgePtr edge;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -34,7 +34,8 @@ class TestSymbolGenerator : public ::testing::Test {
const std::string testSuite = "query_semantic";
memgraph::storage::Config config = disk_test_utils::GenerateOnDiskConfig(testSuite);
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{
db->Access(memgraph::replication::ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
AstStorage storage;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -29,6 +29,8 @@
#include "utils/exceptions.hpp"
#include "utils/memory.hpp"
using memgraph::replication::ReplicationRole;
namespace {
const std::unordered_set<memgraph::query::TriggerEventType> kAllEventTypes{
memgraph::query::TriggerEventType::ANY, memgraph::query::TriggerEventType::VERTEX_CREATE,
@ -69,7 +71,7 @@ class TriggerContextTest : public ::testing::Test {
}
memgraph::storage::Storage::Accessor *StartTransaction() {
accessors.emplace_back(db->Access());
accessors.emplace_back(db->Access(ReplicationRole::MAIN));
return accessors.back().get();
}
@ -902,7 +904,7 @@ class TriggerStoreTest : public ::testing::Test {
config = disk_test_utils::GenerateOnDiskConfig(testSuite);
storage = std::make_unique<StorageType>(config);
storage_accessor = storage->Access();
storage_accessor = storage->Access(ReplicationRole::MAIN);
dba.emplace(storage_accessor.get());
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -28,6 +28,7 @@
#include "formatters.hpp"
using memgraph::replication::ReplicationRole;
using namespace memgraph::query::plan;
using memgraph::query::AstStorage;
using Type = memgraph::query::EdgeAtom::Type;
@ -110,7 +111,7 @@ using StorageTypes = ::testing::Types<memgraph::storage::InMemoryStorage, memgra
TYPED_TEST_CASE(TestVariableStartPlanner, StorageTypes);
TYPED_TEST(TestVariableStartPlanner, MatchReturn) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Make a graph (v1) -[:r]-> (v2)
auto v1 = dba.InsertVertex();
@ -127,7 +128,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchReturn) {
}
TYPED_TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3)
auto v1 = dba.InsertVertex();
@ -159,7 +160,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
}
TYPED_TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3)
auto v1 = dba.InsertVertex();
@ -186,7 +187,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
}
TYPED_TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Graph (v1) -[:r]-> (v2)
memgraph::query::VertexAccessor v1(dba.InsertVertex());
@ -210,7 +211,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
}
TYPED_TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Graph (v1) -[:r]-> (v2)
memgraph::query::VertexAccessor v1(dba.InsertVertex());
@ -230,7 +231,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
}
TYPED_TEST(TestVariableStartPlanner, MatchVariableExpand) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
// Graph (v1) -[:r1]-> (v2) -[:r2]-> (v3)
auto v1 = dba.InsertVertex();
@ -253,7 +254,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchVariableExpand) {
}
TYPED_TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto id = dba.NameToProperty("id");
// Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3})
@ -281,7 +282,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
}
TYPED_TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto id = dba.NameToProperty("id");
// Graph (v1 {id:1}) -[:r1]-> (v2) -[:r2]-> (v3)
@ -307,7 +308,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
}
TYPED_TEST(TestVariableStartPlanner, MatchBfs) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto id = dba.NameToProperty("id");
// Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3})
@ -334,7 +335,7 @@ TYPED_TEST(TestVariableStartPlanner, MatchBfs) {
}
TYPED_TEST(TestVariableStartPlanner, TestBasicSubquery) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
@ -356,7 +357,7 @@ TYPED_TEST(TestVariableStartPlanner, TestBasicSubquery) {
}
TYPED_TEST(TestVariableStartPlanner, TestBasicSubqueryWithMatching) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto v1 = dba.InsertVertex();
@ -377,7 +378,7 @@ TYPED_TEST(TestVariableStartPlanner, TestBasicSubqueryWithMatching) {
}
TYPED_TEST(TestVariableStartPlanner, TestSubqueryWithUnion) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto id = dba.NameToProperty("id");
@ -405,7 +406,7 @@ TYPED_TEST(TestVariableStartPlanner, TestSubqueryWithUnion) {
}
TYPED_TEST(TestVariableStartPlanner, TestSubqueryWithTripleUnion) {
auto storage_dba = this->db->Access();
auto storage_dba = this->db->Access(ReplicationRole::MAIN);
memgraph::query::DbAccessor dba(storage_dba.get());
auto id = dba.NameToProperty("id");

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -30,6 +30,8 @@
#include "storage/v2/view.hpp"
#include "utils/rocksdb_serialization.hpp"
using memgraph::replication::ReplicationRole;
// NOLINTNEXTLINE(google-build-using-namespace)
using namespace memgraph::storage;
@ -57,14 +59,14 @@ class RocksDBStorageTest : public ::testing::TestWithParam<bool> {
};
TEST_F(RocksDBStorageTest, SerializeVertexGID) {
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
auto gid = vertex.Gid();
ASSERT_EQ(memgraph::utils::SerializeVertex(*vertex.vertex_), "|" + gid.ToString());
}
TEST_F(RocksDBStorageTest, SerializeVertexGIDLabels) {
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
auto ser_player_label = acc->NameToLabel("Player");
auto ser_user_label = acc->NameToLabel("User");

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -23,6 +23,8 @@
#include "storage/v2/vertex_accessor.hpp"
#include "storage_test_utils.hpp"
using memgraph::replication::ReplicationRole;
using testing::Types;
using testing::UnorderedElementsAre;
@ -53,7 +55,7 @@ TYPED_TEST_CASE(StorageV2Test, StorageTypes);
TYPED_TEST(StorageV2Test, Commit) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -63,7 +65,7 @@ TYPED_TEST(StorageV2Test, Commit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
ASSERT_TRUE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
EXPECT_EQ(CountVertices(*acc, memgraph::storage::View::OLD), 1U);
ASSERT_TRUE(acc->FindVertex(gid, memgraph::storage::View::NEW).has_value());
@ -71,7 +73,7 @@ TYPED_TEST(StorageV2Test, Commit) {
acc->Abort();
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -87,7 +89,7 @@ TYPED_TEST(StorageV2Test, Commit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
EXPECT_EQ(CountVertices(*acc, memgraph::storage::View::OLD), 0U);
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::NEW).has_value());
@ -100,7 +102,7 @@ TYPED_TEST(StorageV2Test, Commit) {
TYPED_TEST(StorageV2Test, Abort) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -110,7 +112,7 @@ TYPED_TEST(StorageV2Test, Abort) {
acc->Abort();
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
EXPECT_EQ(CountVertices(*acc, memgraph::storage::View::OLD), 0U);
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::NEW).has_value());
@ -124,7 +126,7 @@ TYPED_TEST(StorageV2Test, AdvanceCommandCommit) {
memgraph::storage::Gid gid1 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
memgraph::storage::Gid gid2 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
gid1 = vertex1.Gid();
@ -148,7 +150,7 @@ TYPED_TEST(StorageV2Test, AdvanceCommandCommit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
ASSERT_TRUE(acc->FindVertex(gid1, memgraph::storage::View::OLD).has_value());
ASSERT_TRUE(acc->FindVertex(gid1, memgraph::storage::View::NEW).has_value());
ASSERT_TRUE(acc->FindVertex(gid2, memgraph::storage::View::OLD).has_value());
@ -164,7 +166,7 @@ TYPED_TEST(StorageV2Test, AdvanceCommandAbort) {
memgraph::storage::Gid gid1 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
memgraph::storage::Gid gid2 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
gid1 = vertex1.Gid();
@ -188,7 +190,7 @@ TYPED_TEST(StorageV2Test, AdvanceCommandAbort) {
acc->Abort();
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
ASSERT_FALSE(acc->FindVertex(gid1, memgraph::storage::View::OLD).has_value());
ASSERT_FALSE(acc->FindVertex(gid1, memgraph::storage::View::NEW).has_value());
ASSERT_FALSE(acc->FindVertex(gid2, memgraph::storage::View::OLD).has_value());
@ -201,8 +203,8 @@ TYPED_TEST(StorageV2Test, AdvanceCommandAbort) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(StorageV2Test, SnapshotIsolation) {
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc1->CreateVertex();
auto gid = vertex.Gid();
@ -223,7 +225,7 @@ TYPED_TEST(StorageV2Test, SnapshotIsolation) {
acc2->Abort();
auto acc3 = this->store->Access();
auto acc3 = this->store->Access(ReplicationRole::MAIN);
ASSERT_TRUE(acc3->FindVertex(gid, memgraph::storage::View::OLD).has_value());
EXPECT_EQ(CountVertices(*acc3, memgraph::storage::View::OLD), 1U);
ASSERT_TRUE(acc3->FindVertex(gid, memgraph::storage::View::NEW).has_value());
@ -235,7 +237,7 @@ TYPED_TEST(StorageV2Test, SnapshotIsolation) {
TYPED_TEST(StorageV2Test, AccessorMove) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
@ -254,7 +256,7 @@ TYPED_TEST(StorageV2Test, AccessorMove) {
ASSERT_FALSE(moved->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
ASSERT_TRUE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
EXPECT_EQ(CountVertices(*acc, memgraph::storage::View::OLD), 1U);
ASSERT_TRUE(acc->FindVertex(gid, memgraph::storage::View::NEW).has_value());
@ -266,8 +268,8 @@ TYPED_TEST(StorageV2Test, AccessorMove) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(StorageV2Test, VertexDeleteCommit) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
auto acc1 = this->store->Access(); // read transaction
auto acc2 = this->store->Access(); // write transaction
auto acc1 = this->store->Access(ReplicationRole::MAIN); // read transaction
auto acc2 = this->store->Access(ReplicationRole::MAIN); // write transaction
// Create the vertex in transaction 2
{
@ -280,8 +282,8 @@ TYPED_TEST(StorageV2Test, VertexDeleteCommit) {
ASSERT_FALSE(acc2->Commit().HasError());
}
auto acc3 = this->store->Access(); // read transaction
auto acc4 = this->store->Access(); // write transaction
auto acc3 = this->store->Access(ReplicationRole::MAIN); // read transaction
auto acc4 = this->store->Access(ReplicationRole::MAIN); // write transaction
// Check whether the vertex exists in transaction 1
ASSERT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -314,7 +316,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteCommit) {
ASSERT_FALSE(acc4->Commit().HasError());
}
auto acc5 = this->store->Access(); // read transaction
auto acc5 = this->store->Access(ReplicationRole::MAIN); // read transaction
// Check whether the vertex exists in transaction 1
ASSERT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -339,8 +341,8 @@ TYPED_TEST(StorageV2Test, VertexDeleteCommit) {
TYPED_TEST(StorageV2Test, VertexDeleteAbort) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
auto acc1 = this->store->Access(); // read transaction
auto acc2 = this->store->Access(); // write transaction
auto acc1 = this->store->Access(ReplicationRole::MAIN); // read transaction
auto acc2 = this->store->Access(ReplicationRole::MAIN); // write transaction
// Create the vertex in transaction 2
{
@ -353,8 +355,8 @@ TYPED_TEST(StorageV2Test, VertexDeleteAbort) {
ASSERT_FALSE(acc2->Commit().HasError());
}
auto acc3 = this->store->Access(); // read transaction
auto acc4 = this->store->Access(); // write transaction (aborted)
auto acc3 = this->store->Access(ReplicationRole::MAIN); // read transaction
auto acc4 = this->store->Access(ReplicationRole::MAIN); // write transaction (aborted)
// Check whether the vertex exists in transaction 1
ASSERT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -387,8 +389,8 @@ TYPED_TEST(StorageV2Test, VertexDeleteAbort) {
acc4->Abort();
}
auto acc5 = this->store->Access(); // read transaction
auto acc6 = this->store->Access(); // write transaction
auto acc5 = this->store->Access(ReplicationRole::MAIN); // read transaction
auto acc6 = this->store->Access(ReplicationRole::MAIN); // write transaction
// Check whether the vertex exists in transaction 1
ASSERT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -427,7 +429,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteAbort) {
ASSERT_FALSE(acc6->Commit().HasError());
}
auto acc7 = this->store->Access(); // read transaction
auto acc7 = this->store->Access(ReplicationRole::MAIN); // read transaction
// Check whether the vertex exists in transaction 1
ASSERT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -466,14 +468,14 @@ TYPED_TEST(StorageV2Test, VertexDeleteSerializationError) {
// Create vertex
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
}
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
// Delete vertex in accessor 1
{
@ -546,7 +548,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteSerializationError) {
// Check whether the vertex exists
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_FALSE(vertex);
EXPECT_EQ(CountVertices(*acc, memgraph::storage::View::OLD), 0U);
@ -563,7 +565,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteSpecialCases) {
// Create vertex and delete it in the same transaction, but abort the
// transaction
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid1 = vertex.Gid();
ASSERT_FALSE(acc->FindVertex(gid1, memgraph::storage::View::OLD).has_value());
@ -583,7 +585,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteSpecialCases) {
// Create vertex and delete it in the same transaction
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid2 = vertex.Gid();
ASSERT_FALSE(acc->FindVertex(gid2, memgraph::storage::View::OLD).has_value());
@ -603,7 +605,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteSpecialCases) {
// Check whether the vertices exist
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
ASSERT_FALSE(acc->FindVertex(gid1, memgraph::storage::View::OLD).has_value());
ASSERT_FALSE(acc->FindVertex(gid1, memgraph::storage::View::NEW).has_value());
ASSERT_FALSE(acc->FindVertex(gid2, memgraph::storage::View::OLD).has_value());
@ -620,7 +622,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteLabel) {
// Create the vertex
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -630,7 +632,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteLabel) {
// Add label, delete the vertex and check the label API (same command)
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -684,7 +686,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteLabel) {
// Add label, delete the vertex and check the label API (different command)
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -775,7 +777,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteProperty) {
// Create the vertex
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->FindVertex(gid, memgraph::storage::View::OLD).has_value());
@ -785,7 +787,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteProperty) {
// Set property, delete the vertex and check the property API (same command)
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -833,7 +835,7 @@ TYPED_TEST(StorageV2Test, VertexDeleteProperty) {
// Set property, delete the vertex and check the property API (different
// command)
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -916,7 +918,7 @@ TYPED_TEST(StorageV2Test, VertexLabelCommit) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
@ -948,7 +950,7 @@ TYPED_TEST(StorageV2Test, VertexLabelCommit) {
spdlog::debug("Commit done");
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -977,7 +979,7 @@ TYPED_TEST(StorageV2Test, VertexLabelCommit) {
spdlog::debug("Abort done");
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1009,7 +1011,7 @@ TYPED_TEST(StorageV2Test, VertexLabelCommit) {
spdlog::debug("Commit done");
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1036,7 +1038,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Create the vertex.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -1044,7 +1046,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Add label 5, but abort the transaction.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1077,7 +1079,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Check that label 5 doesn't exist.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1098,7 +1100,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Add label 5.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1131,7 +1133,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Check that label 5 exists.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1161,7 +1163,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Remove label 5, but abort the transaction.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1194,7 +1196,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Check that label 5 exists.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1224,7 +1226,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Remove label 5.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1257,7 +1259,7 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
// Check that label 5 doesn't exist.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1281,14 +1283,14 @@ TYPED_TEST(StorageV2Test, VertexLabelAbort) {
TYPED_TEST(StorageV2Test, VertexLabelSerializationError) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
}
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
// Add label 1 in accessor 1.
{
@ -1371,7 +1373,7 @@ TYPED_TEST(StorageV2Test, VertexLabelSerializationError) {
// Check which labels exist.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1402,7 +1404,7 @@ TYPED_TEST(StorageV2Test, VertexLabelSerializationError) {
TYPED_TEST(StorageV2Test, VertexPropertyCommit) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
@ -1440,7 +1442,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyCommit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1468,7 +1470,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyCommit) {
acc->Abort();
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1499,7 +1501,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyCommit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1525,7 +1527,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Create the vertex.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -1533,7 +1535,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Set property 5 to "nandare", but abort the transaction.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1573,7 +1575,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Check that property 5 is null.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1594,7 +1596,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Set property 5 to "nandare".
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1634,7 +1636,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Check that property 5 is "nandare".
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1664,7 +1666,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Set property 5 to null, but abort the transaction.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1705,7 +1707,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Check that property 5 is "nandare".
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1735,7 +1737,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Set property 5 to null.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1776,7 +1778,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
// Check that property 5 is null.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1800,14 +1802,14 @@ TYPED_TEST(StorageV2Test, VertexPropertyAbort) {
TYPED_TEST(StorageV2Test, VertexPropertySerializationError) {
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
}
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
// Set property 1 to 123 in accessor 1.
{
@ -1884,7 +1886,7 @@ TYPED_TEST(StorageV2Test, VertexPropertySerializationError) {
// Check which properties exist.
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1913,7 +1915,7 @@ TYPED_TEST(StorageV2Test, VertexPropertySerializationError) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(StorageV2Test, VertexLabelPropertyMixed) {
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
auto label = acc->NameToLabel("label5");
@ -2155,7 +2157,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyClear) {
auto property1 = this->store->NameToProperty("property1");
auto property2 = this->store->NameToProperty("property2");
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
@ -2166,7 +2168,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -2198,7 +2200,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyClear) {
acc->Abort();
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -2209,7 +2211,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -2242,7 +2244,7 @@ TYPED_TEST(StorageV2Test, VertexPropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -2258,7 +2260,7 @@ TYPED_TEST(StorageV2Test, VertexNonexistentLabelPropertyEdgeAPI) {
auto label = this->store->NameToLabel("label");
auto property = this->store->NameToProperty("property");
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
// Check state before (OLD view).
@ -2314,8 +2316,8 @@ TYPED_TEST(StorageV2Test, VertexNonexistentLabelPropertyEdgeAPI) {
}
TYPED_TEST(StorageV2Test, VertexVisibilitySingleTransaction) {
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc1->CreateVertex();
auto gid = vertex.Gid();
@ -2334,7 +2336,7 @@ TYPED_TEST(StorageV2Test, VertexVisibilitySingleTransaction) {
ASSERT_TRUE(vertex.SetProperty(acc1->NameToProperty("meaning"), memgraph::storage::PropertyValue(42)).HasValue());
auto acc3 = this->store->Access();
auto acc3 = this->store->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::OLD));
EXPECT_TRUE(acc1->FindVertex(gid, memgraph::storage::View::NEW));
@ -2371,8 +2373,8 @@ TYPED_TEST(StorageV2Test, VertexVisibilityMultipleTransactions) {
memgraph::storage::Gid gid;
{
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc1->CreateVertex();
gid = vertex.Gid();
@ -2401,8 +2403,8 @@ TYPED_TEST(StorageV2Test, VertexVisibilityMultipleTransactions) {
}
{
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc1->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -2435,7 +2437,7 @@ TYPED_TEST(StorageV2Test, VertexVisibilityMultipleTransactions) {
ASSERT_TRUE(vertex->SetProperty(acc1->NameToProperty("meaning"), memgraph::storage::PropertyValue(42)).HasValue());
auto acc3 = this->store->Access();
auto acc3 = this->store->Access(ReplicationRole::MAIN);
EXPECT_TRUE(acc1->FindVertex(gid, memgraph::storage::View::OLD));
EXPECT_TRUE(acc1->FindVertex(gid, memgraph::storage::View::NEW));
@ -2477,15 +2479,15 @@ TYPED_TEST(StorageV2Test, VertexVisibilityMultipleTransactions) {
}
{
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc1->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
ASSERT_TRUE(acc1->DeleteVertex(&*vertex).HasValue());
auto acc3 = this->store->Access();
auto acc3 = this->store->Access(ReplicationRole::MAIN);
EXPECT_TRUE(acc1->FindVertex(gid, memgraph::storage::View::OLD));
EXPECT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::NEW));
@ -2527,7 +2529,7 @@ TYPED_TEST(StorageV2Test, VertexVisibilityMultipleTransactions) {
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
EXPECT_TRUE(acc->FindVertex(gid, memgraph::storage::View::OLD));
EXPECT_TRUE(acc->FindVertex(gid, memgraph::storage::View::NEW));
@ -2541,15 +2543,15 @@ TYPED_TEST(StorageV2Test, VertexVisibilityMultipleTransactions) {
}
{
auto acc1 = this->store->Access();
auto acc2 = this->store->Access();
auto acc1 = this->store->Access(ReplicationRole::MAIN);
auto acc2 = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc1->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
ASSERT_TRUE(acc1->DeleteVertex(&*vertex).HasValue());
auto acc3 = this->store->Access();
auto acc3 = this->store->Access(ReplicationRole::MAIN);
EXPECT_TRUE(acc1->FindVertex(gid, memgraph::storage::View::OLD));
EXPECT_FALSE(acc1->FindVertex(gid, memgraph::storage::View::NEW));
@ -2591,7 +2593,7 @@ TYPED_TEST(StorageV2Test, VertexVisibilityMultipleTransactions) {
}
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->FindVertex(gid, memgraph::storage::View::OLD));
EXPECT_FALSE(acc->FindVertex(gid, memgraph::storage::View::NEW));
@ -2613,14 +2615,14 @@ TYPED_TEST(StorageV2Test, DeletedVertexAccessor) {
std::optional<memgraph::storage::Gid> gid;
// Create the vertex
{
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(vertex.SetProperty(property, property_value).HasError());
ASSERT_FALSE(acc->Commit().HasError());
}
auto acc = this->store->Access();
auto acc = this->store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(*gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto maybe_deleted_vertex = acc->DeleteVertex(&*vertex);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -23,6 +23,8 @@
#include "disk_test_utils.hpp"
using memgraph::replication::ReplicationRole;
// NOLINTNEXTLINE(google-build-using-namespace)
using namespace memgraph::storage;
@ -81,7 +83,7 @@ TYPED_TEST_CASE(ConstraintsTest, StorageTypes);
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_EQ(acc->ListAllConstraints().existence.size(), 0);
ASSERT_NO_ERROR(acc->Commit());
}
@ -92,7 +94,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(this->label1, this->prop1)));
ASSERT_NO_ERROR(acc->Commit());
}
@ -103,7 +105,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(this->label1, this->prop1)));
ASSERT_NO_ERROR(acc->Commit());
}
@ -114,7 +116,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(this->label1, this->prop1),
std::make_pair(this->label2, this->prop1)));
ASSERT_NO_ERROR(acc->Commit());
@ -130,7 +132,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(this->label2, this->prop1)));
ASSERT_NO_ERROR(acc->Commit());
}
@ -145,7 +147,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_EQ(acc->ListAllConstraints().existence.size(), 0);
ASSERT_NO_ERROR(acc->Commit());
}
@ -156,7 +158,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(this->label2, this->prop1)));
ASSERT_NO_ERROR(acc->Commit());
}
@ -165,7 +167,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateFailure1) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(acc->Commit());
@ -180,7 +182,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateFailure1) {
ASSERT_FALSE(unique_acc->Commit().HasError()); // TODO: Check if we are committing here?
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (auto vertex : acc->Vertices(View::OLD)) {
ASSERT_NO_ERROR(acc->DeleteVertex(&vertex));
}
@ -197,7 +199,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateFailure1) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateFailure2) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(acc->Commit());
@ -212,7 +214,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsCreateFailure2) {
ASSERT_FALSE(unique_acc->Commit().HasError()); // TODO: Check if we are committing here?
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (auto vertex : acc->Vertices(View::OLD)) {
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(1)));
}
@ -236,7 +238,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsViolationOnCommit) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
@ -248,7 +250,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsViolationOnCommit) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(1)));
@ -256,7 +258,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsViolationOnCommit) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (auto vertex : acc->Vertices(View::OLD)) {
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue()));
}
@ -269,7 +271,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsViolationOnCommit) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (auto vertex : acc->Vertices(View::OLD)) {
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue()));
}
@ -285,7 +287,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsViolationOnCommit) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(acc->Commit());
@ -295,7 +297,7 @@ TYPED_TEST(ConstraintsTest, ExistenceConstraintsViolationOnCommit) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_EQ(acc->ListAllConstraints().unique.size(), 0);
ASSERT_NO_ERROR(acc->Commit());
}
@ -307,7 +309,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().unique,
UnorderedElementsAre(std::make_pair(this->label1, std::set<PropertyId>{this->prop1})));
ASSERT_NO_ERROR(acc->Commit());
@ -320,7 +322,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().unique,
UnorderedElementsAre(std::make_pair(this->label1, std::set<PropertyId>{this->prop1})));
ASSERT_NO_ERROR(acc->Commit());
@ -333,7 +335,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().unique,
UnorderedElementsAre(std::make_pair(this->label1, std::set<PropertyId>{this->prop1}),
std::make_pair(this->label2, std::set<PropertyId>{this->prop1})));
@ -352,7 +354,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().unique,
UnorderedElementsAre(std::make_pair(this->label2, std::set<PropertyId>{this->prop1})));
ASSERT_NO_ERROR(acc->Commit());
@ -370,7 +372,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_EQ(acc->ListAllConstraints().unique.size(), 0);
ASSERT_NO_ERROR(acc->Commit());
}
@ -381,7 +383,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
EXPECT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().unique,
UnorderedElementsAre(std::make_pair(this->label2, std::set<PropertyId>{this->prop1})));
ASSERT_NO_ERROR(acc->Commit());
@ -391,7 +393,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateFailure1) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 2; ++i) {
auto vertex1 = acc->CreateVertex();
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
@ -411,7 +413,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateFailure1) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (auto vertex : acc->Vertices(View::OLD)) {
ASSERT_NO_ERROR(acc->DeleteVertex(&vertex));
}
@ -430,7 +432,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateFailure1) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateFailure2) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 2; ++i) {
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
@ -450,7 +452,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsCreateFailure2) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
int value = 0;
for (auto vertex : acc->Vertices(View::OLD)) {
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(value)));
@ -473,7 +475,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation1) {
Gid gid1;
Gid gid2;
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
gid1 = vertex1.Gid();
@ -493,7 +495,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation1) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid1, View::OLD);
auto vertex2 = acc->FindVertex(gid2, View::OLD);
@ -505,7 +507,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation1) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid1, View::OLD);
auto vertex2 = acc->FindVertex(gid2, View::OLD);
ASSERT_NO_ERROR(vertex1->SetProperty(this->prop1, PropertyValue(2)));
@ -528,8 +530,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation2) {
// tx1: B---SP(v1, 1)---SP(v1, 2)---OK--
// tx2: -B---SP(v2, 2)---SP(v2, 1)---OK-
auto acc1 = this->storage->Access();
auto acc2 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto vertex2 = acc2->CreateVertex();
@ -561,7 +563,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation3) {
// tx2: --------------------B---SP(v1, 2)---OK--
// tx3: ---------------------B---SP(v2, 1)---OK-
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto gid = vertex1.Gid();
@ -570,8 +572,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation3) {
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc3 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->FindVertex(gid, View::NEW); // vertex1 == vertex2
auto vertex3 = acc3->CreateVertex();
@ -599,7 +601,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation4) {
// tx2: --------------------B---SP(v2, 1)-----OK-
// tx3: ---------------------B---SP(v1, 2)---OK--
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto gid = vertex1.Gid();
@ -607,8 +609,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsNoViolation4) {
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc3 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->CreateVertex();
auto vertex3 = acc3->FindVertex(gid, View::NEW);
@ -632,7 +634,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsViolationOnCommit1) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
@ -663,7 +665,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsViolationOnCommit2) {
// tx2: -------------------------------B---SP(v1, 3)---OK----
// tx3: --------------------------------B---SP(v2, 3)---FAIL-
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto vertex2 = acc1->CreateVertex();
auto gid1 = vertex1.Gid();
@ -676,8 +678,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsViolationOnCommit2) {
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc3 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
auto vertex3 = acc2->FindVertex(gid1, View::NEW); // vertex3 == vertex1
auto vertex4 = acc3->FindVertex(gid2, View::NEW); // vertex4 == vertex2
@ -709,7 +711,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsViolationOnCommit3) {
// tx2: -------------------------------B---SP(v1, 2)---FAIL--
// tx3: --------------------------------B---SP(v2, 1)---FAIL-
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->CreateVertex();
auto vertex2 = acc1->CreateVertex();
auto gid1 = vertex1.Gid();
@ -722,8 +724,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsViolationOnCommit3) {
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc3 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
auto vertex3 = acc2->FindVertex(gid1, View::OLD); // vertex3 == vertex1
auto vertex4 = acc3->FindVertex(gid2, View::OLD); // vertex4 == vertex2
@ -762,7 +764,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsLabelAlteration) {
{
// B---AL(v2)---SP(v1, 1)---SP(v2, 1)---OK
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
gid1 = vertex1.Gid();
@ -782,8 +784,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsLabelAlteration) {
// tx1: B---AL(v1)-----OK-
// tx2: -B---RL(v2)---OK--
auto acc1 = this->storage->Access();
auto acc2 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->FindVertex(gid1, View::OLD);
auto vertex2 = acc2->FindVertex(gid2, View::OLD);
@ -811,7 +813,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsLabelAlteration) {
{
// B---AL(v2)---FAIL
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc->FindVertex(gid2, View::OLD);
ASSERT_NO_ERROR(vertex2->AddLabel(this->label1));
@ -824,7 +826,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsLabelAlteration) {
{
// B---RL(v1)---OK
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid1, View::OLD);
ASSERT_NO_ERROR(vertex1->RemoveLabel(this->label1));
ASSERT_NO_ERROR(acc->Commit());
@ -834,8 +836,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsLabelAlteration) {
// tx1: B---AL(v1)-----FAIL
// tx2: -B---AL(v2)---OK---
auto acc1 = this->storage->Access();
auto acc2 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->FindVertex(gid1, View::OLD);
auto vertex2 = acc2->FindVertex(gid2, View::OLD);
@ -911,7 +913,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsPropertySetSize) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllConstraints().unique, UnorderedElementsAre(std::make_pair(this->label1, properties)));
ASSERT_NO_ERROR(acc->Commit());
}
@ -922,7 +924,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsPropertySetSize) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
ASSERT_TRUE(acc->ListAllConstraints().unique.empty());
ASSERT_NO_ERROR(acc->Commit());
}
@ -951,7 +953,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsMultipleProperties) {
Gid gid1;
Gid gid2;
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
gid1 = vertex1.Gid();
@ -971,7 +973,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsMultipleProperties) {
// Try to change property of the second vertex so it becomes the same as the
// first vertex-> It should fail.
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc->FindVertex(gid2, View::OLD);
ASSERT_NO_ERROR(vertex2->SetProperty(this->prop2, PropertyValue(2)));
auto res = acc->Commit();
@ -985,7 +987,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsMultipleProperties) {
// both vertices should now be equal. However, this operation should succeed
// since null value is treated as non-existing property.
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid1, View::OLD);
auto vertex2 = acc->FindVertex(gid2, View::OLD);
ASSERT_NO_ERROR(vertex1->SetProperty(this->prop2, PropertyValue()));
@ -1005,7 +1007,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertAbortInsert) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(1)));
@ -1014,7 +1016,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertAbortInsert) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop2, PropertyValue(2)));
@ -1034,7 +1036,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertRemoveInsert) {
Gid gid;
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
@ -1044,14 +1046,14 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertRemoveInsert) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, View::OLD);
ASSERT_NO_ERROR(acc->DeleteVertex(&*vertex));
ASSERT_NO_ERROR(acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(1)));
@ -1071,7 +1073,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertRemoveAbortInsert) {
Gid gid;
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
@ -1081,14 +1083,14 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertRemoveAbortInsert) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, View::OLD);
ASSERT_NO_ERROR(acc->DeleteVertex(&*vertex));
acc->Abort();
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop2, PropertyValue(1)));
@ -1114,7 +1116,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsDeleteVertexSetProperty) {
Gid gid1;
Gid gid2;
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
gid1 = vertex1.Gid();
@ -1129,8 +1131,8 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsDeleteVertexSetProperty) {
}
{
auto acc1 = this->storage->Access();
auto acc2 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = acc1->FindVertex(gid1, View::OLD);
auto vertex2 = acc2->FindVertex(gid2, View::OLD);
@ -1156,7 +1158,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertDropInsert) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(1)));
@ -1172,7 +1174,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsInsertDropInsert) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop2, PropertyValue(2)));
@ -1194,7 +1196,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsComparePropertyValues) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(2)));
@ -1203,7 +1205,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsComparePropertyValues) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(1)));
@ -1212,7 +1214,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsComparePropertyValues) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop2, PropertyValue(0)));
@ -1238,7 +1240,7 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsClearOldData) {
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop1, PropertyValue(2)));
@ -1246,14 +1248,14 @@ TYPED_TEST(ConstraintsTest, UniqueConstraintsClearOldData) {
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc2 = this->storage->Access(std::nullopt);
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->FindVertex(vertex.Gid(), memgraph::storage::View::NEW).value();
ASSERT_TRUE(vertex2.SetProperty(this->prop1, memgraph::storage::PropertyValue(2)).HasValue());
ASSERT_FALSE(acc2->Commit().HasError());
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc3 = this->storage->Access(std::nullopt);
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
auto vertex3 = acc3->FindVertex(vertex.Gid(), memgraph::storage::View::NEW).value();
ASSERT_TRUE(vertex3.SetProperty(this->prop1, memgraph::storage::PropertyValue(10)).HasValue());
ASSERT_FALSE(acc3->Commit().HasError());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -48,6 +48,7 @@
#include "utils/timer.hpp"
#include "utils/uuid.hpp"
using memgraph::replication::ReplicationRole;
using testing::Contains;
using testing::UnorderedElementsAre;
@ -93,26 +94,26 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
{
// Create label index.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label_unindexed).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
// Create label index statistics.
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
acc->SetIndexStats(label_unindexed, memgraph::storage::LabelIndexStats{1, 2});
ASSERT_TRUE(acc->GetIndexStats(label_unindexed));
ASSERT_FALSE(acc->Commit().HasError());
}
{
// Create label+property index.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label_indexed, property_id).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
// Create label+property index statistics.
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
acc->SetIndexStats(label_indexed, property_id, memgraph::storage::LabelPropertyIndexStats{1, 2, 3.4, 5.6, 0.0});
ASSERT_TRUE(acc->GetIndexStats(label_indexed, property_id));
ASSERT_FALSE(acc->Commit().HasError());
@ -120,20 +121,20 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
{
// Create existence constraint.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateExistenceConstraint(label_unindexed, property_id).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
// Create unique constraint.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(label_unindexed, {property_id, property_extra}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
// Create vertices.
for (uint64_t i = 0; i < kNumBaseVertices; ++i) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
base_vertex_gids_[i] = vertex.Gid();
if (i < kNumBaseVertices / 2) {
@ -150,7 +151,7 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
// Create edges.
for (uint64_t i = 0; i < kNumBaseEdges; ++i) {
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(base_vertex_gids_[(i / 2) % kNumBaseVertices], memgraph::storage::View::OLD);
ASSERT_TRUE(vertex1);
auto vertex2 = acc->FindVertex(base_vertex_gids_[(i / 3) % kNumBaseVertices], memgraph::storage::View::OLD);
@ -186,26 +187,26 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
{
// Create label index.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label_unused).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
// Create label index statistics.
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
acc->SetIndexStats(label_unused, memgraph::storage::LabelIndexStats{123, 9.87});
ASSERT_TRUE(acc->GetIndexStats(label_unused));
ASSERT_FALSE(acc->Commit().HasError());
}
{
// Create label+property index.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(label_indexed, property_count).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
// Create label+property index statistics.
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
acc->SetIndexStats(label_indexed, property_count,
memgraph::storage::LabelPropertyIndexStats{456798, 312345, 12312312.2, 123123.2, 67876.9});
ASSERT_TRUE(acc->GetIndexStats(label_indexed, property_count));
@ -214,25 +215,25 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
{
// Create existence constraint.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateExistenceConstraint(label_unused, property_count).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
// Create unique constraint.
auto unique_acc = store->UniqueAccess();
auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(label_unused, {property_count}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
// Storage accessor.
std::unique_ptr<memgraph::storage::Storage::Accessor> acc;
if (single_transaction) acc = store->Access();
if (single_transaction) acc = store->Access(ReplicationRole::MAIN);
// Create vertices.
for (uint64_t i = 0; i < kNumExtendedVertices; ++i) {
if (!single_transaction) acc = store->Access();
if (!single_transaction) acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
extended_vertex_gids_[i] = vertex.Gid();
if (i < kNumExtendedVertices / 2) {
@ -246,7 +247,7 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
// Create edges.
for (uint64_t i = 0; i < kNumExtendedEdges; ++i) {
if (!single_transaction) acc = store->Access();
if (!single_transaction) acc = store->Access(ReplicationRole::MAIN);
auto vertex1 =
acc->FindVertex(extended_vertex_gids_[(i / 5) % kNumExtendedVertices], memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -285,7 +286,7 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
auto et4 = store->NameToEdgeType("extended_et4");
// Create storage accessor.
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
// Verify indices info.
{
@ -826,7 +827,7 @@ TEST_P(DurabilityTest, SnapshotOnExit) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -863,7 +864,7 @@ TEST_P(DurabilityTest, SnapshotPeriodic) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -933,7 +934,7 @@ TEST_P(DurabilityTest, SnapshotFallback) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -950,7 +951,7 @@ TEST_P(DurabilityTest, SnapshotEverythingCorrupt) {
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -1035,7 +1036,7 @@ TEST_P(DurabilityTest, SnapshotRetention) {
.durability = {.storage_directory = storage_directory, .snapshot_on_exit = true}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -1096,7 +1097,7 @@ TEST_P(DurabilityTest, SnapshotRetention) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1165,7 +1166,7 @@ TEST_P(DurabilityTest, SnapshotMixedUUID) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1181,7 +1182,7 @@ TEST_P(DurabilityTest, SnapshotBackup) {
.durability = {.storage_directory = storage_directory, .snapshot_on_exit = true}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -1238,7 +1239,7 @@ TEST_F(DurabilityTest, SnapshotWithoutPropertiesOnEdgesRecoveryWithPropertiesOnE
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1292,7 +1293,7 @@ TEST_F(DurabilityTest, SnapshotWithPropertiesOnEdgesButUnusedRecoveryWithoutProp
VerifyDataset(db.storage(), DatasetType::BASE_WITH_EXTENDED, true);
// Remove properties from edges.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
for (auto vertex : acc->Vertices(memgraph::storage::View::OLD)) {
auto in_edges = vertex.InEdges(memgraph::storage::View::OLD);
ASSERT_TRUE(in_edges.HasValue());
@ -1333,7 +1334,7 @@ TEST_F(DurabilityTest, SnapshotWithPropertiesOnEdgesButUnusedRecoveryWithoutProp
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1372,7 +1373,7 @@ TEST_P(DurabilityTest, WalBasic) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1394,7 +1395,7 @@ TEST_P(DurabilityTest, WalBackup) {
.wal_file_flush_every_n_tx = kFlushWalEvery}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -1485,7 +1486,7 @@ TEST_P(DurabilityTest, WalAppendToExisting) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1509,7 +1510,7 @@ TEST_P(DurabilityTest, WalCreateInSingleTransaction) {
.wal_file_flush_every_n_tx = kFlushWalEvery}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
auto v1 = acc->CreateVertex();
gid_v1 = v1.Gid();
auto v2 = acc->CreateVertex();
@ -1545,7 +1546,7 @@ TEST_P(DurabilityTest, WalCreateInSingleTransaction) {
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto indices = acc->ListAllIndices();
ASSERT_EQ(indices.label.size(), 0);
@ -1628,7 +1629,7 @@ TEST_P(DurabilityTest, WalCreateInSingleTransaction) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1652,39 +1653,39 @@ TEST_P(DurabilityTest, WalCreateAndRemoveEverything) {
CreateBaseDataset(db.storage(), GetParam());
CreateExtendedDataset(db.storage());
auto indices = [&] {
auto acc = db.storage()->Access();
auto acc = db.Access();
auto res = acc->ListAllIndices();
acc->Commit();
return res;
}(); // iile
for (const auto &index : indices.label) {
auto unique_acc = db.storage()->UniqueAccess();
auto unique_acc = db.UniqueAccess();
ASSERT_FALSE(unique_acc->DropIndex(index).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
for (const auto &index : indices.label_property) {
auto unique_acc = db.storage()->UniqueAccess();
auto unique_acc = db.UniqueAccess();
ASSERT_FALSE(unique_acc->DropIndex(index.first, index.second).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto constraints = [&] {
auto acc = db.storage()->Access();
auto acc = db.Access();
auto res = acc->ListAllConstraints();
acc->Commit();
return res;
}(); // iile
for (const auto &constraint : constraints.existence) {
auto unique_acc = db.storage()->UniqueAccess();
auto unique_acc = db.UniqueAccess();
ASSERT_FALSE(unique_acc->DropExistenceConstraint(constraint.first, constraint.second).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
for (const auto &constraint : constraints.unique) {
auto unique_acc = db.storage()->UniqueAccess();
auto unique_acc = db.UniqueAccess();
ASSERT_EQ(unique_acc->DropUniqueConstraint(constraint.first, constraint.second),
memgraph::storage::UniqueConstraints::DeletionStatus::SUCCESS);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto acc = db.storage()->Access();
auto acc = db.Access();
for (auto vertex : acc->Vertices(memgraph::storage::View::OLD)) {
ASSERT_TRUE(acc->DetachDeleteVertex(&vertex).HasValue());
}
@ -1702,7 +1703,7 @@ TEST_P(DurabilityTest, WalCreateAndRemoveEverything) {
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto indices = acc->ListAllIndices();
ASSERT_EQ(indices.label.size(), 0);
ASSERT_EQ(indices.label_property.size(), 0);
@ -1719,7 +1720,7 @@ TEST_P(DurabilityTest, WalCreateAndRemoveEverything) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1745,8 +1746,8 @@ TEST_P(DurabilityTest, WalTransactionOrdering) {
}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc1 = db.storage()->Access();
auto acc2 = db.storage()->Access();
auto acc1 = db.Access();
auto acc2 = db.Access();
// Create vertex in transaction 2.
{
@ -1756,7 +1757,7 @@ TEST_P(DurabilityTest, WalTransactionOrdering) {
vertex2.SetProperty(db.storage()->NameToProperty("id"), memgraph::storage::PropertyValue(2)).HasValue());
}
auto acc3 = db.storage()->Access();
auto acc3 = db.Access();
// Create vertex in transaction 3.
{
@ -1839,7 +1840,7 @@ TEST_P(DurabilityTest, WalTransactionOrdering) {
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
{
auto acc = db.storage()->Access();
auto acc = db.Access();
for (auto [gid, id] : std::vector<std::pair<memgraph::storage::Gid, int64_t>>{{gid1, 1}, {gid2, 2}, {gid3, 3}}) {
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -1855,7 +1856,7 @@ TEST_P(DurabilityTest, WalTransactionOrdering) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1880,7 +1881,7 @@ TEST_P(DurabilityTest, WalCreateAndRemoveOnlyBaseDataset) {
CreateExtendedDataset(db.storage());
auto label_indexed = db.storage()->NameToLabel("base_indexed");
auto label_unindexed = db.storage()->NameToLabel("base_unindexed");
auto acc = db.storage()->Access();
auto acc = db.Access();
for (auto vertex : acc->Vertices(memgraph::storage::View::OLD)) {
auto has_indexed = vertex.HasLabel(label_indexed, memgraph::storage::View::OLD);
ASSERT_TRUE(has_indexed.HasValue());
@ -1905,7 +1906,7 @@ TEST_P(DurabilityTest, WalCreateAndRemoveOnlyBaseDataset) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -1930,7 +1931,7 @@ TEST_P(DurabilityTest, WalDeathResilience) {
memgraph::dbms::Database db{config, repl_state};
// Create one million vertices.
for (uint64_t i = 0; i < 1000000; ++i) {
auto acc = db.storage()->Access();
auto acc = db.Access();
acc->CreateVertex();
MG_ASSERT(!acc->Commit().HasError(), "Couldn't commit transaction!");
}
@ -1968,7 +1969,7 @@ TEST_P(DurabilityTest, WalDeathResilience) {
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto iterable = acc->Vertices(memgraph::storage::View::OLD);
for (auto it = iterable.begin(); it != iterable.end(); ++it) {
++count;
@ -1977,7 +1978,7 @@ TEST_P(DurabilityTest, WalDeathResilience) {
}
{
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < kExtraItems; ++i) {
acc->CreateVertex();
}
@ -1997,7 +1998,7 @@ TEST_P(DurabilityTest, WalDeathResilience) {
memgraph::dbms::Database db{config, repl_state};
{
uint64_t current = 0;
auto acc = db.storage()->Access();
auto acc = db.Access();
auto iterable = acc->Vertices(memgraph::storage::View::OLD);
for (auto it = iterable.begin(); it != iterable.end(); ++it) {
++current;
@ -2007,7 +2008,7 @@ TEST_P(DurabilityTest, WalDeathResilience) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -2029,7 +2030,7 @@ TEST_P(DurabilityTest, WalMissingSecond) {
.wal_file_flush_every_n_tx = kFlushWalEvery}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -2059,13 +2060,13 @@ TEST_P(DurabilityTest, WalMissingSecond) {
std::vector<memgraph::storage::Gid> gids;
gids.reserve(kNumVertices);
for (uint64_t i = 0; i < kNumVertices; ++i) {
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
gids.push_back(vertex.Gid());
ASSERT_FALSE(acc->Commit().HasError());
}
for (uint64_t i = 0; i < kNumVertices; ++i) {
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
ASSERT_TRUE(
@ -2124,7 +2125,7 @@ TEST_P(DurabilityTest, WalCorruptSecond) {
.wal_file_flush_every_n_tx = kFlushWalEvery}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -2154,13 +2155,13 @@ TEST_P(DurabilityTest, WalCorruptSecond) {
std::vector<memgraph::storage::Gid> gids;
gids.reserve(kNumVertices);
for (uint64_t i = 0; i < kNumVertices; ++i) {
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
gids.push_back(vertex.Gid());
ASSERT_FALSE(acc->Commit().HasError());
}
for (uint64_t i = 0; i < kNumVertices; ++i) {
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->FindVertex(gids[i], memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
ASSERT_TRUE(
@ -2246,7 +2247,7 @@ TEST_P(DurabilityTest, WalCorruptLastTransaction) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -2268,7 +2269,7 @@ TEST_P(DurabilityTest, WalAllOperationsInSingleTransaction) {
.wal_file_flush_every_n_tx = kFlushWalEvery}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
ASSERT_TRUE(vertex1.AddLabel(acc->NameToLabel("nandare")).HasValue());
@ -2314,7 +2315,7 @@ TEST_P(DurabilityTest, WalAllOperationsInSingleTransaction) {
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
{
auto acc = db.storage()->Access();
auto acc = db.Access();
uint64_t count = 0;
auto iterable = acc->Vertices(memgraph::storage::View::OLD);
for (auto it = iterable.begin(); it != iterable.end(); ++it) {
@ -2325,7 +2326,7 @@ TEST_P(DurabilityTest, WalAllOperationsInSingleTransaction) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -2365,7 +2366,7 @@ TEST_P(DurabilityTest, WalAndSnapshot) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -2428,7 +2429,7 @@ TEST_P(DurabilityTest, WalAndSnapshotAppendToExistingSnapshot) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -2496,7 +2497,7 @@ TEST_P(DurabilityTest, WalAndSnapshotAppendToExistingSnapshotAndWal) {
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
VerifyDataset(db.storage(), DatasetType::BASE_WITH_EXTENDED, GetParam());
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
vertex_gid = vertex.Gid();
if (GetParam()) {
@ -2519,7 +2520,7 @@ TEST_P(DurabilityTest, WalAndSnapshotAppendToExistingSnapshotAndWal) {
VerifyDataset(db.storage(), DatasetType::BASE_WITH_EXTENDED, GetParam(),
/* verify_info = */ false);
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->FindVertex(vertex_gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto labels = vertex->Labels(memgraph::storage::View::OLD);
@ -2537,7 +2538,7 @@ TEST_P(DurabilityTest, WalAndSnapshotAppendToExistingSnapshotAndWal) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -2559,7 +2560,7 @@ TEST_P(DurabilityTest, WalAndSnapshotWalRetention) {
.wal_file_flush_every_n_tx = kFlushWalEvery}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -2592,7 +2593,7 @@ TEST_P(DurabilityTest, WalAndSnapshotWalRetention) {
memgraph::utils::Timer timer;
// Allow at least 6 snapshots to be created.
while (timer.Elapsed().count() < 13.0) {
auto acc = db.storage()->Access();
auto acc = db.Access();
acc->CreateVertex();
ASSERT_FALSE(acc->Commit().HasError());
++items_created;
@ -2617,7 +2618,7 @@ TEST_P(DurabilityTest, WalAndSnapshotWalRetention) {
.durability = {.storage_directory = storage_directory, .recover_on_startup = true}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t j = 0; j < items_created; ++j) {
auto vertex = acc->FindVertex(memgraph::storage::Gid::FromUint(j), memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -2654,7 +2655,7 @@ TEST_P(DurabilityTest, SnapshotAndWalMixedUUID) {
.snapshot_interval = std::chrono::seconds(2)}};
memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)};
memgraph::dbms::Database db{config, repl_state};
auto acc = db.storage()->Access();
auto acc = db.Access();
for (uint64_t i = 0; i < 1000; ++i) {
acc->CreateVertex();
}
@ -2705,7 +2706,7 @@ TEST_P(DurabilityTest, SnapshotAndWalMixedUUID) {
// Try to use the storage.
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());
@ -2744,7 +2745,7 @@ TEST_P(DurabilityTest, ParallelConstraintsRecovery) {
memgraph::dbms::Database db{config, repl_state};
VerifyDataset(db.storage(), DatasetType::BASE_WITH_EXTENDED, GetParam());
{
auto acc = db.storage()->Access();
auto acc = db.Access();
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et"));
ASSERT_TRUE(edge.HasValue());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -17,6 +17,7 @@
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/storage.hpp"
using memgraph::replication::ReplicationRole;
using testing::UnorderedElementsAre;
class StorageEdgeTest : public ::testing::TestWithParam<bool> {};
@ -33,7 +34,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -43,7 +44,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -117,7 +118,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -224,7 +225,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->CreateVertex();
auto vertex_from = acc->CreateVertex();
gid_to = vertex_to.Gid();
@ -234,7 +235,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -302,7 +303,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -396,7 +397,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameCommit) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -404,7 +405,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -462,7 +463,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -543,7 +544,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -553,7 +554,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Create edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -621,7 +622,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -650,7 +651,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -718,7 +719,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -813,7 +814,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->CreateVertex();
auto vertex_from = acc->CreateVertex();
gid_to = vertex_to.Gid();
@ -823,7 +824,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Create edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -891,7 +892,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -920,7 +921,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -988,7 +989,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1082,7 +1083,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -1090,7 +1091,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Create edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1148,7 +1149,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1167,7 +1168,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1225,7 +1226,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1310,7 +1311,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -1320,7 +1321,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1388,7 +1389,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1475,7 +1476,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Delete edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1542,7 +1543,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1579,7 +1580,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->CreateVertex();
auto vertex_from = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -1589,7 +1590,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1657,7 +1658,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1744,7 +1745,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Delete edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1811,7 +1812,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1847,7 +1848,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -1855,7 +1856,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1913,7 +1914,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1990,7 +1991,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Delete edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -2047,7 +2048,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -2074,7 +2075,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -2084,7 +2085,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2152,7 +2153,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2239,7 +2240,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Delete the edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2306,7 +2307,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2393,7 +2394,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Delete the edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2460,7 +2461,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2497,7 +2498,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -2507,7 +2508,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2575,7 +2576,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2662,7 +2663,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Delete the edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2729,7 +2730,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2817,7 +2818,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Delete the edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2884,7 +2885,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2920,7 +2921,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -2928,7 +2929,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -2986,7 +2987,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3063,7 +3064,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Delete the edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3120,7 +3121,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3197,7 +3198,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Delete the edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3254,7 +3255,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3281,7 +3282,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleCommit) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
@ -3330,7 +3331,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleCommit) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -3394,7 +3395,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleCommit) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex_from);
@ -3421,7 +3422,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleCommit) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
@ -3548,7 +3549,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleCommit) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -3686,7 +3687,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleCommit) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex1);
@ -3751,7 +3752,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
@ -3800,7 +3801,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Detach delete vertex, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -3864,7 +3865,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -3905,7 +3906,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -3969,7 +3970,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex_from);
@ -3996,7 +3997,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
@ -4123,7 +4124,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Detach delete vertex, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -4261,7 +4262,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -4439,7 +4440,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -4577,7 +4578,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex1);
@ -4639,7 +4640,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
new memgraph::storage::InMemoryStorage({.items = {.properties_on_edges = true}}));
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -4682,7 +4683,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4711,7 +4712,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
acc->Abort();
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4743,7 +4744,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4772,7 +4773,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Create the vertex.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -4785,7 +4786,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to "nandare", but abort the transaction.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4826,7 +4827,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is null.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4848,7 +4849,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to "nandare".
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4889,7 +4890,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is "nandare".
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4920,7 +4921,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to null, but abort the transaction.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4962,7 +4963,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is "nandare".
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4993,7 +4994,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to null.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5035,7 +5036,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is null.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5062,7 +5063,7 @@ TEST(StorageWithProperties, EdgePropertySerializationError) {
new memgraph::storage::InMemoryStorage({.items = {.properties_on_edges = true}}));
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -5073,8 +5074,8 @@ TEST(StorageWithProperties, EdgePropertySerializationError) {
ASSERT_FALSE(acc->Commit().HasError());
}
auto acc1 = store->Access();
auto acc2 = store->Access();
auto acc1 = store->Access(ReplicationRole::MAIN);
auto acc2 = store->Access(ReplicationRole::MAIN);
// Set property 1 to 123 in accessor 1.
{
@ -5139,7 +5140,7 @@ TEST(StorageWithProperties, EdgePropertySerializationError) {
// Check which properties exist.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5174,7 +5175,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
auto property1 = store->NameToProperty("property1");
auto property2 = store->NameToProperty("property2");
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -5190,7 +5191,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5223,7 +5224,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
acc->Abort();
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5235,7 +5236,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5269,7 +5270,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5288,7 +5289,7 @@ TEST(StorageWithoutProperties, EdgePropertyAbort) {
new memgraph::storage::InMemoryStorage({.items = {.properties_on_edges = false}}));
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -5299,7 +5300,7 @@ TEST(StorageWithoutProperties, EdgePropertyAbort) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5330,7 +5331,7 @@ TEST(StorageWithoutProperties, EdgePropertyAbort) {
acc->Abort();
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5357,7 +5358,7 @@ TEST(StorageWithoutProperties, EdgePropertyClear) {
new memgraph::storage::InMemoryStorage({.items = {.properties_on_edges = false}}));
memgraph::storage::Gid gid;
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -5368,7 +5369,7 @@ TEST(StorageWithoutProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -5385,7 +5386,7 @@ TEST(StorageWithProperties, EdgeNonexistentPropertyAPI) {
auto property = store->NameToProperty("property");
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, acc->NameToEdgeType("edge"));
ASSERT_TRUE(edge.HasValue());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -18,6 +18,7 @@
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/storage.hpp"
using memgraph::replication::ReplicationRole;
using testing::UnorderedElementsAre;
class StorageEdgeTest : public ::testing::TestWithParam<bool> {};
@ -37,7 +38,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -47,7 +48,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -121,7 +122,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -230,7 +231,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->CreateVertex();
auto vertex_from = acc->CreateVertex();
gid_to = vertex_to.Gid();
@ -240,7 +241,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -308,7 +309,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -404,7 +405,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameCommit) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -412,7 +413,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -470,7 +471,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -553,7 +554,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -563,7 +564,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Create edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -631,7 +632,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -660,7 +661,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -728,7 +729,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -825,7 +826,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->CreateVertex();
auto vertex_from = acc->CreateVertex();
gid_to = vertex_to.Gid();
@ -835,7 +836,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Create edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -903,7 +904,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -932,7 +933,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1000,7 +1001,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1096,7 +1097,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -1104,7 +1105,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Create edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1162,7 +1163,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1181,7 +1182,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1239,7 +1240,7 @@ TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1326,7 +1327,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -1336,7 +1337,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1404,7 +1405,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1491,7 +1492,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Delete edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1558,7 +1559,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1597,7 +1598,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->CreateVertex();
auto vertex_from = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -1607,7 +1608,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1675,7 +1676,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1762,7 +1763,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Delete edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1829,7 +1830,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -1867,7 +1868,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -1875,7 +1876,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -1933,7 +1934,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -2010,7 +2011,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Delete edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -2067,7 +2068,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -2096,7 +2097,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -2106,7 +2107,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2174,7 +2175,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2261,7 +2262,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Delete the edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2328,7 +2329,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2415,7 +2416,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Delete the edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2482,7 +2483,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2521,7 +2522,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Create vertices
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
gid_from = vertex_from.Gid();
@ -2531,7 +2532,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2599,7 +2600,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2686,7 +2687,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Delete the edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2753,7 +2754,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2841,7 +2842,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Delete the edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2908,7 +2909,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -2946,7 +2947,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Create vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid_vertex = vertex.Gid();
ASSERT_FALSE(acc->Commit().HasError());
@ -2954,7 +2955,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Create edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3012,7 +3013,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3089,7 +3090,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Delete the edge, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3146,7 +3147,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3223,7 +3224,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Delete the edge
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3280,7 +3281,7 @@ TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
// Check whether the edge exists
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid_vertex, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex);
@ -3309,7 +3310,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleCommit) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
@ -3358,7 +3359,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleCommit) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -3422,7 +3423,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleCommit) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex_from);
@ -3451,7 +3452,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleCommit) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
@ -3578,7 +3579,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleCommit) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -3716,7 +3717,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleCommit) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex1);
@ -3783,7 +3784,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->CreateVertex();
auto vertex_to = acc->CreateVertex();
@ -3832,7 +3833,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Detach delete vertex, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -3896,7 +3897,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -3937,7 +3938,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex_from);
@ -4001,7 +4002,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex_from = acc->FindVertex(gid_from, memgraph::storage::View::NEW);
auto vertex_to = acc->FindVertex(gid_to, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex_from);
@ -4030,7 +4031,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Create dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->CreateVertex();
auto vertex2 = acc->CreateVertex();
@ -4157,7 +4158,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Detach delete vertex, but abort the transaction
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -4295,7 +4296,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -4473,7 +4474,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Detach delete vertex
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_TRUE(vertex1);
@ -4611,7 +4612,7 @@ TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
// Check dataset
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex1 = acc->FindVertex(gid_vertex1, memgraph::storage::View::NEW);
auto vertex2 = acc->FindVertex(gid_vertex2, memgraph::storage::View::NEW);
ASSERT_FALSE(vertex1);
@ -4675,7 +4676,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
std::unique_ptr<memgraph::storage::Storage> store(new memgraph::storage::DiskStorage(config));
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -4718,7 +4719,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -4748,7 +4749,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
acc->Abort();
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -4781,7 +4782,7 @@ TEST(StorageWithProperties, EdgePropertyCommit) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -4813,7 +4814,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Create the vertex.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -4826,7 +4827,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to "nandare", but abort the transaction.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -4868,7 +4869,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is null.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
auto edge = vertex->OutEdges(memgraph::storage::View::NEW).GetValue().edges[0];
@ -4890,7 +4891,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to "nandare".
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -4932,7 +4933,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is "nandare".
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -4964,7 +4965,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to null, but abort the transaction.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5007,7 +5008,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is "nandare".
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5039,7 +5040,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Set property 5 to null.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5082,7 +5083,7 @@ TEST(StorageWithProperties, EdgePropertyAbort) {
// Check that property 5 is null.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5112,7 +5113,7 @@ TEST(StorageWithProperties, EdgePropertySerializationError) {
std::unique_ptr<memgraph::storage::Storage> store(new memgraph::storage::DiskStorage(config));
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -5123,8 +5124,8 @@ TEST(StorageWithProperties, EdgePropertySerializationError) {
ASSERT_FALSE(acc->Commit().HasError());
}
auto acc1 = store->Access();
auto acc2 = store->Access();
auto acc1 = store->Access(ReplicationRole::MAIN);
auto acc2 = store->Access(ReplicationRole::MAIN);
// Set property 1 to 123 in accessor 1.
{
@ -5195,7 +5196,7 @@ TEST(StorageWithProperties, EdgePropertySerializationError) {
// Check which properties exist.
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5233,7 +5234,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
auto property1 = store->NameToProperty("property1");
auto property2 = store->NameToProperty("property2");
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -5249,7 +5250,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5283,7 +5284,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
acc->Abort();
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5296,7 +5297,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5331,7 +5332,7 @@ TEST(StorageWithProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5353,7 +5354,7 @@ TEST(StorageWithoutProperties, EdgePropertyAbort) {
std::unique_ptr<memgraph::storage::Storage> store(new memgraph::storage::DiskStorage(config));
memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
spdlog::trace("Created vertex with gid: {}", gid.AsInt());
@ -5365,7 +5366,7 @@ TEST(StorageWithoutProperties, EdgePropertyAbort) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5397,7 +5398,7 @@ TEST(StorageWithoutProperties, EdgePropertyAbort) {
acc->Abort();
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5427,7 +5428,7 @@ TEST(StorageWithoutProperties, EdgePropertyClear) {
std::unique_ptr<memgraph::storage::Storage> store(new memgraph::storage::DiskStorage(config));
memgraph::storage::Gid gid;
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
gid = vertex.Gid();
auto et = acc->NameToEdgeType("et5");
@ -5438,7 +5439,7 @@ TEST(StorageWithoutProperties, EdgePropertyClear) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
ASSERT_TRUE(vertex);
@ -5458,7 +5459,7 @@ TEST(StorageWithProperties, EdgeNonexistentPropertyAPI) {
auto property = store->NameToProperty("property");
auto acc = store->Access();
auto acc = store->Access(ReplicationRole::MAIN);
auto vertex = acc->CreateVertex();
auto edge = acc->CreateEdge(&vertex, &vertex, acc->NameToEdgeType("edge"));
ASSERT_TRUE(edge.HasValue());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -14,6 +14,7 @@
#include "storage/v2/inmemory/storage.hpp"
using memgraph::replication::ReplicationRole;
using testing::UnorderedElementsAre;
// TODO: The point of these is not to test GC fully, these are just simple
@ -31,7 +32,7 @@ TEST(StorageV2Gc, Sanity) {
std::vector<memgraph::storage::Gid> vertices;
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
// Create some vertices, but delete some of them immediately.
for (uint64_t i = 0; i < 1000; ++i) {
auto vertex = acc->CreateVertex();
@ -63,7 +64,7 @@ TEST(StorageV2Gc, Sanity) {
// Verify existing vertices and add labels to some of them.
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
for (uint64_t i = 0; i < 1000; ++i) {
auto vertex = acc->FindVertex(vertices[i], memgraph::storage::View::OLD);
EXPECT_EQ(vertex.has_value(), i % 5 != 0);
@ -101,7 +102,7 @@ TEST(StorageV2Gc, Sanity) {
// Add and remove some edges.
{
auto acc = storage->Access();
auto acc = storage->Access(ReplicationRole::MAIN);
for (uint64_t i = 0; i < 1000; ++i) {
auto from_vertex = acc->FindVertex(vertices[i], memgraph::storage::View::OLD);
auto to_vertex = acc->FindVertex(vertices[(i + 1) % 1000], memgraph::storage::View::OLD);
@ -171,13 +172,13 @@ TEST(StorageV2Gc, Indices) {
std::make_unique<memgraph::storage::InMemoryStorage>(memgraph::storage::Config{
.gc = {.type = memgraph::storage::Config::Gc::Type::PERIODIC, .interval = std::chrono::milliseconds(100)}}));
{
auto unique_acc = storage->UniqueAccess();
auto unique_acc = storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(storage->NameToLabel("label")).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto acc0 = storage->Access();
auto acc0 = storage->Access(ReplicationRole::MAIN);
for (uint64_t i = 0; i < 1000; ++i) {
auto vertex = acc0->CreateVertex();
ASSERT_TRUE(*vertex.AddLabel(acc0->NameToLabel("label")));
@ -185,9 +186,9 @@ TEST(StorageV2Gc, Indices) {
ASSERT_FALSE(acc0->Commit().HasError());
}
{
auto acc1 = storage->Access();
auto acc1 = storage->Access(ReplicationRole::MAIN);
auto acc2 = storage->Access();
auto acc2 = storage->Access(ReplicationRole::MAIN);
for (auto vertex : acc2->Vertices(memgraph::storage::View::OLD)) {
ASSERT_TRUE(*vertex.RemoveLabel(acc2->NameToLabel("label")));
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -22,7 +22,7 @@
// NOLINTNEXTLINE(google-build-using-namespace)
using namespace memgraph::storage;
using memgraph::replication::ReplicationRole;
constexpr auto testSuite = "storage_v2_get_info";
const std::filesystem::path storage_directory{std::filesystem::temp_directory_path() / testSuite};
@ -63,22 +63,22 @@ TYPED_TEST(InfoTest, InfoCheck) {
{
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateExistenceConstraint(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->DropExistenceConstraint(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto v1 = acc->CreateVertex();
auto v2 = acc->CreateVertex();
auto v3 = acc->CreateVertex();
auto v4 = acc->CreateVertex();
auto v5 = acc->CreateVertex();
[[maybe_unused]] auto v5 = acc->CreateVertex();
ASSERT_FALSE(v2.AddLabel(lbl).HasError());
ASSERT_FALSE(v3.AddLabel(lbl).HasError());
@ -93,49 +93,49 @@ TYPED_TEST(InfoTest, InfoCheck) {
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(lbl).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateIndex(lbl, prop2).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->DropIndex(lbl, prop).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(lbl, {prop2}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(lbl2, {prop}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_FALSE(unique_acc->CreateUniqueConstraint(lbl3, {prop}).HasError());
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
ASSERT_EQ(unique_acc->DropUniqueConstraint(lbl, {prop2}),
memgraph::storage::UniqueConstraints::DeletionStatus::SUCCESS);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
StorageInfo info = this->storage->GetInfo(true); // force to use configured directory
StorageInfo info = this->storage->GetInfo(true, ReplicationRole::MAIN); // force to use configured directory
ASSERT_EQ(info.vertex_count, 5);
ASSERT_EQ(info.edge_count, 2);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -25,7 +25,7 @@
// NOLINTNEXTLINE(google-build-using-namespace)
using namespace memgraph::storage;
using memgraph::replication::ReplicationRole;
using testing::IsEmpty;
using testing::Types;
using testing::UnorderedElementsAre;
@ -39,7 +39,7 @@ class IndexTest : public testing::Test {
void SetUp() override {
config_ = disk_test_utils::GenerateOnDiskConfig(testSuite);
this->storage = std::make_unique<StorageType>(config_);
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
this->prop_id = acc->NameToProperty("id");
this->prop_val = acc->NameToProperty("val");
this->label1 = acc->NameToLabel("label1");
@ -89,13 +89,13 @@ TYPED_TEST_CASE(IndexTest, StorageTypes);
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(IndexTest, LabelIndexCreate) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->LabelIndexExists(this->label1));
EXPECT_EQ(acc->ListAllIndices().label.size(), 0);
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 10; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? this->label1 : this->label2));
@ -104,19 +104,19 @@ TYPED_TEST(IndexTest, LabelIndexCreate) {
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 10; i < 20; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? this->label1 : this->label2));
@ -138,7 +138,7 @@ TYPED_TEST(IndexTest, LabelIndexCreate) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 10; i < 20; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? this->label1 : this->label2));
@ -159,7 +159,7 @@ TYPED_TEST(IndexTest, LabelIndexCreate) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::OLD), View::OLD),
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::NEW), View::NEW),
@ -179,13 +179,13 @@ TYPED_TEST(IndexTest, LabelIndexCreate) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(IndexTest, LabelIndexDrop) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->LabelIndexExists(this->label1));
EXPECT_EQ(acc->ListAllIndices().label.size(), 0);
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 10; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? this->label1 : this->label2));
@ -194,41 +194,41 @@ TYPED_TEST(IndexTest, LabelIndexDrop) {
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->DropIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->LabelIndexExists(this->label1));
EXPECT_EQ(acc->ListAllIndices().label.size(), 0);
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_TRUE(unique_acc->DropIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->LabelIndexExists(this->label1));
EXPECT_EQ(acc->ListAllIndices().label.size(), 0);
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 10; i < 20; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? this->label1 : this->label2));
@ -237,18 +237,18 @@ TYPED_TEST(IndexTest, LabelIndexDrop) {
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_TRUE(acc->LabelIndexExists(this->label1));
EXPECT_THAT(acc->ListAllIndices().label, UnorderedElementsAre(this->label1));
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::OLD), View::OLD),
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
@ -274,17 +274,17 @@ TYPED_TEST(IndexTest, LabelIndexBasic) {
// vertices.
// 4. Delete even numbered vertices.
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label2).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllIndices().label, UnorderedElementsAre(this->label1, this->label2));
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::OLD), View::OLD), IsEmpty());
EXPECT_THAT(this->GetIds(acc->Vertices(this->label2, View::OLD), View::OLD), IsEmpty());
@ -347,18 +347,18 @@ TYPED_TEST(IndexTest, LabelIndexDuplicateVersions) {
// the same vertex in the index (they only differ by the timestamp). This test
// checks that duplicates are properly filtered out.
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label2).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 5; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
@ -370,7 +370,7 @@ TYPED_TEST(IndexTest, LabelIndexDuplicateVersions) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, View::OLD), View::OLD), UnorderedElementsAre(0, 1, 2, 3, 4));
for (auto vertex : acc->Vertices(View::OLD)) {
@ -395,19 +395,19 @@ TYPED_TEST(IndexTest, LabelIndexDuplicateVersions) {
TYPED_TEST(IndexTest, LabelIndexTransactionalIsolation) {
// Check that transactions only see entries they are supposed to see.
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label2).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc_before = this->storage->Access();
auto acc = this->storage->Access();
auto acc_after = this->storage->Access();
auto acc_before = this->storage->Access(ReplicationRole::MAIN);
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto acc_after = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 5; ++i) {
auto vertex = this->CreateVertex(acc.get());
@ -422,7 +422,7 @@ TYPED_TEST(IndexTest, LabelIndexTransactionalIsolation) {
ASSERT_NO_ERROR(acc->Commit());
auto acc_after_commit = this->storage->Access();
auto acc_after_commit = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc_before->Vertices(this->label1, View::NEW), View::NEW), IsEmpty());
@ -436,17 +436,17 @@ TYPED_TEST(IndexTest, LabelIndexTransactionalIsolation) {
TYPED_TEST(IndexTest, LabelIndexCountEstimate) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label2).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 20; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(i % 3 ? this->label1 : this->label2));
@ -460,23 +460,23 @@ TYPED_TEST(IndexTest, LabelIndexCountEstimate) {
TYPED_TEST(IndexTest, LabelIndexDeletedVertex) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::DiskStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
auto vertex2 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex2.AddLabel(this->label1));
EXPECT_THAT(this->GetIds(acc1->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1));
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex_to_delete = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW);
auto res = acc2->DeleteVertex(&*vertex_to_delete);
ASSERT_FALSE(res.HasError());
ASSERT_NO_ERROR(acc2->Commit());
auto acc3 = this->storage->Access();
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc3->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(1));
}
}
@ -484,23 +484,23 @@ TYPED_TEST(IndexTest, LabelIndexDeletedVertex) {
TYPED_TEST(IndexTest, LabelIndexRemoveIndexedLabel) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::DiskStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
auto vertex2 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex2.AddLabel(this->label1));
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc2->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1));
auto vertex_to_delete = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW);
auto res = vertex_to_delete->RemoveLabel(this->label1);
ASSERT_FALSE(res.HasError());
ASSERT_NO_ERROR(acc2->Commit());
auto acc3 = this->storage->Access();
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc3->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(1));
}
}
@ -508,17 +508,17 @@ TYPED_TEST(IndexTest, LabelIndexRemoveIndexedLabel) {
TYPED_TEST(IndexTest, LabelIndexRemoveAndAddIndexedLabel) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::DiskStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
auto vertex2 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex2.AddLabel(this->label1));
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc2->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1));
auto vertex_to_delete = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW);
auto res_remove = vertex_to_delete->RemoveLabel(this->label1);
@ -526,7 +526,7 @@ TYPED_TEST(IndexTest, LabelIndexRemoveAndAddIndexedLabel) {
auto res_add = vertex_to_delete->AddLabel(this->label1);
ASSERT_FALSE(res_add.HasError());
ASSERT_NO_ERROR(acc2->Commit());
auto acc3 = this->storage->Access();
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc3->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1));
}
}
@ -537,11 +537,11 @@ TYPED_TEST(IndexTest, LabelIndexClearOldDataFromDisk) {
static_cast<memgraph::storage::DiskLabelIndex *>(this->storage->indices_.label_index_.get());
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop_val, PropertyValue(10)));
@ -550,14 +550,14 @@ TYPED_TEST(IndexTest, LabelIndexClearOldDataFromDisk) {
auto *tx_db = disk_label_index->GetRocksDBStorage()->db_;
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc2 = this->storage->Access(std::nullopt);
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->FindVertex(vertex.Gid(), memgraph::storage::View::NEW).value();
ASSERT_TRUE(vertex2.SetProperty(this->prop_val, memgraph::storage::PropertyValue(10)).HasValue());
ASSERT_FALSE(acc2->Commit().HasError());
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc3 = this->storage->Access(std::nullopt);
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
auto vertex3 = acc3->FindVertex(vertex.Gid(), memgraph::storage::View::NEW).value();
ASSERT_TRUE(vertex3.SetProperty(this->prop_val, memgraph::storage::PropertyValue(15)).HasValue());
ASSERT_FALSE(acc3->Commit().HasError());
@ -569,92 +569,92 @@ TYPED_TEST(IndexTest, LabelIndexClearOldDataFromDisk) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(IndexTest, LabelPropertyIndexCreateAndDrop) {
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_EQ(acc->ListAllIndices().label_property.size(), 0);
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_id).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_TRUE(acc->LabelPropertyIndexExists(this->label1, this->prop_id));
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllIndices().label_property,
UnorderedElementsAre(std::make_pair(this->label1, this->prop_id)));
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->LabelPropertyIndexExists(this->label2, this->prop_id));
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_TRUE(unique_acc->CreateIndex(this->label1, this->prop_id).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllIndices().label_property,
UnorderedElementsAre(std::make_pair(this->label1, this->prop_id)));
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label2, this->prop_id).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_TRUE(acc->LabelPropertyIndexExists(this->label2, this->prop_id));
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(
acc->ListAllIndices().label_property,
UnorderedElementsAre(std::make_pair(this->label1, this->prop_id), std::make_pair(this->label2, this->prop_id)));
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->DropIndex(this->label1, this->prop_id).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->LabelPropertyIndexExists(this->label1, this->prop_id));
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(acc->ListAllIndices().label_property,
UnorderedElementsAre(std::make_pair(this->label2, this->prop_id)));
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_TRUE(unique_acc->DropIndex(this->label1, this->prop_id).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->DropIndex(this->label2, this->prop_id).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_FALSE(acc->LabelPropertyIndexExists(this->label2, this->prop_id));
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_EQ(acc->ListAllIndices().label_property.size(), 0);
}
}
@ -667,17 +667,17 @@ TYPED_TEST(IndexTest, LabelPropertyIndexCreateAndDrop) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(IndexTest, LabelPropertyIndexBasic) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label2, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, this->prop_val, View::OLD), View::OLD), IsEmpty());
for (int i = 0; i < 10; ++i) {
@ -762,13 +762,13 @@ TYPED_TEST(IndexTest, LabelPropertyIndexBasic) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(IndexTest, LabelPropertyIndexDuplicateVersions) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 5; ++i) {
auto vertex = this->CreateVertex(acc.get());
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
@ -782,7 +782,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexDuplicateVersions) {
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, this->prop_val, View::OLD), View::OLD),
UnorderedElementsAre(0, 1, 2, 3, 4));
@ -809,14 +809,14 @@ TYPED_TEST(IndexTest, LabelPropertyIndexDuplicateVersions) {
// NOLINTNEXTLINE(hicpp-special-member-functions)
TYPED_TEST(IndexTest, LabelPropertyIndexTransactionalIsolation) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc_before = this->storage->Access();
auto acc = this->storage->Access();
auto acc_after = this->storage->Access();
auto acc_before = this->storage->Access(ReplicationRole::MAIN);
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto acc_after = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 5; ++i) {
auto vertex = this->CreateVertex(acc.get());
@ -833,7 +833,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexTransactionalIsolation) {
ASSERT_NO_ERROR(acc->Commit());
auto acc_after_commit = this->storage->Access();
auto acc_after_commit = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc_before->Vertices(this->label1, this->prop_val, View::NEW), View::NEW), IsEmpty());
@ -852,13 +852,13 @@ TYPED_TEST(IndexTest, LabelPropertyIndexFiltering) {
// properly.
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 10; ++i) {
auto vertex = this->CreateVertex(acc.get());
@ -868,7 +868,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexFiltering) {
ASSERT_NO_ERROR(acc->Commit());
}
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 0; i < 5; ++i) {
EXPECT_THAT(this->GetIds(acc->Vertices(this->label1, this->prop_val, PropertyValue(i), View::OLD)),
UnorderedElementsAre(2 * i, 2 * i + 1));
@ -926,12 +926,12 @@ TYPED_TEST(IndexTest, LabelPropertyIndexFiltering) {
TYPED_TEST(IndexTest, LabelPropertyIndexCountEstimate) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (int i = 1; i <= 10; ++i) {
for (int j = 0; j < i; ++j) {
auto vertex = this->CreateVertex(acc.get());
@ -954,7 +954,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexCountEstimate) {
TYPED_TEST(IndexTest, LabelPropertyIndexMixedIteration) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
@ -993,7 +993,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexMixedIteration) {
// Create vertices, each with one of the values above.
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
for (const auto &value : values) {
auto v = acc->CreateVertex();
ASSERT_TRUE(v.AddLabel(this->label1).HasValue());
@ -1004,7 +1004,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexMixedIteration) {
// Verify that all nodes are in the index.
{
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto iterable = acc->Vertices(this->label1, this->prop_val, View::OLD);
auto it = iterable.begin();
for (const auto &value : values) {
@ -1021,7 +1021,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexMixedIteration) {
auto verify = [&](const std::optional<memgraph::utils::Bound<PropertyValue>> &from,
const std::optional<memgraph::utils::Bound<PropertyValue>> &to,
const std::vector<PropertyValue> &expected) {
auto acc = this->storage->Access();
auto acc = this->storage->Access(ReplicationRole::MAIN);
auto iterable = acc->Vertices(this->label1, this->prop_val, from, to, View::OLD);
size_t i = 0;
for (auto it = iterable.begin(); it != iterable.end(); ++it, ++i) {
@ -1166,11 +1166,11 @@ TYPED_TEST(IndexTest, LabelPropertyIndexMixedIteration) {
TYPED_TEST(IndexTest, LabelPropertyIndexDeletedVertex) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::DiskStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
@ -1183,13 +1183,13 @@ TYPED_TEST(IndexTest, LabelPropertyIndexDeletedVertex) {
EXPECT_THAT(this->GetIds(acc1->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1));
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex_to_delete = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW);
auto res = acc2->DeleteVertex(&*vertex_to_delete);
ASSERT_FALSE(res.HasError());
ASSERT_NO_ERROR(acc2->Commit());
auto acc3 = this->storage->Access();
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc3->Vertices(this->label1, this->prop_val, View::NEW), View::NEW),
UnorderedElementsAre(1));
}
@ -1199,11 +1199,11 @@ TYPED_TEST(IndexTest, LabelPropertyIndexDeletedVertex) {
TYPED_TEST(IndexTest, LabelPropertyIndexRemoveIndexedLabel) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::DiskStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
@ -1216,13 +1216,13 @@ TYPED_TEST(IndexTest, LabelPropertyIndexRemoveIndexedLabel) {
EXPECT_THAT(this->GetIds(acc1->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1));
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex_to_delete = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW);
auto res = vertex_to_delete->RemoveLabel(this->label1);
ASSERT_FALSE(res.HasError());
ASSERT_NO_ERROR(acc2->Commit());
auto acc3 = this->storage->Access();
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
EXPECT_THAT(this->GetIds(acc3->Vertices(this->label1, this->prop_val, View::NEW), View::NEW),
UnorderedElementsAre(1));
}
@ -1231,11 +1231,11 @@ TYPED_TEST(IndexTest, LabelPropertyIndexRemoveIndexedLabel) {
TYPED_TEST(IndexTest, LabelPropertyIndexRemoveAndAddIndexedLabel) {
if constexpr ((std::is_same_v<TypeParam, memgraph::storage::DiskStorage>)) {
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex1 = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex1.AddLabel(this->label1));
@ -1248,7 +1248,7 @@ TYPED_TEST(IndexTest, LabelPropertyIndexRemoveAndAddIndexedLabel) {
EXPECT_THAT(this->GetIds(acc1->Vertices(this->label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1));
ASSERT_NO_ERROR(acc1->Commit());
auto acc2 = this->storage->Access();
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto target_vertex = acc2->FindVertex(vertex1.Gid(), memgraph::storage::View::NEW);
auto remove_res = target_vertex->RemoveLabel(this->label1);
ASSERT_FALSE(remove_res.HasError());
@ -1264,11 +1264,11 @@ TYPED_TEST(IndexTest, LabelPropertyIndexClearOldDataFromDisk) {
static_cast<memgraph::storage::DiskLabelPropertyIndex *>(this->storage->indices_.label_property_index_.get());
{
auto unique_acc = this->storage->UniqueAccess();
auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN);
EXPECT_FALSE(unique_acc->CreateIndex(this->label1, this->prop_val).HasError());
ASSERT_NO_ERROR(unique_acc->Commit());
}
auto acc1 = this->storage->Access();
auto acc1 = this->storage->Access(ReplicationRole::MAIN);
auto vertex = this->CreateVertex(acc1.get());
ASSERT_NO_ERROR(vertex.AddLabel(this->label1));
ASSERT_NO_ERROR(vertex.SetProperty(this->prop_val, PropertyValue(10)));
@ -1277,14 +1277,14 @@ TYPED_TEST(IndexTest, LabelPropertyIndexClearOldDataFromDisk) {
auto *tx_db = disk_label_property_index->GetRocksDBStorage()->db_;
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc2 = this->storage->Access(std::nullopt);
auto acc2 = this->storage->Access(ReplicationRole::MAIN);
auto vertex2 = acc2->FindVertex(vertex.Gid(), memgraph::storage::View::NEW).value();
ASSERT_TRUE(vertex2.SetProperty(this->prop_val, memgraph::storage::PropertyValue(10)).HasValue());
ASSERT_FALSE(acc2->Commit().HasError());
ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1);
auto acc3 = this->storage->Access(std::nullopt);
auto acc3 = this->storage->Access(ReplicationRole::MAIN);
auto vertex3 = acc3->FindVertex(vertex.Gid(), memgraph::storage::View::NEW).value();
ASSERT_TRUE(vertex3.SetProperty(this->prop_val, memgraph::storage::PropertyValue(15)).HasValue());
ASSERT_FALSE(acc3->Commit().HasError());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -16,6 +16,7 @@
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/isolation_level.hpp"
#include "utils/on_scope_exit.hpp"
using memgraph::replication::ReplicationRole;
namespace {
int64_t VerticesCount(memgraph::storage::Storage::Accessor *accessor) {
@ -44,9 +45,9 @@ class StorageIsolationLevelTest : public ::testing::TestWithParam<memgraph::stor
void TestVisibility(std::unique_ptr<memgraph::storage::Storage> &storage,
const memgraph::storage::IsolationLevel &default_isolation_level,
const memgraph::storage::IsolationLevel &override_isolation_level) {
auto creator = storage->Access();
auto default_isolation_level_reader = storage->Access();
auto override_isolation_level_reader = storage->Access(override_isolation_level);
auto creator = storage->Access(ReplicationRole::MAIN);
auto default_isolation_level_reader = storage->Access(ReplicationRole::MAIN);
auto override_isolation_level_reader = storage->Access(ReplicationRole::MAIN, override_isolation_level);
ASSERT_EQ(VerticesCount(default_isolation_level_reader.get()), 0);
ASSERT_EQ(VerticesCount(override_isolation_level_reader.get()), 0);
@ -89,7 +90,7 @@ class StorageIsolationLevelTest : public ::testing::TestWithParam<memgraph::stor
ASSERT_FALSE(override_isolation_level_reader->Commit().HasError());
SCOPED_TRACE("Visibility after a new transaction is started");
auto verifier = storage->Access();
auto verifier = storage->Access(ReplicationRole::MAIN);
ASSERT_EQ(VerticesCount(verifier.get()), iteration_count);
ASSERT_FALSE(verifier->Commit().HasError());
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -458,18 +458,18 @@ TEST_F(ReplicationTest, MultipleSynchronousReplicationTest) {
ASSERT_FALSE(acc->Commit().HasError());
}
const auto check_replica = [&](Storage *replica_store) {
auto acc = replica_store->Access();
const auto check_replica = [&](memgraph::dbms::Database &replica_database) {
auto acc = replica_database.Access();
const auto v = acc->FindVertex(*vertex_gid, View::OLD);
ASSERT_TRUE(v);
const auto labels = v->Labels(View::OLD);
ASSERT_TRUE(labels.HasValue());
ASSERT_THAT(*labels, UnorderedElementsAre(replica_store->NameToLabel(vertex_label)));
ASSERT_THAT(*labels, UnorderedElementsAre(replica_database.storage()->NameToLabel(vertex_label)));
ASSERT_FALSE(acc->Commit().HasError());
};
check_replica(replica1.db.storage());
check_replica(replica2.db.storage());
check_replica(replica1.db);
check_replica(replica2.db);
auto handler = main.repl_handler;
handler.UnregisterReplica(replicas[1]);
@ -482,7 +482,7 @@ TEST_F(ReplicationTest, MultipleSynchronousReplicationTest) {
// REPLICA1 should contain the new vertex
{
auto acc = replica1.db.storage()->Access();
auto acc = replica1.db.Access();
const auto v = acc->FindVertex(*vertex_gid, View::OLD);
ASSERT_TRUE(v);
ASSERT_FALSE(acc->Commit().HasError());
@ -490,7 +490,7 @@ TEST_F(ReplicationTest, MultipleSynchronousReplicationTest) {
// REPLICA2 should not contain the new vertex
{
auto acc = replica2.db.storage()->Access();
auto acc = replica2.db.Access();
const auto v = acc->FindVertex(*vertex_gid, View::OLD);
ASSERT_FALSE(v);
ASSERT_FALSE(acc->Commit().HasError());
@ -677,7 +677,7 @@ TEST_F(ReplicationTest, BasicAsynchronousReplicationTest) {
}
ASSERT_TRUE(std::all_of(created_vertices.begin(), created_vertices.end(), [&](const auto vertex_gid) {
auto acc = replica_async.db.storage()->Access();
auto acc = replica_async.db.Access();
auto v = acc->FindVertex(vertex_gid, View::OLD);
const bool exists = v.has_value();
EXPECT_FALSE(acc->Commit().HasError());
@ -726,13 +726,13 @@ TEST_F(ReplicationTest, EpochTest) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = replica1.db.storage()->Access();
auto acc = replica1.db.Access();
const auto v = acc->FindVertex(*vertex_gid, View::OLD);
ASSERT_TRUE(v);
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = replica2.db.storage()->Access();
auto acc = replica2.db.Access();
const auto v = acc->FindVertex(*vertex_gid, View::OLD);
ASSERT_TRUE(v);
ASSERT_FALSE(acc->Commit().HasError());
@ -759,14 +759,14 @@ TEST_F(ReplicationTest, EpochTest) {
ASSERT_FALSE(acc->Commit().HasError());
}
{
auto acc = replica1.db.storage()->Access();
auto acc = replica1.db.Access();
auto v = acc->CreateVertex();
vertex_gid.emplace(v.Gid());
ASSERT_FALSE(acc->Commit().HasError());
}
// Replica1 should forward it's vertex to Replica2
{
auto acc = replica2.db.storage()->Access();
auto acc = replica2.db.Access();
const auto v = acc->FindVertex(*vertex_gid, View::OLD);
ASSERT_TRUE(v);
ASSERT_FALSE(acc->Commit().HasError());
@ -795,7 +795,7 @@ TEST_F(ReplicationTest, EpochTest) {
// Replica1 is not compatible with the main so it shouldn't contain
// it's newest vertex
{
auto acc = replica1.db.storage()->Access();
auto acc = replica1.db.Access();
const auto v = acc->FindVertex(*vertex_gid, View::OLD);
ASSERT_FALSE(v);
ASSERT_FALSE(acc->Commit().HasError());

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -44,7 +44,7 @@ class ShowStorageInfoTest : public testing::Test {
};
TEST_F(ShowStorageInfoTest, CountOnAbort) {
auto acc = this->storage->Access();
auto acc = this->storage->Access(memgraph::replication::ReplicationRole::MAIN);
auto src_vertex = acc->CreateVertex();
auto dest_vertex = acc->CreateVertex();
auto et = acc->NameToEdgeType("et5");

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -44,8 +44,8 @@ TEST_P(StorageModeTest, Mode) {
.transaction{.isolation_level = memgraph::storage::IsolationLevel::SNAPSHOT_ISOLATION}});
static_cast<memgraph::storage::InMemoryStorage *>(storage.get())->SetStorageMode(storage_mode);
auto creator = storage->Access();
auto other_analytics_mode_reader = storage->Access();
auto creator = storage->Access(memgraph::replication::ReplicationRole::MAIN);
auto other_analytics_mode_reader = storage->Access(memgraph::replication::ReplicationRole::MAIN);
ASSERT_EQ(CountVertices(*creator, memgraph::storage::View::OLD), 0);
ASSERT_EQ(CountVertices(*other_analytics_mode_reader, memgraph::storage::View::OLD), 0);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -37,7 +37,8 @@ class AllTypesFixture : public testing::Test {
std::vector<TypedValue> values_;
memgraph::storage::Config config_{disk_test_utils::GenerateOnDiskConfig(testSuite)};
std::unique_ptr<memgraph::storage::Storage> db{new StorageType(config_)};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{db->Access()};
std::unique_ptr<memgraph::storage::Storage::Accessor> storage_dba{
db->Access(memgraph::replication::ReplicationRole::MAIN)};
memgraph::query::DbAccessor dba{storage_dba.get()};
void SetUp() override {