diff --git a/src/database/distributed_graph_db.cpp b/src/database/distributed_graph_db.cpp index b52eab96e..d9ef38d81 100644 --- a/src/database/distributed_graph_db.cpp +++ b/src/database/distributed_graph_db.cpp @@ -31,6 +31,188 @@ using namespace std::literals::chrono_literals; namespace database { +// Accessors +namespace { + +class DistributedAccessor : public GraphDbAccessor { + distributed::UpdatesRpcClients *updates_clients_{nullptr}; + distributed::DataManager *data_manager_{nullptr}; + + protected: + DistributedAccessor(DistributedGraphDb *db, tx::TransactionId tx_id) + : GraphDbAccessor(*db, tx_id), + updates_clients_(&db->updates_clients()), + data_manager_(&db->data_manager()) {} + + explicit DistributedAccessor(DistributedGraphDb *db) + : GraphDbAccessor(*db), + updates_clients_(&db->updates_clients()), + data_manager_(&db->data_manager()) {} + + public: + bool RemoveVertex(VertexAccessor &vertex_accessor, + bool check_empty = true) override { + if (!vertex_accessor.is_local()) { + auto address = vertex_accessor.address(); + updates_clients_->RemoveVertex(address.worker_id(), transaction_id(), + address.gid(), check_empty); + // We can't know if we are going to be able to remove vertex until + // deferred updates on a remote worker are executed + return true; + } + return GraphDbAccessor::RemoveVertex(vertex_accessor, check_empty); + } + + void RemoveEdge(EdgeAccessor &edge, bool remove_out_edge = true, + bool remove_in_edge = true) override { + if (edge.is_local()) { + return GraphDbAccessor::RemoveEdge(edge, remove_out_edge, remove_in_edge); + } + auto edge_addr = edge.GlobalAddress(); + auto from_addr = db().storage().GlobalizedAddress(edge.from_addr()); + CHECK(edge_addr.worker_id() == from_addr.worker_id()) + << "Edge and it's 'from' vertex not on the same worker"; + auto to_addr = db().storage().GlobalizedAddress(edge.to_addr()); + updates_clients_->RemoveEdge(transaction_id(), edge_addr.worker_id(), + edge_addr.gid(), from_addr.gid(), to_addr); + // Another RPC is necessary only if the first did not handle vertices on + // both sides. + if (edge_addr.worker_id() != to_addr.worker_id()) { + updates_clients_->RemoveInEdge(transaction_id(), to_addr.worker_id(), + to_addr.gid(), edge_addr); + } + } + + storage::EdgeAddress InsertEdgeOnFrom( + VertexAccessor *from, VertexAccessor *to, + const storage::EdgeType &edge_type, + const std::experimental::optional<gid::Gid> &requested_gid, + const std::experimental::optional<int64_t> &cypher_id) override { + if (from->is_local()) { + return GraphDbAccessor::InsertEdgeOnFrom(from, to, edge_type, + requested_gid, cypher_id); + } + auto edge_address = + updates_clients_->CreateEdge(transaction_id(), *from, *to, edge_type); + auto *from_updated = + data_manager_->Elements<Vertex>(transaction_id()).FindNew(from->gid()); + // Create an Edge and insert it into the Cache so we see it locally. + data_manager_->Elements<Edge>(transaction_id()) + .emplace( + edge_address.gid(), nullptr, + std::make_unique<Edge>(from->address(), to->address(), edge_type)); + from_updated->out_.emplace( + db().storage().LocalizedAddressIfPossible(to->address()), edge_address, + edge_type); + return edge_address; + } + + void InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to, + const storage::EdgeType &edge_type, + const storage::EdgeAddress &edge_address) override { + if (to->is_local()) { + return GraphDbAccessor::InsertEdgeOnTo(from, to, edge_type, edge_address); + } + // The RPC call for the `to` side is already handled if `from` is not + // local. + if (from->is_local() || + from->address().worker_id() != to->address().worker_id()) { + updates_clients_->AddInEdge( + transaction_id(), *from, + db().storage().GlobalizedAddress(edge_address), *to, edge_type); + } + auto *to_updated = + data_manager_->Elements<Vertex>(transaction_id()).FindNew(to->gid()); + to_updated->in_.emplace( + db().storage().LocalizedAddressIfPossible(from->address()), + edge_address, edge_type); + } +}; + +class MasterAccessor final : public DistributedAccessor { + distributed::IndexRpcClients *index_rpc_clients_{nullptr}; + int worker_id_{0}; + + public: + explicit MasterAccessor(Master *db, + distributed::IndexRpcClients *index_rpc_clients) + : DistributedAccessor(db), + index_rpc_clients_(index_rpc_clients), + worker_id_(db->WorkerId()) {} + MasterAccessor(Master *db, tx::TransactionId tx_id, + distributed::IndexRpcClients *index_rpc_clients) + : DistributedAccessor(db, tx_id), + index_rpc_clients_(index_rpc_clients), + worker_id_(db->WorkerId()) {} + + void PostCreateIndex(const LabelPropertyIndex::Key &key) override { + std::experimental::optional<std::vector<utils::Future<bool>>> + index_rpc_completions; + + // Notify all workers to create the index + index_rpc_completions.emplace(index_rpc_clients_->GetCreateIndexFutures( + key.label_, key.property_, worker_id_)); + + if (index_rpc_completions) { + // Wait first, check later - so that every thread finishes and none + // terminates - this can probably be optimized in case we fail early so + // that we notify other workers to stop building indexes + for (auto &index_built : *index_rpc_completions) index_built.wait(); + for (auto &index_built : *index_rpc_completions) { + if (!index_built.get()) { + db().storage().label_property_index().DeleteIndex(key); + throw IndexCreationOnWorkerException("Index exists on a worker"); + } + } + } + } + + void PopulateIndexFromBuildIndex( + const LabelPropertyIndex::Key &key) override { + // Notify all workers to start populating an index if we are the master + // since they don't have to wait anymore + std::experimental::optional<std::vector<utils::Future<bool>>> + index_rpc_completions; + index_rpc_completions.emplace(index_rpc_clients_->GetPopulateIndexFutures( + key.label_, key.property_, transaction_id(), worker_id_)); + + // Populate our own storage + GraphDbAccessor::PopulateIndexFromBuildIndex(key); + + // Check if all workers successfully built their indexes and after this we + // can set the index as built + if (index_rpc_completions) { + // Wait first, check later - so that every thread finishes and none + // terminates - this can probably be optimized in case we fail early so + // that we notify other workers to stop building indexes + for (auto &index_built : *index_rpc_completions) index_built.wait(); + for (auto &index_built : *index_rpc_completions) { + if (!index_built.get()) { + db().storage().label_property_index().DeleteIndex(key); + throw IndexCreationOnWorkerException("Index exists on a worker"); + } + } + } + } +}; + +class WorkerAccessor final : public DistributedAccessor { + public: + explicit WorkerAccessor(Worker *db) : DistributedAccessor(db) {} + WorkerAccessor(Worker *db, tx::TransactionId tx_id) + : DistributedAccessor(db, tx_id) {} + + void BuildIndex(storage::Label, storage::Property) override { + // TODO: Rethink BuildIndex API or inheritance. It's rather strange that a + // derived type blocks this functionality. + LOG(FATAL) << "BuildIndex invoked on worker."; + } +}; + +} // namespace + +// GraphDb implementations + namespace impl { template <template <typename TId> class TMapper> @@ -80,12 +262,12 @@ class Master { distributed::BfsRpcClients bfs_subcursor_clients_{ self_, &subcursor_storage_, &rpc_worker_clients_, &data_manager_}; distributed::DurabilityRpcMaster durability_rpc_{rpc_worker_clients_}; - distributed::DataRpcServer data_server_{*self_, server_}; + distributed::DataRpcServer data_server_{self_, &server_}; distributed::DataRpcClients data_clients_{rpc_worker_clients_}; distributed::PlanDispatcher plan_dispatcher_{rpc_worker_clients_}; - distributed::PullRpcClients pull_clients_{rpc_worker_clients_}; + distributed::PullRpcClients pull_clients_{&rpc_worker_clients_, &data_manager_}; distributed::IndexRpcClients index_rpc_clients_{rpc_worker_clients_}; - distributed::UpdatesRpcServer updates_server_{*self_, server_}; + distributed::UpdatesRpcServer updates_server_{self_, &server_}; distributed::UpdatesRpcClients updates_clients_{rpc_worker_clients_}; distributed::DataManager data_manager_{*self_, data_clients_}; distributed::TransactionalCacheCleaner cache_cleaner_{ @@ -159,8 +341,8 @@ Master::Master(Config config) snapshot_creator_->Run( "Snapshot", std::chrono::seconds(impl_->config_.snapshot_cycle_sec), [this] { - GraphDbAccessor dba(*this); - MakeSnapshot(dba); + auto dba = this->Access(); + MakeSnapshot(*dba); }); } @@ -194,11 +376,20 @@ Master::~Master() { // We are not a worker, so we can do a snapshot on exit if it's enabled. Doing // this on the master forces workers to do the same through rpcs if (impl_->config_.snapshot_on_exit) { - GraphDbAccessor dba(*this); - MakeSnapshot(dba); + auto dba = Access(); + MakeSnapshot(*dba); } } +std::unique_ptr<GraphDbAccessor> Master::Access() { + return std::make_unique<MasterAccessor>(this, &impl_->index_rpc_clients_); +} + +std::unique_ptr<GraphDbAccessor> Master::Access(tx::TransactionId tx_id) { + return std::make_unique<MasterAccessor>(this, tx_id, + &impl_->index_rpc_clients_); +} + Storage &Master::storage() { return *impl_->storage_; } durability::WriteAheadLog &Master::wal() { return impl_->wal_; } @@ -299,6 +490,30 @@ distributed::IndexRpcClients &Master::index_rpc_clients() { return impl_->index_rpc_clients_; } +VertexAccessor InsertVertexIntoRemote( + GraphDbAccessor *dba, int worker_id, + const std::vector<storage::Label> &labels, + const std::unordered_map<storage::Property, query::TypedValue> + &properties) { + // TODO: Replace this with virtual call or some other mechanism. + auto *distributed_db = + dynamic_cast<database::DistributedGraphDb *>(&dba->db()); + CHECK(distributed_db); + CHECK(worker_id != distributed_db->WorkerId()) + << "Not allowed to call InsertVertexIntoRemote for local worker"; + auto *updates_clients = &distributed_db->updates_clients(); + auto *data_manager = &distributed_db->data_manager(); + CHECK(updates_clients && data_manager); + gid::Gid gid = updates_clients->CreateVertex(worker_id, dba->transaction_id(), + labels, properties); + auto vertex = std::make_unique<Vertex>(); + vertex->labels_ = labels; + for (auto &kv : properties) vertex->properties_.set(kv.first, kv.second); + data_manager->Elements<Vertex>(dba->transaction_id()) + .emplace(gid, nullptr, std::move(vertex)); + return VertexAccessor({gid, worker_id}, *dba); +} + // Worker namespace impl { @@ -341,13 +556,13 @@ class Worker { &subcursor_storage_}; distributed::BfsRpcClients bfs_subcursor_clients_{ self_, &subcursor_storage_, &rpc_worker_clients_, &data_manager_}; - distributed::DataRpcServer data_server_{*self_, server_}; + distributed::DataRpcServer data_server_{self_, &server_}; distributed::DataRpcClients data_clients_{rpc_worker_clients_}; distributed::PlanConsumer plan_consumer_{server_}; - distributed::ProduceRpcServer produce_server_{*self_, tx_engine_, server_, + distributed::ProduceRpcServer produce_server_{self_, &tx_engine_, server_, plan_consumer_, &data_manager_}; distributed::IndexRpcServer index_rpc_server_{*self_, server_}; - distributed::UpdatesRpcServer updates_server_{*self_, server_}; + distributed::UpdatesRpcServer updates_server_{self_, &server_}; distributed::UpdatesRpcClients updates_clients_{rpc_worker_clients_}; distributed::DataManager data_manager_{*self_, data_clients_}; distributed::WorkerTransactionalCacheCleaner cache_cleaner_{ @@ -373,8 +588,7 @@ Worker::Worker(Config config) // Durability recovery. { // What we should recover. - std::experimental::optional<tx::TransactionId> snapshot_to_recover; - snapshot_to_recover = impl_->cluster_discovery_.snapshot_to_recover(); + auto snapshot_to_recover = impl_->cluster_discovery_.snapshot_to_recover(); // What we recover. std::experimental::optional<durability::RecoveryInfo> recovery_info; @@ -426,6 +640,14 @@ Worker::~Worker() { [](auto &t) { t.set_should_abort(); }); } +std::unique_ptr<GraphDbAccessor> Worker::Access() { + return std::make_unique<WorkerAccessor>(this); +} + +std::unique_ptr<GraphDbAccessor> Worker::Access(tx::TransactionId tx_id) { + return std::make_unique<WorkerAccessor>(this, tx_id); +} + Storage &Worker::storage() { return *impl_->storage_; } durability::WriteAheadLog &Worker::wal() { return impl_->wal_; } diff --git a/src/database/distributed_graph_db.hpp b/src/database/distributed_graph_db.hpp index 3a3aef635..8671cad6a 100644 --- a/src/database/distributed_graph_db.hpp +++ b/src/database/distributed_graph_db.hpp @@ -1,3 +1,5 @@ +/// @file + #pragma once #include "database/graph_db.hpp" @@ -28,6 +30,9 @@ class Worker; /// Abstract base class for concrete distributed versions of GraphDb class DistributedGraphDb : public GraphDb { public: + virtual int WorkerId() const = 0; + virtual std::vector<int> GetWorkerIds() const = 0; + virtual distributed::BfsRpcClients &bfs_subcursor_clients() = 0; virtual distributed::DataRpcClients &data_clients() = 0; virtual distributed::UpdatesRpcServer &updates_server() = 0; @@ -44,6 +49,9 @@ class Master final : public DistributedGraphDb { return GraphDb::Type::DISTRIBUTED_MASTER; } + std::unique_ptr<GraphDbAccessor> Access() override; + std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) override; + Storage &storage() override; durability::WriteAheadLog &wal() override; tx::Engine &tx_engine() override; @@ -89,6 +97,9 @@ class Worker final : public DistributedGraphDb { return GraphDb::Type::DISTRIBUTED_WORKER; } + std::unique_ptr<GraphDbAccessor> Access() override; + std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) override; + Storage &storage() override; durability::WriteAheadLog &wal() override; tx::Engine &tx_engine() override; @@ -124,4 +135,11 @@ class Worker final : public DistributedGraphDb { utils::Scheduler transaction_killer_; }; +/// Creates a new Vertex on the given worker. +/// It is NOT allowed to call this function with this worker's id. +VertexAccessor InsertVertexIntoRemote( + GraphDbAccessor *dba, int worker_id, + const std::vector<storage::Label> &labels, + const std::unordered_map<storage::Property, query::TypedValue> &properties); + } // namespace database diff --git a/src/database/graph_db.cpp b/src/database/graph_db.cpp index 7b8a714f3..552bc18fb 100644 --- a/src/database/graph_db.cpp +++ b/src/database/graph_db.cpp @@ -81,8 +81,8 @@ SingleNode::SingleNode(Config config) snapshot_creator_->Run( "Snapshot", std::chrono::seconds(impl_->config_.snapshot_cycle_sec), [this] { - GraphDbAccessor dba(*this); - this->MakeSnapshot(dba); + auto dba = this->Access(); + this->MakeSnapshot(*dba); }); } @@ -114,11 +114,30 @@ SingleNode::~SingleNode() { [](auto &t) { t.set_should_abort(); }); if (impl_->config_.snapshot_on_exit) { - GraphDbAccessor dba(*this); - MakeSnapshot(dba); + auto dba = this->Access(); + MakeSnapshot(*dba); } } +class SingleNodeAccessor : public GraphDbAccessor { + public: + explicit SingleNodeAccessor(GraphDb &db) : GraphDbAccessor(db) {} + SingleNodeAccessor(GraphDb &db, tx::TransactionId tx_id) + : GraphDbAccessor(db, tx_id) {} +}; + +std::unique_ptr<GraphDbAccessor> SingleNode::Access() { + // NOTE: We are doing a heap allocation to allow polymorphism. If this poses + // performance issues, we may want to have a stack allocated GraphDbAccessor + // which is constructed with a pointer to some global implementation struct + // which contains only pure functions (without any state). + return std::make_unique<SingleNodeAccessor>(*this); +} + +std::unique_ptr<GraphDbAccessor> SingleNode::Access(tx::TransactionId tx_id) { + return std::make_unique<SingleNodeAccessor>(*this, tx_id); +} + Storage &SingleNode::storage() { return *impl_->storage_; } durability::WriteAheadLog &SingleNode::wal() { return impl_->wal_; } @@ -141,10 +160,6 @@ database::Counters &SingleNode::counters() { return impl_->counters_; } void SingleNode::CollectGarbage() { impl_->storage_gc_->CollectGarbage(); } -int SingleNode::WorkerId() const { return impl_->config_.worker_id; } - -std::vector<int> SingleNode::GetWorkerIds() const { return {0}; } - bool SingleNode::MakeSnapshot(GraphDbAccessor &accessor) { const bool status = durability::MakeSnapshot( *this, accessor, fs::path(impl_->config_.durability_directory), diff --git a/src/database/graph_db.hpp b/src/database/graph_db.hpp index 411a5f912..d329c686c 100644 --- a/src/database/graph_db.hpp +++ b/src/database/graph_db.hpp @@ -44,29 +44,29 @@ struct Config { int recovering_cluster_size{0}; }; -/** - * An abstract base class for a SingleNode/Master/Worker graph db. - * - * Always be sure that GraphDb object is destructed before main exits, i. e. - * GraphDb object shouldn't be part of global/static variable, except if its - * destructor is explicitly called before main exits. Consider code: - * - * GraphDb db; // KeyIndex is created as a part of database::Storage - * int main() { - * GraphDbAccessor dba(db); - * auto v = dba.InsertVertex(); - * v.add_label(dba.Label( - * "Start")); // New SkipList is created in KeyIndex for LabelIndex. - * // That SkipList creates SkipListGc which - * // initialises static Executor object. - * return 0; - * } - * - * After main exits: 1. Executor is destructed, 2. KeyIndex is destructed. - * Destructor of KeyIndex calls delete on created SkipLists which destroy - * SkipListGc that tries to use Excutioner object that doesn't exist anymore. - * -> CRASH - */ +class GraphDbAccessor; + +/// An abstract base class for a SingleNode/Master/Worker graph db. +/// +/// Always be sure that GraphDb object is destructed before main exits, i. e. +/// GraphDb object shouldn't be part of global/static variable, except if its +/// destructor is explicitly called before main exits. Consider code: +/// +/// GraphDb db; // KeyIndex is created as a part of database::Storage +/// int main() { +/// GraphDbAccessor dba(db); +/// auto v = dba.InsertVertex(); +/// v.add_label(dba.Label( +/// "Start")); // New SkipList is created in KeyIndex for LabelIndex. +/// // That SkipList creates SkipListGc which +/// // initialises static Executor object. +/// return 0; +/// } +/// +/// After main exits: 1. Executor is destructed, 2. KeyIndex is destructed. +/// Destructor of KeyIndex calls delete on created SkipLists which destroy +/// SkipListGc that tries to use Excutioner object that doesn't exist anymore. +/// -> CRASH class GraphDb { public: enum class Type { SINGLE_NODE, DISTRIBUTED_MASTER, DISTRIBUTED_WORKER }; @@ -80,6 +80,12 @@ class GraphDb { virtual ~GraphDb() {} virtual Type type() const = 0; + + /// Create a new accessor by starting a new transaction. + virtual std::unique_ptr<GraphDbAccessor> Access() = 0; + /// Create an accessor for a running transaction. + virtual std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) = 0; + virtual Storage &storage() = 0; virtual durability::WriteAheadLog &wal() = 0; virtual tx::Engine &tx_engine() = 0; @@ -89,19 +95,17 @@ class GraphDb { virtual storage::ConcurrentIdMapper<storage::Property> &property_mapper() = 0; virtual database::Counters &counters() = 0; virtual void CollectGarbage() = 0; - virtual int WorkerId() const = 0; - virtual std::vector<int> GetWorkerIds() const = 0; - // Makes a snapshot from the visibility of the given accessor + /// Makes a snapshot from the visibility of the given accessor virtual bool MakeSnapshot(GraphDbAccessor &accessor) = 0; - // Releases the storage object safely and creates a new object. - // This is needed because of recovery, otherwise we might try to recover into - // a storage which has already been polluted because of a failed previous - // recovery + /// Releases the storage object safely and creates a new object. + /// This is needed because of recovery, otherwise we might try to recover into + /// a storage which has already been polluted because of a failed previous + /// recovery virtual void ReinitializeStorage() = 0; - /** When this is false, no new transactions should be created. */ + /// When this is false, no new transactions should be created. bool is_accepting_transactions() const { return is_accepting_transactions_; } protected: @@ -118,6 +122,10 @@ class SingleNode final : public GraphDb { ~SingleNode(); Type type() const override { return GraphDb::Type::SINGLE_NODE; } + + std::unique_ptr<GraphDbAccessor> Access() override; + std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) override; + Storage &storage() override; durability::WriteAheadLog &wal() override; tx::Engine &tx_engine() override; @@ -126,8 +134,6 @@ class SingleNode final : public GraphDb { storage::ConcurrentIdMapper<storage::Property> &property_mapper() override; database::Counters &counters() override; void CollectGarbage() override; - int WorkerId() const override; - std::vector<int> GetWorkerIds() const override; bool MakeSnapshot(GraphDbAccessor &accessor) override; void ReinitializeStorage() override; diff --git a/src/database/graph_db_accessor.cpp b/src/database/graph_db_accessor.cpp index 8dd670bae..db48bc4df 100644 --- a/src/database/graph_db_accessor.cpp +++ b/src/database/graph_db_accessor.cpp @@ -1,20 +1,17 @@ #include "database/graph_db_accessor.hpp" -#include <functional> +#include <chrono> +#include <thread> -#include "glog/logging.h" +#include <glog/logging.h> -#include "database/distributed_graph_db.hpp" #include "database/state_delta.hpp" -#include "distributed/data_manager.hpp" -#include "distributed/rpc_worker_clients.hpp" -#include "distributed/updates_rpc_clients.hpp" #include "storage/address_types.hpp" #include "storage/edge.hpp" #include "storage/edge_accessor.hpp" #include "storage/vertex.hpp" #include "storage/vertex_accessor.hpp" -#include "utils/atomic.hpp" +#include "utils/cast.hpp" #include "utils/on_scope_exit.hpp" namespace database { @@ -83,34 +80,6 @@ VertexAccessor GraphDbAccessor::InsertVertex( return va; } -VertexAccessor GraphDbAccessor::InsertVertexIntoRemote( - int worker_id, const std::vector<storage::Label> &labels, - const std::unordered_map<storage::Property, query::TypedValue> - &properties) { - CHECK(worker_id != db().WorkerId()) - << "Not allowed to call InsertVertexIntoRemote for local worker"; - - distributed::UpdatesRpcClients *updates_clients = nullptr; - distributed::DataManager *data_manager = nullptr; - // TODO: Replace this with virtual call or some other mechanism. - if (auto *distributed_db = - dynamic_cast<database::DistributedGraphDb *>(&db())) { - updates_clients = &distributed_db->updates_clients(); - data_manager = &distributed_db->data_manager(); - } - CHECK(updates_clients && data_manager); - gid::Gid gid = updates_clients->CreateVertex(worker_id, transaction_id(), - labels, properties); - - auto vertex = std::make_unique<Vertex>(); - vertex->labels_ = labels; - for (auto &kv : properties) vertex->properties_.set(kv.first, kv.second); - - data_manager->Elements<Vertex>(transaction_id()) - .emplace(gid, nullptr, std::move(vertex)); - return VertexAccessor({gid, worker_id}, *this); -} - std::experimental::optional<VertexAccessor> GraphDbAccessor::FindVertexOptional( gid::Gid gid, bool current_state) { VertexAccessor record_accessor( @@ -144,8 +113,6 @@ EdgeAccessor GraphDbAccessor::FindEdge(gid::Gid gid, bool current_state) { void GraphDbAccessor::BuildIndex(storage::Label label, storage::Property property) { DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted"; - DCHECK(db_.type() != GraphDb::Type::DISTRIBUTED_WORKER) - << "BuildIndex invoked on worker"; db_.storage().index_build_tx_in_progress_.access().insert(transaction_.id_); @@ -157,37 +124,15 @@ void GraphDbAccessor::BuildIndex(storage::Label label, DCHECK(removed) << "Index creation transaction should be inside set"; }); + // Create the index const LabelPropertyIndex::Key key(label, property); if (db_.storage().label_property_index_.CreateIndex(key) == false) { throw IndexExistsException( "Index is either being created by another transaction or already " "exists."); } - - std::experimental::optional<std::vector<utils::Future<bool>>> - index_rpc_completions; - - // Notify all workers to create the index - if (db_.type() == GraphDb::Type::DISTRIBUTED_MASTER) { - // TODO: Replace this with virtual call or some other mechanism. - database::Master *master_db = dynamic_cast<database::Master *>(&db_); - index_rpc_completions.emplace( - master_db->index_rpc_clients().GetCreateIndexFutures( - label, property, this->db_.WorkerId())); - } - - if (index_rpc_completions) { - // Wait first, check later - so that every thread finishes and none - // terminates - this can probably be optimized in case we fail early so that - // we notify other workers to stop building indexes - for (auto &index_built : *index_rpc_completions) index_built.wait(); - for (auto &index_built : *index_rpc_completions) { - if (!index_built.get()) { - db_.storage().label_property_index_.DeleteIndex(key); - throw IndexCreationOnWorkerException("Index exists on a worker"); - } - } - } + // Call the hook for inherited classes. + PostCreateIndex(key); // Everything that happens after the line above ended will be added to the // index automatically, but we still have to add to index everything that @@ -212,21 +157,11 @@ void GraphDbAccessor::BuildIndex(storage::Label label, // This accessor's transaction surely sees everything that happened before // CreateIndex. - GraphDbAccessor dba(db_); - - // Notify all workers to start populating an index if we are the master since - // they don't have to wait anymore - if (db_.type() == GraphDb::Type::DISTRIBUTED_MASTER) { - // TODO: Replace this with virtual call or some other mechanism. - database::Master *master_db = dynamic_cast<database::Master *>(&db_); - index_rpc_completions.emplace( - master_db->index_rpc_clients().GetPopulateIndexFutures( - label, property, dba.transaction_id(), this->db_.WorkerId())); - } + auto dba = db_.Access(); // Add transaction to the build_tx_in_progress as this transaction doesn't // change data and shouldn't block other parallel index creations - auto read_transaction_id = dba.transaction().id_; + auto read_transaction_id = dba->transaction().id_; db_.storage().index_build_tx_in_progress_.access().insert( read_transaction_id); // on function exit remove the read transaction from build_tx_in_progress @@ -236,25 +171,10 @@ void GraphDbAccessor::BuildIndex(storage::Label label, DCHECK(removed) << "Index building (read) transaction should be inside set"; }); - dba.PopulateIndex(key); + dba->PopulateIndexFromBuildIndex(key); - // Check if all workers sucesfully built their indexes and after this we can - // set the index as built - if (index_rpc_completions) { - // Wait first, check later - so that every thread finishes and none - // terminates - this can probably be optimized in case we fail early so that - // we notify other workers to stop building indexes - for (auto &index_built : *index_rpc_completions) index_built.wait(); - for (auto &index_built : *index_rpc_completions) { - if (!index_built.get()) { - db_.storage().label_property_index_.DeleteIndex(key); - throw IndexCreationOnWorkerException("Index exists on a worker"); - } - } - } - - dba.EnableIndex(key); - dba.Commit(); + dba->EnableIndex(key); + dba->Commit(); } void GraphDbAccessor::EnableIndex(const LabelPropertyIndex::Key &key) { @@ -376,22 +296,6 @@ int64_t GraphDbAccessor::VerticesCount( bool GraphDbAccessor::RemoveVertex(VertexAccessor &vertex_accessor, bool check_empty) { DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted"; - - if (!vertex_accessor.is_local()) { - auto address = vertex_accessor.address(); - distributed::UpdatesRpcClients *updates_clients = nullptr; - // TODO: Replace this with virtual call or some other mechanism. - if (auto *distributed_db = - dynamic_cast<database::DistributedGraphDb *>(&db())) { - updates_clients = &distributed_db->updates_clients(); - } - CHECK(updates_clients); - updates_clients->RemoveVertex(address.worker_id(), transaction_id(), - address.gid(), check_empty); - // We can't know if we are going to be able to remove vertex until deferred - // updates on a remote worker are executed - return true; - } vertex_accessor.SwitchNew(); // it's possible the vertex was removed already in this transaction // due to it getting matched multiple times by some patterns @@ -431,86 +335,52 @@ EdgeAccessor GraphDbAccessor::InsertEdge( std::experimental::optional<int64_t> cypher_id) { DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted"; - // The address of an edge we'll create. - storage::EdgeAddress edge_address; - - Vertex *from_updated; - if (from.is_local()) { - auto edge_accessor = InsertOnlyEdge(from.address(), to.address(), edge_type, - requested_gid, cypher_id); - edge_address = edge_accessor.address(); - - from.SwitchNew(); - from_updated = &from.update(); - - // TODO when preparing WAL for distributed, most likely never use - // `CREATE_EDGE`, but always have it split into 3 parts (edge insertion, - // in/out modification). - wal().Emplace(database::StateDelta::CreateEdge( - transaction_.id_, edge_accessor.gid(), edge_accessor.cypher_id(), - from.gid(), to.gid(), edge_type, EdgeTypeName(edge_type))); - - } else { - distributed::UpdatesRpcClients *updates_clients = nullptr; - distributed::DataManager *data_manager = nullptr; - // TODO: Replace this with virtual call or some other mechanism. - if (auto *distributed_db = - dynamic_cast<database::DistributedGraphDb *>(&db())) { - updates_clients = &distributed_db->updates_clients(); - data_manager = &distributed_db->data_manager(); - } - CHECK(updates_clients && data_manager); - edge_address = - updates_clients->CreateEdge(transaction_id(), from, to, edge_type); - - from_updated = - data_manager->Elements<Vertex>(transaction_id()).FindNew(from.gid()); - - // Create an Edge and insert it into the Cache so we see it locally. - data_manager->Elements<Edge>(transaction_id()) - .emplace( - edge_address.gid(), nullptr, - std::make_unique<Edge>(from.address(), to.address(), edge_type)); - } - from_updated->out_.emplace( - db_.storage().LocalizedAddressIfPossible(to.address()), edge_address, - edge_type); - - Vertex *to_updated; - if (to.is_local()) { - // ensure that the "to" accessor has the latest version (Switch new) - // WARNING: must do that after the above "from.update()" for cases when - // we are creating a cycle and "from" and "to" are the same vlist - to.SwitchNew(); - to_updated = &to.update(); - } else { - distributed::UpdatesRpcClients *updates_clients = nullptr; - distributed::DataManager *data_manager = nullptr; - // TODO: Replace this with virtual call or some other mechanism. - if (auto *distributed_db = - dynamic_cast<database::DistributedGraphDb *>(&db())) { - updates_clients = &distributed_db->updates_clients(); - data_manager = &distributed_db->data_manager(); - } - CHECK(updates_clients && data_manager); - // The RPC call for the `to` side is already handled if `from` is not local. - if (from.is_local() || - from.address().worker_id() != to.address().worker_id()) { - updates_clients->AddInEdge(transaction_id(), from, - db().storage().GlobalizedAddress(edge_address), - to, edge_type); - } - to_updated = - data_manager->Elements<Vertex>(transaction_id()).FindNew(to.gid()); - } - to_updated->in_.emplace( - db_.storage().LocalizedAddressIfPossible(from.address()), edge_address, - edge_type); + auto edge_address = + InsertEdgeOnFrom(&from, &to, edge_type, requested_gid, cypher_id); + InsertEdgeOnTo(&from, &to, edge_type, edge_address); return EdgeAccessor(edge_address, *this, from.address(), to.address(), edge_type); } +storage::EdgeAddress GraphDbAccessor::InsertEdgeOnFrom( + VertexAccessor *from, VertexAccessor *to, + const storage::EdgeType &edge_type, + const std::experimental::optional<gid::Gid> &requested_gid, + const std::experimental::optional<int64_t> &cypher_id) { + auto edge_accessor = InsertOnlyEdge(from->address(), to->address(), edge_type, + requested_gid, cypher_id); + auto edge_address = edge_accessor.address(); + + from->SwitchNew(); + auto from_updated = &from->update(); + + // TODO when preparing WAL for distributed, most likely never use + // `CREATE_EDGE`, but always have it split into 3 parts (edge insertion, + // in/out modification). + wal().Emplace(database::StateDelta::CreateEdge( + transaction_.id_, edge_accessor.gid(), edge_accessor.cypher_id(), + from->gid(), to->gid(), edge_type, EdgeTypeName(edge_type))); + + from_updated->out_.emplace( + db_.storage().LocalizedAddressIfPossible(to->address()), edge_address, + edge_type); + return edge_address; +} + +void GraphDbAccessor::InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to, + const storage::EdgeType &edge_type, + const storage::EdgeAddress &edge_address) { + // ensure that the "to" accessor has the latest version (Switch new) + // WARNING: must do that after the above "from->update()" for cases when + // we are creating a cycle and "from" and "to" are the same vlist + to->SwitchNew(); + auto *to_updated = &to->update(); + to_updated->in_.emplace( + db_.storage().LocalizedAddressIfPossible(from->address()), edge_address, + edge_type); +} + EdgeAccessor GraphDbAccessor::InsertOnlyEdge( storage::VertexAddress from, storage::VertexAddress to, storage::EdgeType edge_type, @@ -541,41 +411,16 @@ int64_t GraphDbAccessor::EdgesCount() const { void GraphDbAccessor::RemoveEdge(EdgeAccessor &edge, bool remove_out_edge, bool remove_in_edge) { DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted"; - if (edge.is_local()) { - // it's possible the edge was removed already in this transaction - // due to it getting matched multiple times by some patterns - // we can only delete it once, so check if it's already deleted - edge.SwitchNew(); - if (edge.current().is_expired_by(transaction_)) return; - if (remove_out_edge) edge.from().RemoveOutEdge(edge.address()); - if (remove_in_edge) edge.to().RemoveInEdge(edge.address()); + // it's possible the edge was removed already in this transaction + // due to it getting matched multiple times by some patterns + // we can only delete it once, so check if it's already deleted + edge.SwitchNew(); + if (edge.current().is_expired_by(transaction_)) return; + if (remove_out_edge) edge.from().RemoveOutEdge(edge.address()); + if (remove_in_edge) edge.to().RemoveInEdge(edge.address()); - edge.address().local()->remove(edge.current_, transaction_); - wal().Emplace( - database::StateDelta::RemoveEdge(transaction_.id_, edge.gid())); - } else { - auto edge_addr = edge.GlobalAddress(); - auto from_addr = db().storage().GlobalizedAddress(edge.from_addr()); - CHECK(edge_addr.worker_id() == from_addr.worker_id()) - << "Edge and it's 'from' vertex not on the same worker"; - auto to_addr = db().storage().GlobalizedAddress(edge.to_addr()); - distributed::UpdatesRpcClients *updates_clients = nullptr; - // TODO: Replace this with virtual call or some other mechanism. - if (auto *distributed_db = - dynamic_cast<database::DistributedGraphDb *>(&db())) { - updates_clients = &distributed_db->updates_clients(); - } - CHECK(updates_clients); - updates_clients->RemoveEdge(transaction_id(), edge_addr.worker_id(), - edge_addr.gid(), from_addr.gid(), to_addr); - - // Another RPC is necessary only if the first did not handle vertices on - // both sides. - if (edge_addr.worker_id() != to_addr.worker_id()) { - updates_clients->RemoveInEdge(transaction_id(), to_addr.worker_id(), - to_addr.gid(), edge_addr); - } - } + edge.address().local()->remove(edge.current_, transaction_); + wal().Emplace(database::StateDelta::RemoveEdge(transaction_.id_, edge.gid())); } storage::Label GraphDbAccessor::Label(const std::string &label_name) { diff --git a/src/database/graph_db_accessor.hpp b/src/database/graph_db_accessor.hpp index 9de9f1b4d..098eedd8f 100644 --- a/src/database/graph_db_accessor.hpp +++ b/src/database/graph_db_accessor.hpp @@ -1,24 +1,22 @@ #pragma once #include <experimental/optional> -#include <unordered_map> +#include <string> #include <vector> -#include "cppitertools/filter.hpp" -#include "cppitertools/imap.hpp" -#include "glog/logging.h" +#include <cppitertools/filter.hpp> +#include <cppitertools/imap.hpp> +#include <glog/logging.h> #include "database/graph_db.hpp" -#include "distributed/cache.hpp" -#include "query/typed_value.hpp" #include "storage/address_types.hpp" #include "storage/edge_accessor.hpp" #include "storage/types.hpp" #include "storage/vertex_accessor.hpp" -#include "transactions/engine_single_node.hpp" #include "transactions/transaction.hpp" #include "transactions/type.hpp" #include "utils/bound.hpp" +#include "utils/exceptions.hpp" namespace database { @@ -33,27 +31,28 @@ class IndexCreationOnWorkerException : public utils::BasicException { }; /** - * An accessor for the database object: exposes functions for operating on the + * Base accessor for the database object: exposes functions for operating on the * database. All the functions in this class should be self-sufficient: for * example the function for creating a new Vertex should take care of all the * book-keeping around the creation. */ - class GraphDbAccessor { // We need to make friends with this guys since they need to access private // methods for updating indices. friend class ::RecordAccessor<Vertex>; - friend class ::RecordAccessor<Edge>; friend class ::VertexAccessor; - friend class ::EdgeAccessor; - public: + protected: + // Construction should only be done through GraphDb::Access function and + // concrete GraphDbAccessor type. + /// Creates a new accessor by starting a new transaction. explicit GraphDbAccessor(GraphDb &db); - /// Creates an accessor for a running transaction. GraphDbAccessor(GraphDb &db, tx::TransactionId tx_id); - ~GraphDbAccessor(); + + public: + virtual ~GraphDbAccessor(); GraphDbAccessor(const GraphDbAccessor &other) = delete; GraphDbAccessor(GraphDbAccessor &&other) = delete; @@ -82,13 +81,6 @@ class GraphDbAccessor { std::experimental::optional<int64_t> cypher_id = std::experimental::nullopt); - /** Creates a new Vertex on the given worker. It is NOT allowed to call this - * function with this worker's id. */ - VertexAccessor InsertVertexIntoRemote( - int worker_id, const std::vector<storage::Label> &labels, - const std::unordered_map<storage::Property, query::TypedValue> - &properties); - /** * Removes the vertex of the given accessor. If the vertex has any outgoing or * incoming edges, it is not deleted. See `DetachRemoveVertex` if you want to @@ -102,7 +94,8 @@ class GraphDbAccessor { * before deletion. * @return If or not the vertex was deleted. */ - bool RemoveVertex(VertexAccessor &vertex_accessor, bool check_empty = true); + virtual bool RemoveVertex(VertexAccessor &vertex_accessor, + bool check_empty = true); /** * Removes the vertex of the given accessor along with all it's outgoing @@ -340,8 +333,8 @@ class GraphDbAccessor { * @param remove_in_edge If the edge should be removed from the its * destination side. */ - void RemoveEdge(EdgeAccessor &edge, bool remove_out_edge = true, - bool remove_in_edge = true); + virtual void RemoveEdge(EdgeAccessor &edge, bool remove_out_edge = true, + bool remove_in_edge = true); /** * Obtains the edge for the given ID. If there is no edge for the given @@ -444,7 +437,7 @@ class GraphDbAccessor { * @param label - label to build for * @param property - property to build for */ - void BuildIndex(storage::Label label, storage::Property property); + virtual void BuildIndex(storage::Label label, storage::Property property); /// Populates index with vertices containing the key void PopulateIndex(const LabelPropertyIndex::Key &key); @@ -607,6 +600,34 @@ class GraphDbAccessor { /* Returns a list of index names present in the database. */ std::vector<std::string> IndexInfo() const; + protected: + /** Called in `BuildIndex` after creating an index, but before populating. */ + virtual void PostCreateIndex(const LabelPropertyIndex::Key &key) {} + + /** Populates the index from a *new* transaction after creating the index. */ + virtual void PopulateIndexFromBuildIndex(const LabelPropertyIndex::Key &key) { + PopulateIndex(key); + } + + /** + * Insert a new edge to `from` vertex and return the address. + * Called from `InsertEdge` as the first step in edge insertion. + * */ + virtual storage::EdgeAddress InsertEdgeOnFrom( + VertexAccessor *from, VertexAccessor *to, + const storage::EdgeType &edge_type, + const std::experimental::optional<gid::Gid> &requested_gid, + const std::experimental::optional<int64_t> &cypher_id); + + /** + * Set the newly created edge on `to` vertex. + * Called after `InsertEdgeOnFrom` in `InsertEdge`. The given `edge_address` + * is from the created edge, returned by `InsertEdgeOnFrom`. + */ + virtual void InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to, + const storage::EdgeType &edge_type, + const storage::EdgeAddress &edge_address); + private: GraphDb &db_; tx::Transaction &transaction_; @@ -640,4 +661,5 @@ class GraphDbAccessor { const RecordAccessor<Vertex> &vertex_accessor, const Vertex *const vertex); }; + } // namespace database diff --git a/src/distributed/bfs_rpc_clients.cpp b/src/distributed/bfs_rpc_clients.cpp index 1b49c1c46..5f013f6da 100644 --- a/src/distributed/bfs_rpc_clients.cpp +++ b/src/distributed/bfs_rpc_clients.cpp @@ -6,7 +6,7 @@ namespace distributed { -BfsRpcClients::BfsRpcClients(database::GraphDb *db, +BfsRpcClients::BfsRpcClients(database::DistributedGraphDb *db, BfsSubcursorStorage *subcursor_storage, RpcWorkerClients *clients, DataManager *data_manager) diff --git a/src/distributed/bfs_rpc_clients.hpp b/src/distributed/bfs_rpc_clients.hpp index a86fbe78b..6baf71bd8 100644 --- a/src/distributed/bfs_rpc_clients.hpp +++ b/src/distributed/bfs_rpc_clients.hpp @@ -5,6 +5,10 @@ #include "distributed/rpc_worker_clients.hpp" #include "transactions/transaction.hpp" +namespace database { +class DistributedGraphDb; +} + namespace distributed { class DataManager; @@ -17,7 +21,7 @@ class DataManager; /// directly. class BfsRpcClients { public: - BfsRpcClients(database::GraphDb *db, + BfsRpcClients(database::DistributedGraphDb *db, BfsSubcursorStorage *subcursor_storage, RpcWorkerClients *clients, DataManager *data_manager); @@ -60,7 +64,7 @@ class BfsRpcClients { const std::unordered_map<int16_t, int64_t> &subcursor_ids, bool clear); private: - database::GraphDb *db_{nullptr}; + database::DistributedGraphDb *db_{nullptr}; distributed::BfsSubcursorStorage *subcursor_storage_{nullptr}; distributed::RpcWorkerClients *clients_{nullptr}; distributed::DataManager *data_manager_{nullptr}; diff --git a/src/distributed/bfs_rpc_messages.lcp b/src/distributed/bfs_rpc_messages.lcp index 2971772d3..e96ad1359 100644 --- a/src/distributed/bfs_rpc_messages.lcp +++ b/src/distributed/bfs_rpc_messages.lcp @@ -120,10 +120,9 @@ cpp<# << "Only global addresses should be used with SerializedGraphElement"; } - SerializedGraphElement(const RecordAccessor<TElement> &accessor) + SerializedGraphElement(const RecordAccessor<TElement> &accessor, int16_t worker_id) : SerializedGraphElement(accessor.GlobalAddress(), accessor.GetOld(), - accessor.GetNew(), - accessor.db_accessor().db().WorkerId()) {} + accessor.GetNew(), worker_id) {} SerializedGraphElement() {} cpp<#) @@ -265,12 +264,13 @@ cpp<# ReconstructPathRes( const std::vector<EdgeAccessor> &edge_accessors, std::experimental::optional<storage::VertexAddress> next_vertex, - std::experimental::optional<storage::EdgeAddress> next_edge) + std::experimental::optional<storage::EdgeAddress> next_edge, + int16_t worker_id) : next_vertex(std::move(next_vertex)), next_edge(std::move(next_edge)) { CHECK(!static_cast<bool>(next_vertex) || !static_cast<bool>(next_edge)) << "At most one of `next_vertex` and `next_edge` should be set"; for (const auto &edge : edge_accessors) { - edges.emplace_back(edge); + edges.emplace_back(edge, worker_id); } } cpp<#))) diff --git a/src/distributed/bfs_rpc_server.hpp b/src/distributed/bfs_rpc_server.hpp index 8ce85bdd1..597ce923d 100644 --- a/src/distributed/bfs_rpc_server.hpp +++ b/src/distributed/bfs_rpc_server.hpp @@ -16,7 +16,8 @@ namespace distributed { /// subcursor storage. class BfsRpcServer { public: - BfsRpcServer(database::GraphDb *db, communication::rpc::Server *server, + BfsRpcServer(database::DistributedGraphDb *db, + communication::rpc::Server *server, BfsSubcursorStorage *subcursor_storage) : db_(db), server_(server), subcursor_storage_(subcursor_storage) { server_->Register<CreateBfsSubcursorRpc>( @@ -38,14 +39,14 @@ class BfsRpcServer { res.Save(res_builder); }); - server_->Register<ResetSubcursorRpc>([this](const auto &req_reader, - auto *res_builder) { - ResetSubcursorReq req; - req.Load(req_reader); - subcursor_storage_->Get(req.subcursor_id)->Reset(); - ResetSubcursorRes res; - res.Save(res_builder); - }); + server_->Register<ResetSubcursorRpc>( + [this](const auto &req_reader, auto *res_builder) { + ResetSubcursorReq req; + req.Load(req_reader); + subcursor_storage_->Get(req.subcursor_id)->Reset(); + ResetSubcursorRes res; + res.Save(res_builder); + }); server_->Register<RemoveBfsSubcursorRpc>( [this](const auto &req_reader, auto *res_builder) { @@ -83,7 +84,7 @@ class BfsRpcServer { res.Save(res_builder); return; } - SubcursorPullRes res(*vertex); + SubcursorPullRes res(SerializedVertex(*vertex, db_->WorkerId())); res.Save(res_builder); }); @@ -111,7 +112,7 @@ class BfsRpcServer { LOG(FATAL) << "`edge` or `vertex` should be set in ReconstructPathReq"; } ReconstructPathRes res(result.edges, result.next_vertex, - result.next_edge); + result.next_edge, db_->WorkerId()); res.Save(res_builder); }); @@ -126,7 +127,7 @@ class BfsRpcServer { } private: - database::GraphDb *db_; + database::DistributedGraphDb *db_; communication::rpc::Server *server_; BfsSubcursorStorage *subcursor_storage_; diff --git a/src/distributed/bfs_subcursor.cpp b/src/distributed/bfs_subcursor.cpp index c376bd04a..ccb3fba98 100644 --- a/src/distributed/bfs_subcursor.cpp +++ b/src/distributed/bfs_subcursor.cpp @@ -18,7 +18,7 @@ ExpandBfsSubcursor::ExpandBfsSubcursor( std::vector<storage::EdgeType> edge_types, query::GraphView graph_view, BfsRpcClients *bfs_subcursor_clients) : bfs_subcursor_clients_(bfs_subcursor_clients), - dba_(*db, tx_id), + dba_(db->Access(tx_id)), direction_(direction), edge_types_(std::move(edge_types)), graph_view_(graph_view) { @@ -34,7 +34,7 @@ void ExpandBfsSubcursor::Reset() { void ExpandBfsSubcursor::SetSource(storage::VertexAddress source_address) { Reset(); - auto source = VertexAccessor(source_address, dba_); + auto source = VertexAccessor(source_address, *dba_); SwitchAccessor(source, graph_view_); processed_.emplace(source, std::experimental::nullopt); ExpandFromVertex(source); @@ -70,7 +70,7 @@ bool ExpandBfsSubcursor::ExpandToLocalVertex(storage::EdgeAddress edge, CHECK(vertex.address().is_local()) << "ExpandToLocalVertex called with remote vertex"; - edge = dba_.db().storage().LocalizedAddressIfPossible(edge); + edge = dba_->db().storage().LocalizedAddressIfPossible(edge); SwitchAccessor(vertex, graph_view_); std::lock_guard<std::mutex> lock(mutex_); @@ -83,13 +83,13 @@ bool ExpandBfsSubcursor::ExpandToLocalVertex(storage::EdgeAddress edge, bool ExpandBfsSubcursor::ExpandToLocalVertex(storage::EdgeAddress edge, storage::VertexAddress vertex) { - auto vertex_accessor = VertexAccessor(vertex, dba_); - return ExpandToLocalVertex(edge, VertexAccessor(vertex, dba_)); + auto vertex_accessor = VertexAccessor(vertex, *dba_); + return ExpandToLocalVertex(edge, VertexAccessor(vertex, *dba_)); } PathSegment ExpandBfsSubcursor::ReconstructPath( storage::EdgeAddress edge_address) { - EdgeAccessor edge(edge_address, dba_); + EdgeAccessor edge(edge_address, *dba_); CHECK(edge.address().is_local()) << "ReconstructPath called with remote edge"; DCHECK(edge.from_addr().is_local()) << "`from` vertex should always be local"; DCHECK(!edge.to_addr().is_local()) << "`to` vertex should be remote when " @@ -103,7 +103,7 @@ PathSegment ExpandBfsSubcursor::ReconstructPath( PathSegment ExpandBfsSubcursor::ReconstructPath( storage::VertexAddress vertex_addr) { - VertexAccessor vertex(vertex_addr, dba_); + VertexAccessor vertex(vertex_addr, *dba_); CHECK(vertex.address().is_local()) << "ReconstructPath called with remote vertex"; PathSegment result; @@ -126,7 +126,7 @@ void ExpandBfsSubcursor::ReconstructPathHelper(VertexAccessor vertex, break; } - result->edges.emplace_back(*in_edge_address, dba_); + result->edges.emplace_back(*in_edge_address, *dba_); auto &in_edge = result->edges.back(); auto next_vertex_address = @@ -139,7 +139,7 @@ void ExpandBfsSubcursor::ReconstructPathHelper(VertexAccessor vertex, break; } - vertex = VertexAccessor(next_vertex_address, dba_); + vertex = VertexAccessor(next_vertex_address, *dba_); in_edge_address = processed_[vertex]; } } diff --git a/src/distributed/bfs_subcursor.hpp b/src/distributed/bfs_subcursor.hpp index 08cbfcc1a..3cdd81214 100644 --- a/src/distributed/bfs_subcursor.hpp +++ b/src/distributed/bfs_subcursor.hpp @@ -94,7 +94,7 @@ class ExpandBfsSubcursor { BfsRpcClients *bfs_subcursor_clients_{nullptr}; - database::GraphDbAccessor dba_; + std::unique_ptr<database::GraphDbAccessor> dba_; /// IDs of subcursors on other workers, used when sending RPCs. std::unordered_map<int16_t, int64_t> subcursor_ids_; diff --git a/src/distributed/data_rpc_server.cpp b/src/distributed/data_rpc_server.cpp index 62c09ce4c..038e108ed 100644 --- a/src/distributed/data_rpc_server.cpp +++ b/src/distributed/data_rpc_server.cpp @@ -1,40 +1,42 @@ +#include "distributed/data_rpc_server.hpp" + #include <memory> -#include "data_rpc_server.hpp" +#include "database/distributed_graph_db.hpp" #include "database/graph_db_accessor.hpp" #include "distributed/data_rpc_messages.hpp" namespace distributed { -DataRpcServer::DataRpcServer(database::GraphDb &db, - communication::rpc::Server &server) +DataRpcServer::DataRpcServer(database::DistributedGraphDb *db, + communication::rpc::Server *server) : db_(db), rpc_server_(server) { - rpc_server_.Register<VertexRpc>( + rpc_server_->Register<VertexRpc>( [this](const auto &req_reader, auto *res_builder) { - database::GraphDbAccessor dba(db_, req_reader.getMember().getTxId()); - auto vertex = dba.FindVertex(req_reader.getMember().getGid(), false); + auto dba = db_->Access(req_reader.getMember().getTxId()); + auto vertex = dba->FindVertex(req_reader.getMember().getGid(), false); CHECK(vertex.GetOld()) << "Old record must exist when sending vertex by RPC"; - VertexRes response(vertex.GetOld(), db_.WorkerId()); + VertexRes response(vertex.GetOld(), db_->WorkerId()); response.Save(res_builder); }); - rpc_server_.Register<EdgeRpc>([this](const auto &req_reader, - auto *res_builder) { - database::GraphDbAccessor dba(db_, req_reader.getMember().getTxId()); - auto edge = dba.FindEdge(req_reader.getMember().getGid(), false); + rpc_server_->Register<EdgeRpc>([this](const auto &req_reader, + auto *res_builder) { + auto dba = db_->Access(req_reader.getMember().getTxId()); + auto edge = dba->FindEdge(req_reader.getMember().getGid(), false); CHECK(edge.GetOld()) << "Old record must exist when sending edge by RPC"; - EdgeRes response(edge.GetOld(), db_.WorkerId()); + EdgeRes response(edge.GetOld(), db_->WorkerId()); response.Save(res_builder); }); - rpc_server_.Register<VertexCountRpc>( + rpc_server_->Register<VertexCountRpc>( [this](const auto &req_reader, auto *res_builder) { VertexCountReq req; req.Load(req_reader); - database::GraphDbAccessor dba(db_, req.member); + auto dba = db_->Access(req.member); int64_t size = 0; - for (auto vertex : dba.Vertices(false)) ++size; + for (auto vertex : dba->Vertices(false)) ++size; VertexCountRes res(size); res.Save(res_builder); }); diff --git a/src/distributed/data_rpc_server.hpp b/src/distributed/data_rpc_server.hpp index 91612a5cc..9fb47a336 100644 --- a/src/distributed/data_rpc_server.hpp +++ b/src/distributed/data_rpc_server.hpp @@ -3,15 +3,21 @@ #include "communication/rpc/server.hpp" #include "database/graph_db.hpp" +namespace database { +class DistributedGraphDb; +} + namespace distributed { /// Serves this worker's data to others. class DataRpcServer { public: - DataRpcServer(database::GraphDb &db, communication::rpc::Server &server); + DataRpcServer(database::DistributedGraphDb *db, + communication::rpc::Server *server); private: - database::GraphDb &db_; - communication::rpc::Server &rpc_server_; + database::DistributedGraphDb *db_; + communication::rpc::Server *rpc_server_; }; + } // namespace distributed diff --git a/src/distributed/durability_rpc_worker.cpp b/src/distributed/durability_rpc_worker.cpp index 3b2d0a48a..a9b976b7f 100644 --- a/src/distributed/durability_rpc_worker.cpp +++ b/src/distributed/durability_rpc_worker.cpp @@ -11,8 +11,8 @@ DurabilityRpcWorker::DurabilityRpcWorker(database::Worker *db, : db_(db), rpc_server_(server) { rpc_server_->Register<MakeSnapshotRpc>( [this](const auto &req_reader, auto *res_builder) { - database::GraphDbAccessor dba(*this->db_, req_reader.getMember()); - MakeSnapshotRes res(this->db_->MakeSnapshot(dba)); + auto dba = db_->Access(req_reader.getMember()); + MakeSnapshotRes res(db_->MakeSnapshot(*dba)); res.Save(res_builder); }); diff --git a/src/distributed/index_rpc_server.cpp b/src/distributed/index_rpc_server.cpp index ea47b6373..9a782c480 100644 --- a/src/distributed/index_rpc_server.cpp +++ b/src/distributed/index_rpc_server.cpp @@ -1,5 +1,6 @@ #include "database/graph_db.hpp" #include "database/graph_db_accessor.hpp" +#include "distributed/index_rpc_messages.hpp" #include "distributed/index_rpc_server.hpp" namespace distributed { @@ -20,9 +21,9 @@ IndexRpcServer::IndexRpcServer(database::GraphDb &db, PopulateIndexReq req; req.Load(req_reader); database::LabelPropertyIndex::Key key{req.label, req.property}; - database::GraphDbAccessor dba(db_, req.tx_id); - dba.PopulateIndex(key); - dba.EnableIndex(key); + auto dba = db_.Access(req.tx_id); + dba->PopulateIndex(key); + dba->EnableIndex(key); }); } diff --git a/src/distributed/produce_rpc_server.cpp b/src/distributed/produce_rpc_server.cpp index 456e65881..642ed1d4f 100644 --- a/src/distributed/produce_rpc_server.cpp +++ b/src/distributed/produce_rpc_server.cpp @@ -1,4 +1,6 @@ #include "distributed/produce_rpc_server.hpp" + +#include "database/distributed_graph_db.hpp" #include "distributed/data_manager.hpp" #include "distributed/pull_produce_rpc_messages.hpp" #include "query/common.hpp" @@ -8,15 +10,15 @@ namespace distributed { ProduceRpcServer::OngoingProduce::OngoingProduce( - database::GraphDb &db, tx::TransactionId tx_id, + database::Worker *db, tx::TransactionId tx_id, std::shared_ptr<query::plan::LogicalOperator> op, query::SymbolTable symbol_table, Parameters parameters, int64_t timestamp, std::vector<query::Symbol> pull_symbols) - : dba_{db, tx_id}, - context_(dba_), + : dba_(db->Access(tx_id)), + context_(*dba_), pull_symbols_(std::move(pull_symbols)), frame_(symbol_table.max_position()), - cursor_(op->MakeCursor(dba_)) { + cursor_(op->MakeCursor(*dba_)) { context_.symbol_table_ = std::move(symbol_table); context_.parameters_ = std::move(parameters); context_.timestamp_ = timestamp; @@ -95,11 +97,11 @@ ProduceRpcServer::OngoingProduce::PullOneFromCursor() { return std::make_pair(std::move(results), cursor_state_); } -ProduceRpcServer::ProduceRpcServer( - database::GraphDb &db, tx::Engine &tx_engine, - communication::rpc::Server &server, - const PlanConsumer &plan_consumer, - DataManager *data_manager) +ProduceRpcServer::ProduceRpcServer(database::Worker *db, + tx::WorkerEngine *tx_engine, + communication::rpc::Server &server, + const PlanConsumer &plan_consumer, + DataManager *data_manager) : db_(db), produce_rpc_server_(server), plan_consumer_(plan_consumer), @@ -127,7 +129,7 @@ ProduceRpcServer::ProduceRpcServer( [this, data_manager](const auto &req_reader, auto *res_builder) { TransactionCommandAdvancedReq req; req.Load(req_reader); - tx_engine_.UpdateCommand(req.member); + tx_engine_->UpdateCommand(req.member); data_manager->ClearCacheForSingleTransaction(req.member); TransactionCommandAdvancedRes res; res.Save(res_builder); @@ -154,11 +156,8 @@ ProduceRpcServer::OngoingProduce &ProduceRpcServer::GetOngoingProduce( if (found != ongoing_produces_.end()) { return found->second; } - if (db_.type() == database::GraphDb::Type::DISTRIBUTED_WORKER) { - // On the worker cache the snapshot to have one RPC less. - dynamic_cast<tx::WorkerEngine &>(tx_engine_) - .RunningTransaction(req.tx_id, req.tx_snapshot); - } + // On the worker cache the snapshot to have one RPC less. + tx_engine_->RunningTransaction(req.tx_id, req.tx_snapshot); auto &plan_pack = plan_consumer_.PlanForId(req.plan_id); return ongoing_produces_ .emplace(std::piecewise_construct, std::forward_as_tuple(key_tuple), @@ -171,7 +170,7 @@ ProduceRpcServer::OngoingProduce &ProduceRpcServer::GetOngoingProduce( PullResData ProduceRpcServer::Pull(const PullReq &req) { auto &ongoing_produce = GetOngoingProduce(req); - PullResData result(db_.WorkerId(), req.send_old, req.send_new); + PullResData result(db_->WorkerId(), req.send_old, req.send_new); result.pull_state = PullState::CURSOR_IN_PROGRESS; if (req.accumulate) { diff --git a/src/distributed/produce_rpc_server.hpp b/src/distributed/produce_rpc_server.hpp index 60570cfeb..0927d889d 100644 --- a/src/distributed/produce_rpc_server.hpp +++ b/src/distributed/produce_rpc_server.hpp @@ -16,9 +16,16 @@ #include "query/parameters.hpp" #include "query/plan/operator.hpp" #include "query/typed_value.hpp" -#include "transactions/engine.hpp" #include "transactions/type.hpp" +namespace database { +class Worker; +} + +namespace tx { +class WorkerEngine; +} + namespace distributed { class DataManager; @@ -34,7 +41,7 @@ class ProduceRpcServer { /// MG (see query::plan::Synchronize). class OngoingProduce { public: - OngoingProduce(database::GraphDb &db, tx::TransactionId tx_id, + OngoingProduce(database::Worker *db, tx::TransactionId tx_id, std::shared_ptr<query::plan::LogicalOperator> op, query::SymbolTable symbol_table, Parameters parameters, int64_t timestamp, std::vector<query::Symbol> pull_symbols); @@ -51,7 +58,7 @@ class ProduceRpcServer { void Reset(); private: - database::GraphDbAccessor dba_; + std::unique_ptr<database::GraphDbAccessor> dba_; query::Context context_; std::vector<query::Symbol> pull_symbols_; query::Frame frame_; @@ -64,7 +71,7 @@ class ProduceRpcServer { }; public: - ProduceRpcServer(database::GraphDb &db, tx::Engine &tx_engine, + ProduceRpcServer(database::Worker *db, tx::WorkerEngine *tx_engine, communication::rpc::Server &server, const PlanConsumer &plan_consumer, DataManager *data_manager); @@ -81,10 +88,10 @@ class ProduceRpcServer { std::map<std::tuple<tx::TransactionId, tx::CommandId, int64_t>, OngoingProduce> ongoing_produces_; - database::GraphDb &db_; + database::Worker *db_; communication::rpc::Server &produce_rpc_server_; const distributed::PlanConsumer &plan_consumer_; - tx::Engine &tx_engine_; + tx::WorkerEngine *tx_engine_; /// Gets an ongoing produce for the given pull request. Creates a new one if /// there is none currently existing. diff --git a/src/distributed/pull_produce_rpc_messages.lcp b/src/distributed/pull_produce_rpc_messages.lcp index 2847a654d..a58bb6cd8 100644 --- a/src/distributed/pull_produce_rpc_messages.lcp +++ b/src/distributed/pull_produce_rpc_messages.lcp @@ -36,6 +36,9 @@ cpp<# (lcp:capnp-type-conversion "tx::TransactionId" "UInt64") #>cpp +// Forward declare for LoadGraphElement. +class DataManager; + /// The default number of results returned via RPC from remote execution to the /// master that requested it. constexpr int kDefaultBatchSize = 20; @@ -97,8 +100,8 @@ the relevant parts of the response, ready for use.")) query::TypedValue value; utils::LoadCapnpTypedValue( value_reader, &value, - [this, dba](const auto &reader, auto *value) { - this->LoadGraphElement(dba, reader, value); + [this, dba, data_manager](const auto &reader, auto *value) { + this->LoadGraphElement(dba, reader, value, data_manager); }); current_frame.emplace_back(value); } @@ -302,9 +305,10 @@ to the appropriate value. Not used on side that generates the response.") distributed::capnp::TypedValue::Builder *) const; void LoadGraphElement(database::GraphDbAccessor *, const distributed::capnp::TypedValue::Reader &, - query::TypedValue *); + query::TypedValue *, distributed::DataManager *); cpp<#) - (:serialize :capnp :load-args '((dba "database::GraphDbAccessor *")))) + (:serialize :capnp :load-args '((dba "database::GraphDbAccessor *") + (data-manager "distributed::DataManager *")))) (lcp:in-impl #>cpp @@ -369,14 +373,7 @@ to the appropriate value. Not used on side that generates the response.") void PullResData::LoadGraphElement( database::GraphDbAccessor *dba, const distributed::capnp::TypedValue::Reader &reader, - query::TypedValue *value) { - distributed::DataManager *data_manager = nullptr; - // TODO: Pass in a DistributedGraphDb or data_manager. - if (auto *distributed_db = - dynamic_cast<database::DistributedGraphDb *>(&dba->db())) { - data_manager = &distributed_db->data_manager(); - } - CHECK(data_manager); + query::TypedValue *value, distributed::DataManager *data_manager) { auto load_vertex = [dba, data_manager](const auto &vertex_reader) { storage::VertexAddress global_address(vertex_reader.getAddress()); auto old_record = @@ -537,7 +534,8 @@ cpp<#) } } ")) - (:serialize :capnp :base t :load-args '((dba "database::GraphDbAccessor *"))))) + (:serialize :capnp :base t :load-args '((dba "database::GraphDbAccessor *") + (data-manager "distributed::DataManager *"))))) ;; TODO make a separate RPC for the continuation of an existing pull, as an ;; optimization not to have to send the full PullReqData pack every time. diff --git a/src/distributed/pull_rpc_clients.cpp b/src/distributed/pull_rpc_clients.cpp index d03f82e72..24b7f7b72 100644 --- a/src/distributed/pull_rpc_clients.cpp +++ b/src/distributed/pull_rpc_clients.cpp @@ -1,6 +1,5 @@ #include <functional> -#include "distributed/data_manager.hpp" #include "distributed/pull_rpc_clients.hpp" #include "storage/edge.hpp" #include "storage/vertex.hpp" @@ -12,13 +11,14 @@ utils::Future<PullData> PullRpcClients::Pull( tx::CommandId command_id, const Parameters ¶ms, const std::vector<query::Symbol> &symbols, int64_t timestamp, bool accumulate, int batch_size) { - return clients_.ExecuteOnWorker< - PullData>(worker_id, [dba, plan_id, command_id, params, symbols, - timestamp, accumulate, batch_size]( - int worker_id, ClientPool &client_pool) { - auto load_pull_res = [dba](const auto &res_reader) { + return clients_->ExecuteOnWorker< + PullData>(worker_id, [data_manager = data_manager_, dba, plan_id, + command_id, params, symbols, timestamp, accumulate, + batch_size](int worker_id, + ClientPool &client_pool) { + auto load_pull_res = [data_manager, dba](const auto &res_reader) { PullRes res; - res.Load(res_reader, dba); + res.Load(res_reader, dba, data_manager); return res; }; auto result = client_pool.CallWithLoad<PullRpc>( @@ -32,7 +32,7 @@ utils::Future<PullData> PullRpcClients::Pull( utils::Future<void> PullRpcClients::ResetCursor(database::GraphDbAccessor *dba, int worker_id, int64_t plan_id, tx::CommandId command_id) { - return clients_.ExecuteOnWorker<void>( + return clients_->ExecuteOnWorker<void>( worker_id, [dba, plan_id, command_id](int worker_id, auto &client) { auto res = client.template Call<ResetCursorRpc>(dba->transaction_id(), plan_id, command_id); @@ -42,7 +42,7 @@ utils::Future<void> PullRpcClients::ResetCursor(database::GraphDbAccessor *dba, std::vector<utils::Future<void>> PullRpcClients::NotifyAllTransactionCommandAdvanced(tx::TransactionId tx_id) { - return clients_.ExecuteOnWorkers<void>( + return clients_->ExecuteOnWorkers<void>( 0, [tx_id](int worker_id, auto &client) { auto res = client.template Call<TransactionCommandAdvancedRpc>(tx_id); CHECK(res) << "TransactionCommandAdvanceRpc failed"; diff --git a/src/distributed/pull_rpc_clients.hpp b/src/distributed/pull_rpc_clients.hpp index 030bcecb7..e1c8a9a3f 100644 --- a/src/distributed/pull_rpc_clients.hpp +++ b/src/distributed/pull_rpc_clients.hpp @@ -12,6 +12,8 @@ namespace distributed { +class DataManager; + /// Provides means of calling for the execution of a plan on some remote worker, /// and getting the results of that execution. The results are returned in /// batches and are therefore accompanied with an enum indicator of the state of @@ -20,7 +22,8 @@ class PullRpcClients { using ClientPool = communication::rpc::ClientPool; public: - PullRpcClients(RpcWorkerClients &clients) : clients_(clients) {} + PullRpcClients(RpcWorkerClients *clients, DataManager *data_manager) + : clients_(clients), data_manager_(data_manager) {} /// Calls a remote pull asynchroniously. IMPORTANT: take care not to call this /// function for the same (tx_id, worker_id, plan_id, command_id) before the @@ -39,13 +42,14 @@ class PullRpcClients { utils::Future<void> ResetCursor(database::GraphDbAccessor *dba, int worker_id, int64_t plan_id, tx::CommandId command_id); - auto GetWorkerIds() { return clients_.GetWorkerIds(); } + auto GetWorkerIds() { return clients_->GetWorkerIds(); } std::vector<utils::Future<void>> NotifyAllTransactionCommandAdvanced( tx::TransactionId tx_id); private: - RpcWorkerClients &clients_; + RpcWorkerClients *clients_{nullptr}; + DataManager *data_manager_{nullptr}; }; } // namespace distributed diff --git a/src/distributed/updates_rpc_server.cpp b/src/distributed/updates_rpc_server.cpp index 106d2d8f5..d3f967029 100644 --- a/src/distributed/updates_rpc_server.cpp +++ b/src/distributed/updates_rpc_server.cpp @@ -1,8 +1,9 @@ +#include "distributed/updates_rpc_server.hpp" + #include <utility> -#include "glog/logging.h" +#include <glog/logging.h> -#include "distributed/updates_rpc_server.hpp" #include "utils/thread/sync.hpp" namespace distributed { @@ -63,7 +64,7 @@ gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateVertex( const std::vector<storage::Label> &labels, const std::unordered_map<storage::Property, query::TypedValue> &properties) { - auto result = db_accessor_.InsertVertex(); + auto result = db_accessor_->InsertVertex(); for (auto &label : labels) result.add_label(label); for (auto &kv : properties) result.PropsSet(kv.first, kv.second); std::lock_guard<utils::SpinLock> guard{lock_}; @@ -74,12 +75,13 @@ gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateVertex( template <typename TRecordAccessor> gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateEdge( - gid::Gid from, storage::VertexAddress to, storage::EdgeType edge_type) { - auto &db = db_accessor_.db(); + gid::Gid from, storage::VertexAddress to, storage::EdgeType edge_type, + int worker_id) { + auto &db = db_accessor_->db(); auto from_addr = db.storage().LocalizedAddressIfPossible( - storage::VertexAddress(from, db.WorkerId())); + storage::VertexAddress(from, worker_id)); auto to_addr = db.storage().LocalizedAddressIfPossible(to); - auto edge = db_accessor_.InsertOnlyEdge(from_addr, to_addr, edge_type); + auto edge = db_accessor_->InsertOnlyEdge(from_addr, to_addr, edge_type); std::lock_guard<utils::SpinLock> guard{lock_}; deltas_.emplace(edge.gid(), std::make_pair(edge, std::vector<database::StateDelta>{})); @@ -96,7 +98,7 @@ UpdateResult UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::Apply() { record_accessor.Reconstruct(); for (database::StateDelta &delta : kv.second.second) { try { - auto &dba = db_accessor_; + auto &dba = *db_accessor_; switch (delta.type) { case database::StateDelta::Type::TRANSACTION_BEGIN: case database::StateDelta::Type::TRANSACTION_COMMIT: @@ -147,7 +149,7 @@ UpdateResult UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::Apply() { // We only remove the edge as a result of this StateDelta, // because the removal of edge from vertex in/out is performed // in REMOVE_[IN/OUT]_EDGE deltas. - db_accessor_.RemoveEdge( + db_accessor_->RemoveEdge( reinterpret_cast<EdgeAccessor &>(record_accessor), false, false); break; @@ -172,10 +174,11 @@ UpdateResult UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::Apply() { return UpdateResult::DONE; } -UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, - communication::rpc::Server &server) +UpdatesRpcServer::UpdatesRpcServer(database::DistributedGraphDb *db, + communication::rpc::Server *server) : db_(db) { - server.Register<UpdateRpc>([this](const auto &req_reader, auto *res_builder) { + server->Register<UpdateRpc>([this](const auto &req_reader, + auto *res_builder) { UpdateReq req; req.Load(req_reader); using DeltaType = database::StateDelta::Type; @@ -203,7 +206,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, } }); - server.Register<UpdateApplyRpc>( + server->Register<UpdateApplyRpc>( [this](const auto &req_reader, auto *res_builder) { UpdateApplyReq req; req.Load(req_reader); @@ -211,8 +214,8 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, res.Save(res_builder); }); - server.Register<CreateVertexRpc>([this](const auto &req_reader, - auto *res_builder) { + server->Register<CreateVertexRpc>([this](const auto &req_reader, + auto *res_builder) { CreateVertexReq req; req.Load(req_reader); gid::Gid gid = GetUpdates(vertex_updates_, req.member.tx_id) @@ -221,7 +224,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, res.Save(res_builder); }); - server.Register<CreateEdgeRpc>( + server->Register<CreateEdgeRpc>( [this](const auto &req_reader, auto *res_builder) { CreateEdgeReq req; req.Load(req_reader); @@ -231,10 +234,10 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, // If `from` and `to` are both on this worker, we handle it in this // RPC call. Do it only if CreateEdge succeeded. if (creation_result.result == UpdateResult::DONE && - data.to.worker_id() == db_.WorkerId()) { + data.to.worker_id() == db_->WorkerId()) { auto to_delta = database::StateDelta::AddInEdge( - data.tx_id, data.to.gid(), {data.from, db_.WorkerId()}, - {creation_result.gid, db_.WorkerId()}, data.edge_type); + data.tx_id, data.to.gid(), {data.from, db_->WorkerId()}, + {creation_result.gid, db_->WorkerId()}, data.edge_type); creation_result.result = GetUpdates(vertex_updates_, data.tx_id).Emplace(to_delta); } @@ -243,7 +246,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, res.Save(res_builder); }); - server.Register<AddInEdgeRpc>( + server->Register<AddInEdgeRpc>( [this](const auto &req_reader, auto *res_builder) { AddInEdgeReq req; req.Load(req_reader); @@ -256,7 +259,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, res.Save(res_builder); }); - server.Register<RemoveVertexRpc>( + server->Register<RemoveVertexRpc>( [this](const auto &req_reader, auto *res_builder) { RemoveVertexReq req; req.Load(req_reader); @@ -268,7 +271,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, res.Save(res_builder); }); - server.Register<RemoveEdgeRpc>( + server->Register<RemoveEdgeRpc>( [this](const auto &req_reader, auto *res_builder) { RemoveEdgeReq req; req.Load(req_reader); @@ -276,8 +279,8 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db, res.Save(res_builder); }); - server.Register<RemoveInEdgeRpc>([this](const auto &req_reader, - auto *res_builder) { + server->Register<RemoveInEdgeRpc>([this](const auto &req_reader, + auto *res_builder) { RemoveInEdgeReq req; req.Load(req_reader); auto data = req.member; @@ -335,10 +338,10 @@ UpdatesRpcServer::TransactionUpdates<TAccessor> &UpdatesRpcServer::GetUpdates( CreateResult UpdatesRpcServer::CreateEdge(const CreateEdgeReqData &req) { auto gid = GetUpdates(edge_updates_, req.tx_id) - .CreateEdge(req.from, req.to, req.edge_type); + .CreateEdge(req.from, req.to, req.edge_type, db_->WorkerId()); auto from_delta = database::StateDelta::AddOutEdge( - req.tx_id, req.from, req.to, {gid, db_.WorkerId()}, req.edge_type); + req.tx_id, req.from, req.to, {gid, db_->WorkerId()}, req.edge_type); auto result = GetUpdates(vertex_updates_, req.tx_id).Emplace(from_delta); return {result, gid}; @@ -353,16 +356,16 @@ UpdateResult UpdatesRpcServer::RemoveEdge(const RemoveEdgeData &data) { // Out-edge removal, for sure is local. if (result == UpdateResult::DONE) { auto remove_out_delta = database::StateDelta::RemoveOutEdge( - data.tx_id, data.vertex_from_id, {data.edge_id, db_.WorkerId()}); + data.tx_id, data.vertex_from_id, {data.edge_id, db_->WorkerId()}); result = GetUpdates(vertex_updates_, data.tx_id).Emplace(remove_out_delta); } // In-edge removal, might not be local. if (result == UpdateResult::DONE && - data.vertex_to_address.worker_id() == db_.WorkerId()) { + data.vertex_to_address.worker_id() == db_->WorkerId()) { auto remove_in_delta = database::StateDelta::RemoveInEdge( data.tx_id, data.vertex_to_address.gid(), - {data.edge_id, db_.WorkerId()}); + {data.edge_id, db_->WorkerId()}); result = GetUpdates(vertex_updates_, data.tx_id).Emplace(remove_in_delta); } @@ -373,13 +376,13 @@ template <> VertexAccessor UpdatesRpcServer::TransactionUpdates<VertexAccessor>::FindAccessor( gid::Gid gid) { - return db_accessor_.FindVertex(gid, false); + return db_accessor_->FindVertex(gid, false); } template <> EdgeAccessor UpdatesRpcServer::TransactionUpdates<EdgeAccessor>::FindAccessor( gid::Gid gid) { - return db_accessor_.FindEdge(gid, false); + return db_accessor_->FindEdge(gid, false); } } // namespace distributed diff --git a/src/distributed/updates_rpc_server.hpp b/src/distributed/updates_rpc_server.hpp index dc81d9eb6..9052597a4 100644 --- a/src/distributed/updates_rpc_server.hpp +++ b/src/distributed/updates_rpc_server.hpp @@ -7,7 +7,7 @@ #include "communication/rpc/server.hpp" #include "data_structures/concurrent/concurrent_map.hpp" -#include "database/graph_db.hpp" +#include "database/distributed_graph_db.hpp" #include "database/graph_db_accessor.hpp" #include "database/state_delta.hpp" #include "distributed/updates_rpc_messages.hpp" @@ -32,8 +32,9 @@ class UpdatesRpcServer { template <typename TRecordAccessor> class TransactionUpdates { public: - TransactionUpdates(database::GraphDb &db, tx::TransactionId tx_id) - : db_accessor_(db, tx_id) {} + TransactionUpdates(database::DistributedGraphDb *db, + tx::TransactionId tx_id) + : db_accessor_(db->Access(tx_id)) {} /// Adds a delta and returns the result. Does not modify the state (data) of /// the graph element the update is for, but calls the `update` method to @@ -49,15 +50,15 @@ class UpdatesRpcServer { /// Creates a new edge and returns it's gid. Does not update vertices at the /// end of the edge. gid::Gid CreateEdge(gid::Gid from, storage::VertexAddress to, - storage::EdgeType edge_type); + storage::EdgeType edge_type, int worker_id); /// Applies all the deltas on the record. UpdateResult Apply(); - auto &db_accessor() { return db_accessor_; } + auto &db_accessor() { return *db_accessor_; } private: - database::GraphDbAccessor db_accessor_; + std::unique_ptr<database::GraphDbAccessor> db_accessor_; std::unordered_map< gid::Gid, std::pair<TRecordAccessor, std::vector<database::StateDelta>>> deltas_; @@ -69,7 +70,8 @@ class UpdatesRpcServer { }; public: - UpdatesRpcServer(database::GraphDb &db, communication::rpc::Server &server); + UpdatesRpcServer(database::DistributedGraphDb *db, + communication::rpc::Server *server); /// Applies all existsing updates for the given transaction ID. If there are /// no updates for that transaction, nothing happens. Clears the updates cache @@ -81,11 +83,10 @@ class UpdatesRpcServer { void ClearTransactionalCache(tx::TransactionId oldest_active); private: - database::GraphDb &db_; + database::DistributedGraphDb *db_; template <typename TAccessor> - using MapT = - ConcurrentMap<tx::TransactionId, TransactionUpdates<TAccessor>>; + using MapT = ConcurrentMap<tx::TransactionId, TransactionUpdates<TAccessor>>; MapT<VertexAccessor> vertex_updates_; MapT<EdgeAccessor> edge_updates_; diff --git a/src/durability/recovery.cpp b/src/durability/recovery.cpp index 8c99abaf0..11e4b4766 100644 --- a/src/durability/recovery.cpp +++ b/src/durability/recovery.cpp @@ -4,6 +4,7 @@ #include <limits> #include <unordered_map> +#include "database/distributed_graph_db.hpp" #include "database/graph_db_accessor.hpp" #include "database/indexes/label_property_index.hpp" #include "durability/hashed_file_reader.hpp" @@ -66,9 +67,18 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db, RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int) && dv.ValueInt() == durability::kVersion); + int worker_id = 0; + // TODO: Figure out a better solution for SingleNode recovery vs + // DistributedGraphDb. + if (auto *distributed_db = + dynamic_cast<database::DistributedGraphDb *>(db)) { + worker_id = distributed_db->WorkerId(); + } else { + CHECK(dynamic_cast<database::SingleNode *>(db)); + } // Checks worker id was set correctly RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int) && - dv.ValueInt() == db->WorkerId()); + dv.ValueInt() == worker_id); // Vertex and edge generator ids RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int)); @@ -101,7 +111,7 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db, property.ValueString()); } - database::GraphDbAccessor dba(*db); + auto dba = db->Access(); std::unordered_map<gid::Gid, std::pair<storage::VertexAddress, storage::VertexAddress>> edge_gid_endpoints_mapping; @@ -110,47 +120,47 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db, auto vertex = decoder.ReadSnapshotVertex(); RETURN_IF_NOT(vertex); - auto vertex_accessor = dba.InsertVertex(vertex->gid, vertex->cypher_id); + auto vertex_accessor = dba->InsertVertex(vertex->gid, vertex->cypher_id); for (const auto &label : vertex->labels) { - vertex_accessor.add_label(dba.Label(label)); + vertex_accessor.add_label(dba->Label(label)); } for (const auto &property_pair : vertex->properties) { - vertex_accessor.PropsSet(dba.Property(property_pair.first), + vertex_accessor.PropsSet(dba->Property(property_pair.first), glue::ToTypedValue(property_pair.second)); } auto vertex_record = vertex_accessor.GetNew(); for (const auto &edge : vertex->in) { vertex_record->in_.emplace(edge.vertex, edge.address, - dba.EdgeType(edge.type)); + dba->EdgeType(edge.type)); edge_gid_endpoints_mapping[edge.address.gid()] = { edge.vertex, vertex_accessor.GlobalAddress()}; } for (const auto &edge : vertex->out) { vertex_record->out_.emplace(edge.vertex, edge.address, - dba.EdgeType(edge.type)); + dba->EdgeType(edge.type)); edge_gid_endpoints_mapping[edge.address.gid()] = { vertex_accessor.GlobalAddress(), edge.vertex}; } } auto vertex_transform_to_local_if_possible = - [&db, &dba](storage::VertexAddress &address) { + [&dba, worker_id](storage::VertexAddress &address) { if (address.is_local()) return; // If the worker id matches it should be a local apperance - if (address.worker_id() == db->WorkerId()) { + if (address.worker_id() == worker_id) { address = storage::VertexAddress( - dba.db().storage().LocalAddress<Vertex>(address.gid())); + dba->db().storage().LocalAddress<Vertex>(address.gid())); CHECK(address.is_local()) << "Address should be local but isn't"; } }; auto edge_transform_to_local_if_possible = - [&db, &dba](storage::EdgeAddress &address) { + [&dba, worker_id](storage::EdgeAddress &address) { if (address.is_local()) return; // If the worker id matches it should be a local apperance - if (address.worker_id() == db->WorkerId()) { + if (address.worker_id() == worker_id) { address = storage::EdgeAddress( - dba.db().storage().LocalAddress<Edge>(address.gid())); + dba->db().storage().LocalAddress<Edge>(address.gid())); CHECK(address.is_local()) << "Address should be local but isn't"; } }; @@ -181,11 +191,11 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db, vertex_transform_to_local_if_possible(from); vertex_transform_to_local_if_possible(to); - auto edge_accessor = dba.InsertOnlyEdge(from, to, dba.EdgeType(edge.type), - edge.id.AsUint(), cypher_id); + auto edge_accessor = dba->InsertOnlyEdge(from, to, dba->EdgeType(edge.type), + edge.id.AsUint(), cypher_id); for (const auto &property_pair : edge.properties) - edge_accessor.PropsSet(dba.Property(property_pair.first), + edge_accessor.PropsSet(dba->Property(property_pair.first), glue::ToTypedValue(property_pair.second)); } @@ -194,14 +204,14 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db, reader.ReadType(vertex_count); reader.ReadType(edge_count); if (!reader.Close() || reader.hash() != hash) { - dba.Abort(); + dba->Abort(); return false; } // We have to replace global_ids with local ids where possible for all edges // in every vertex and this can only be done after we inserted the edges; this // is to speedup execution - for (auto &vertex_accessor : dba.Vertices(true)) { + for (auto &vertex_accessor : dba->Vertices(true)) { auto vertex = vertex_accessor.GetNew(); auto iterate_and_transform = [vertex_transform_to_local_if_possible, @@ -231,8 +241,8 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db, tx::TransactionId max_id = recovery_data->snapshooter_tx_id; auto &snap = recovery_data->snapshooter_tx_snapshot; if (!snap.empty()) max_id = *std::max_element(snap.begin(), snap.end()); - dba.db().tx_engine().EnsureNextIdGreater(max_id); - dba.Commit(); + dba->db().tx_engine().EnsureNextIdGreater(max_id); + dba->Commit(); return true; } @@ -319,7 +329,9 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db, !utils::Contains(common_wal_tx, tx_id); }; - std::unordered_map<tx::TransactionId, database::GraphDbAccessor> accessors; + std::unordered_map<tx::TransactionId, + std::unique_ptr<database::GraphDbAccessor>> + accessors; auto get_accessor = [db, &accessors](tx::TransactionId tx_id) -> database::GraphDbAccessor & { auto found = accessors.find(tx_id); @@ -328,12 +340,13 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db, // don't have a transaction begin, the accessors are not created. if (db->type() == database::GraphDb::Type::DISTRIBUTED_WORKER && found == accessors.end()) { - std::tie(found, std::ignore) = accessors.emplace(tx_id, *db); + // TODO: Do we want to call db->Access with tx_id? + std::tie(found, std::ignore) = accessors.emplace(tx_id, db->Access()); } CHECK(found != accessors.end()) << "Accessor does not exist for transaction: " << tx_id; - return found->second; + return *found->second; }; // Ensure that the next transaction ID in the recovered DB will be greater @@ -353,7 +366,7 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db, case database::StateDelta::Type::TRANSACTION_BEGIN: CHECK(accessors.find(delta.transaction_id) == accessors.end()) << "Double transaction start"; - accessors.emplace(delta.transaction_id, *db); + accessors.emplace(delta.transaction_id, db->Access()); break; case database::StateDelta::Type::TRANSACTION_ABORT: get_accessor(delta.transaction_id).Abort(); @@ -381,6 +394,7 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db, db->tx_engine().EnsureNextIdGreater(max_observed_tx_id); return true; } + } // anonymous namespace RecoveryInfo RecoverOnlySnapshot( @@ -439,15 +453,16 @@ void RecoverWalAndIndexes(const fs::path &durability_dir, database::GraphDb *db, RecoverWal(durability_dir / kWalDir, db, recovery_data); // Index recovery. - database::GraphDbAccessor db_accessor_indices{*db}; + auto db_accessor_indices = db->Access(); for (const auto &label_prop : recovery_data->indexes) { const database::LabelPropertyIndex::Key key{ - db_accessor_indices.Label(label_prop.first), - db_accessor_indices.Property(label_prop.second)}; - db_accessor_indices.db().storage().label_property_index().CreateIndex(key); - db_accessor_indices.PopulateIndex(key); - db_accessor_indices.EnableIndex(key); + db_accessor_indices->Label(label_prop.first), + db_accessor_indices->Property(label_prop.second)}; + db_accessor_indices->db().storage().label_property_index().CreateIndex(key); + db_accessor_indices->PopulateIndex(key); + db_accessor_indices->EnableIndex(key); } - db_accessor_indices.Commit(); + db_accessor_indices->Commit(); } + } // namespace durability diff --git a/src/durability/snapshooter.cpp b/src/durability/snapshooter.cpp index 6c4402424..31deeae66 100644 --- a/src/durability/snapshooter.cpp +++ b/src/durability/snapshooter.cpp @@ -1,9 +1,10 @@ +#include "durability/snapshooter.hpp" + #include <algorithm> #include <glog/logging.h> -#include "durability/snapshooter.hpp" - +#include "database/distributed_graph_db.hpp" #include "database/graph_db_accessor.hpp" #include "durability/hashed_file_writer.hpp" #include "durability/paths.hpp" @@ -31,9 +32,18 @@ bool Encode(const fs::path &snapshot_file, database::GraphDb &db, durability::kMagicNumber.size()); encoder.WriteInt(durability::kVersion); + int worker_id = 0; + // TODO: Figure out a better solution for SingleNode recovery vs + // DistributedGraphDb. + if (auto *distributed_db = + dynamic_cast<database::DistributedGraphDb *>(&dba.db())) { + worker_id = distributed_db->WorkerId(); + } else { + CHECK(dynamic_cast<database::SingleNode *>(&dba.db())); + } // Writes the worker id to snapshot, used to guarantee consistent cluster // state after recovery - encoder.WriteInt(db.WorkerId()); + encoder.WriteInt(worker_id); // Write the number of generated vertex and edges, used to recover // generators internal states @@ -125,8 +135,17 @@ bool MakeSnapshot(database::GraphDb &db, database::GraphDbAccessor &dba, const fs::path &durability_dir, const int snapshot_max_retained) { if (!utils::EnsureDir(durability_dir / kSnapshotDir)) return false; + int worker_id = 0; + // TODO: Figure out a better solution for SingleNode recovery vs + // DistributedGraphDb. + if (auto *distributed_db = + dynamic_cast<database::DistributedGraphDb *>(&db)) { + worker_id = distributed_db->WorkerId(); + } else { + CHECK(dynamic_cast<database::SingleNode *>(&db)); + } const auto snapshot_file = - MakeSnapshotPath(durability_dir, db.WorkerId(), dba.transaction_id()); + MakeSnapshotPath(durability_dir, worker_id, dba.transaction_id()); if (fs::exists(snapshot_file)) return false; if (Encode(snapshot_file, db, dba)) { RemoveOldSnapshots(durability_dir / kSnapshotDir, snapshot_max_retained); diff --git a/src/memgraph_bolt.cpp b/src/memgraph_bolt.cpp index 667d8b85b..dce8ef68b 100644 --- a/src/memgraph_bolt.cpp +++ b/src/memgraph_bolt.cpp @@ -268,14 +268,14 @@ void SingleNodeMain() { auto stream_writer = [&session_data](const std::vector<std::string> &queries) { for (auto &query : queries) { - database::GraphDbAccessor dba(session_data.db); + auto dba = session_data.db.Access(); KafkaResultStream stream; try { - session_data.interpreter(query, dba, {}, false).PullAll(stream); - dba.Commit(); + session_data.interpreter(query, *dba, {}, false).PullAll(stream); + dba->Commit(); } catch (const query::QueryException &e) { LOG(ERROR) << e.what(); - dba.Abort(); + dba->Abort(); } } }; @@ -314,8 +314,8 @@ void SingleNodeMain() { "telemetry", std::chrono::minutes(10)); telemetry->AddCollector("db", [&db]() -> nlohmann::json { - database::GraphDbAccessor dba(db); - return {{"vertices", dba.VerticesCount()}, {"edges", dba.EdgesCount()}}; + auto dba = db.Access(); + return {{"vertices", dba->VerticesCount()}, {"edges", dba->EdgesCount()}}; }); } @@ -358,14 +358,14 @@ void MasterMain() { auto stream_writer = [&session_data](const std::vector<std::string> &queries) { for (auto &query : queries) { - database::GraphDbAccessor dba(session_data.db); + auto dba = session_data.db.Access(); KafkaResultStream stream; try { - session_data.interpreter(query, dba, {}, false).PullAll(stream); - dba.Commit(); + session_data.interpreter(query, *dba, {}, false).PullAll(stream); + dba->Commit(); } catch (const query::QueryException &e) { LOG(ERROR) << e.what(); - dba.Abort(); + dba->Abort(); } } }; diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp index 591dfca24..82e928b54 100644 --- a/src/query/plan/operator.cpp +++ b/src/query/plan/operator.cpp @@ -126,7 +126,7 @@ CreateNode::CreateNode(const std::shared_ptr<LogicalOperator> &input, namespace { // Returns a random worker id. Worker ID is obtained from the Db. -int RandomWorkerId(database::GraphDb &db) { +int RandomWorkerId(const database::DistributedGraphDb &db) { thread_local std::mt19937 gen_{std::random_device{}()}; thread_local std::uniform_int_distribution<int> rand_; @@ -156,7 +156,16 @@ VertexAccessor &CreateVertexOnWorker(int worker_id, NodeAtom *node_atom, Frame &frame, Context &context) { auto &dba = context.db_accessor_; - if (worker_id == dba.db().WorkerId()) + int current_worker_id = 0; + // TODO: Figure out a better solution. + if (auto *distributed_db = + dynamic_cast<database::DistributedGraphDb *>(&dba.db())) { + current_worker_id = distributed_db->WorkerId(); + } else { + CHECK(dynamic_cast<database::SingleNode *>(&dba.db())); + } + + if (worker_id == current_worker_id) return CreateLocalVertex(node_atom, frame, context); std::unordered_map<storage::Property, query::TypedValue> properties; @@ -173,11 +182,12 @@ VertexAccessor &CreateVertexOnWorker(int worker_id, NodeAtom *node_atom, properties.emplace(kv.first.second, std::move(value)); } - auto new_node = - dba.InsertVertexIntoRemote(worker_id, node_atom->labels_, properties); + auto new_node = database::InsertVertexIntoRemote( + &dba, worker_id, node_atom->labels_, properties); frame[context.symbol_table_.at(*node_atom->identifier_)] = new_node; return frame[context.symbol_table_.at(*node_atom->identifier_)].ValueVertex(); } + } // namespace ACCEPT_WITH_INPUT(CreateNode) @@ -201,8 +211,12 @@ CreateNode::CreateNodeCursor::CreateNodeCursor(const CreateNode &self, bool CreateNode::CreateNodeCursor::Pull(Frame &frame, Context &context) { if (input_cursor_->Pull(frame, context)) { if (self_.on_random_worker_) { - CreateVertexOnWorker(RandomWorkerId(db_.db()), self_.node_atom_, frame, - context); + // TODO: Replace this with some other mechanism + auto *distributed_db = + dynamic_cast<database::DistributedGraphDb *>(&db_.db()); + CHECK(distributed_db); + CreateVertexOnWorker(RandomWorkerId(*distributed_db), self_.node_atom_, + frame, context); } else { CreateLocalVertex(self_.node_atom_, frame, context); } @@ -3475,7 +3489,10 @@ class SynchronizeCursor : public Cursor { input_cursor_(self.input()->MakeCursor(db)), pull_remote_cursor_( self.pull_remote() ? self.pull_remote()->MakeCursor(db) : nullptr), - command_id_(db.transaction().cid()) {} + command_id_(db.transaction().cid()), + master_id_( + // TODO: Pass in a Master GraphDb. + dynamic_cast<database::Master *>(&db.db())->WorkerId()) {} bool Pull(Frame &frame, Context &context) override { if (!initial_pull_done_) { @@ -3526,17 +3543,17 @@ class SynchronizeCursor : public Cursor { bool initial_pull_done_{false}; std::vector<std::vector<TypedValue>> local_frames_; tx::CommandId command_id_; + int master_id_; void InitialPull(Frame &frame, Context &context) { VLOG(10) << "[SynchronizeCursor] [" << context.db_accessor_.transaction_id() << "] initial pull"; - auto &db = context.db_accessor_.db(); // Tell all workers to accumulate, only if there is a remote pull. std::vector<utils::Future<distributed::PullData>> worker_accumulations; if (pull_remote_cursor_) { for (auto worker_id : pull_clients_->GetWorkerIds()) { - if (worker_id == db.WorkerId()) continue; + if (worker_id == master_id_) continue; worker_accumulations.emplace_back(pull_clients_->Pull( &context.db_accessor_, worker_id, self_.pull_remote()->plan_id(), command_id_, context.parameters_, self_.pull_remote()->symbols(), @@ -3595,7 +3612,7 @@ class SynchronizeCursor : public Cursor { // Make all the workers apply their deltas. auto tx_id = context.db_accessor_.transaction_id(); - auto apply_futures = updates_clients_->UpdateApplyAll(db.WorkerId(), tx_id); + auto apply_futures = updates_clients_->UpdateApplyAll(master_id_, tx_id); updates_server_->Apply(tx_id); for (auto &future : apply_futures) { switch (future.get()) { diff --git a/src/query/repl.cpp b/src/query/repl.cpp index c43f115e0..9ad528abc 100644 --- a/src/query/repl.cpp +++ b/src/query/repl.cpp @@ -63,14 +63,14 @@ void query::Repl(database::GraphDb &db) { // regular cypher queries try { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); ResultStreamFaker<query::TypedValue> stream; - auto results = interpeter(command, dba, {}, false); + auto results = interpeter(command, *dba, {}, false); stream.Header(results.header()); results.PullAll(stream); stream.Summary(results.summary()); std::cout << stream; - dba.Commit(); + dba->Commit(); } catch (const query::SyntaxException &e) { std::cout << "SYNTAX EXCEPTION: " << e.what() << std::endl; } catch (const query::LexingException &e) { diff --git a/src/query/transaction_engine.hpp b/src/query/transaction_engine.hpp index d8d0a3a7b..fde1bc1b4 100644 --- a/src/query/transaction_engine.hpp +++ b/src/query/transaction_engine.hpp @@ -60,7 +60,7 @@ class TransactionEngine final { // Create a DB accessor if we don't yet have one. if (!db_accessor_) - db_accessor_ = std::make_unique<database::GraphDbAccessor>(db_); + db_accessor_ = db_.Access(); // Interpret the query and return the headers. try { diff --git a/src/storage/dynamic_graph_partitioner/dgp.cpp b/src/storage/dynamic_graph_partitioner/dgp.cpp index 637640a07..479080215 100644 --- a/src/storage/dynamic_graph_partitioner/dgp.cpp +++ b/src/storage/dynamic_graph_partitioner/dgp.cpp @@ -29,20 +29,20 @@ DynamicGraphPartitioner::DynamicGraphPartitioner( : db_(db) {} void DynamicGraphPartitioner::Run() { - database::GraphDbAccessor dba(*db_); + auto dba = db_->Access(); VLOG(21) << "Starting DynamicGraphPartitioner in tx: " - << dba.transaction().id_; + << dba->transaction().id_; - auto migrations = FindMigrations(dba); + auto migrations = FindMigrations(*dba); try { - VertexMigrator migrator(&dba); + VertexMigrator migrator(dba.get()); for (auto &migration : migrations) { migrator.MigrateVertex(migration.first, migration.second); } auto apply_futures = db_->updates_clients().UpdateApplyAll( - db_->WorkerId(), dba.transaction().id_); + db_->WorkerId(), dba->transaction().id_); for (auto &future : apply_futures) { switch (future.get()) { @@ -62,11 +62,11 @@ void DynamicGraphPartitioner::Run() { } } - dba.Commit(); + dba->Commit(); VLOG(21) << "Sucesfully migrated " << migrations.size() << " vertices.."; } catch (const utils::BasicException &e) { VLOG(21) << "Didn't succeed in relocating; " << e.what(); - dba.Abort(); + dba->Abort(); } } diff --git a/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp b/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp index 7b4ec59d4..0fa958aca 100644 --- a/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp +++ b/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp @@ -1,5 +1,6 @@ #include "storage/dynamic_graph_partitioner/vertex_migrator.hpp" +#include "database/distributed_graph_db.hpp" #include "database/graph_db_accessor.hpp" #include "query/typed_value.hpp" @@ -20,8 +21,8 @@ void VertexMigrator::MigrateVertex(VertexAccessor &vertex, int destination) { } }; - auto relocated_vertex = dba_->InsertVertexIntoRemote( - destination, vertex.labels(), get_props(vertex)); + auto relocated_vertex = database::InsertVertexIntoRemote( + dba_, destination, vertex.labels(), get_props(vertex)); vertex_migrated_to_[vertex.gid()] = relocated_vertex.address(); diff --git a/src/storage/record_accessor.cpp b/src/storage/record_accessor.cpp index 4245b81d6..e7f2b2c70 100644 --- a/src/storage/record_accessor.cpp +++ b/src/storage/record_accessor.cpp @@ -109,9 +109,17 @@ typename RecordAccessor<TRecord>::AddressT RecordAccessor<TRecord>::address() template <typename TRecord> typename RecordAccessor<TRecord>::AddressT RecordAccessor<TRecord>::GlobalAddress() const { - return is_local() ? storage::Address<mvcc::VersionList<TRecord>>( - gid(), db_accessor_->db_.WorkerId()) - : address_; + // TODO: Replace this with some other mechanism, i.e. virtual call. + int worker_id = 0; + if (auto *distributed_db = + dynamic_cast<database::DistributedGraphDb *>(&db_accessor_->db())) { + worker_id = distributed_db->WorkerId(); + } else { + CHECK(dynamic_cast<database::SingleNode *>(&db_accessor_->db())); + } + return is_local() + ? storage::Address<mvcc::VersionList<TRecord>>(gid(), worker_id) + : address_; } template <typename TRecord> diff --git a/src/utils/random_graph_generator.hpp b/src/utils/random_graph_generator.hpp index 41bbe307b..8b7263d80 100644 --- a/src/utils/random_graph_generator.hpp +++ b/src/utils/random_graph_generator.hpp @@ -60,10 +60,10 @@ class RandomGraphGenerator { */ void AddVertices(int count, const std::vector<std::string> &label_names, int thread_count, int batch_size = 2000) { - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); std::vector<storage::Label> labels; for (const auto &label_name : label_names) - labels.push_back(dba.Label(label_name)); + labels.push_back(dba->Label(label_name)); Map( [&labels, this](database::GraphDbAccessor &dba) { @@ -80,8 +80,8 @@ class RandomGraphGenerator { * regardless of their labels. */ int64_t VertexCount() const { - database::GraphDbAccessor accessor(db_); - return CountIterable(accessor.Vertices(true)); + auto accessor = db_.Access(); + return CountIterable(accessor->Vertices(true)); } /** @@ -105,12 +105,12 @@ class RandomGraphGenerator { auto vertices_from = FilterVertices(from_filter); auto vertices_to = FilterVertices(to_filter); - database::GraphDbAccessor dba(db_); - auto edge_type = dba.EdgeType(edge_type_name); + auto dba = db_.Access(); + auto edge_type = dba->EdgeType(edge_type_name); // for small vertex counts reduce the batch size batch_size = - std::min(batch_size, static_cast<int>(dba.VerticesCount() / 1000 + 1)); + std::min(batch_size, static_cast<int>(dba->VerticesCount() / 1000 + 1)); Map( [&vertices_from, &vertices_to, edge_type, @@ -132,8 +132,8 @@ class RandomGraphGenerator { * regardless of their types and origin/destination labels. */ int64_t EdgeCount() const { - database::GraphDbAccessor accessor(db_); - return CountIterable(accessor.Edges(true)); + auto accessor = db_.Access(); + return CountIterable(accessor->Edges(true)); } /** @@ -150,11 +150,11 @@ class RandomGraphGenerator { const std::string &prop_name, std::function<TValue()> value_generator, std::function<bool(VertexAccessor &va)> predicate = {}) { if (!predicate) predicate = [](VertexAccessor &) { return true; }; - database::GraphDbAccessor dba(db_); - auto property = dba.Property(prop_name); - for (VertexAccessor va : dba.Vertices(false)) + auto dba = db_.Access(); + auto property = dba->Property(prop_name); + for (VertexAccessor va : dba->Vertices(false)) if (predicate(va)) va.PropsSet(property, value_generator()); - dba.Commit(); + dba->Commit(); } private: @@ -179,8 +179,8 @@ class RandomGraphGenerator { std::function<bool(VertexAccessor &item)> predicate = {}) { if (!predicate) predicate = [](VertexAccessor &) { return true; }; std::vector<VertexAccessor> r_val; - database::GraphDbAccessor dba(db_); - for (VertexAccessor &item : dba.Vertices(false)) + auto dba = db_.Access(); + for (VertexAccessor &item : dba->Vertices(false)) if (predicate(item)) r_val.emplace_back(item); return r_val; @@ -210,14 +210,14 @@ class RandomGraphGenerator { threads.emplace_back([count_per_thread, &f, this, elements_per_commit]() { for (int i = 0; i < count_per_thread; i += elements_per_commit) { while (true) { - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); try { int apply_count = std::min(elements_per_commit, count_per_thread - i); while (apply_count--) { - f(dba); + f(*dba); } - dba.Commit(); + dba->Commit(); break; } catch (...) { } diff --git a/tests/benchmark/expansion.cpp b/tests/benchmark/expansion.cpp index 369b75437..abce0359d 100644 --- a/tests/benchmark/expansion.cpp +++ b/tests/benchmark/expansion.cpp @@ -18,24 +18,24 @@ class ExpansionBenchFixture : public benchmark::Fixture { void SetUp(const benchmark::State &state) override { db_.emplace(); interpreter_.emplace(db_.value()); - database::GraphDbAccessor dba(*db_); - for (int i = 0; i < state.range(0); i++) dba.InsertVertex(); + auto dba = db_->Access(); + for (int i = 0; i < state.range(0); i++) dba->InsertVertex(); // the fixed part is one vertex expanding to 1000 others - auto start = dba.InsertVertex(); - start.add_label(dba.Label("Starting")); - auto edge_type = dba.EdgeType("edge_type"); + auto start = dba->InsertVertex(); + start.add_label(dba->Label("Starting")); + auto edge_type = dba->EdgeType("edge_type"); for (int i = 0; i < 1000; i++) { - auto dest = dba.InsertVertex(); - dba.InsertEdge(start, dest, edge_type); + auto dest = dba->InsertVertex(); + dba->InsertEdge(start, dest, edge_type); } - dba.Commit(); + dba->Commit(); } void TearDown(const benchmark::State &) override { - database::GraphDbAccessor dba(*db_); - for (auto vertex : dba.Vertices(false)) dba.DetachRemoveVertex(vertex); - dba.Commit(); + auto dba = db_->Access(); + for (auto vertex : dba->Vertices(false)) dba->DetachRemoveVertex(vertex); + dba->Commit(); interpreter_ = std::experimental::nullopt; db_ = std::experimental::nullopt; } @@ -45,10 +45,10 @@ class ExpansionBenchFixture : public benchmark::Fixture { BENCHMARK_DEFINE_F(ExpansionBenchFixture, Match)(benchmark::State &state) { auto query = "MATCH (s:Starting) return s"; - database::GraphDbAccessor dba(*db_); + auto dba = db_->Access(); while (state.KeepRunning()) { ResultStreamFaker<query::TypedValue> results; - interpreter()(query, dba, {}, false).PullAll(results); + interpreter()(query, *dba, {}, false).PullAll(results); } } @@ -59,10 +59,10 @@ BENCHMARK_REGISTER_F(ExpansionBenchFixture, Match) BENCHMARK_DEFINE_F(ExpansionBenchFixture, Expand)(benchmark::State &state) { auto query = "MATCH (s:Starting) WITH s MATCH (s)--(d) RETURN count(d)"; - database::GraphDbAccessor dba(*db_); + auto dba = db_->Access(); while (state.KeepRunning()) { ResultStreamFaker<query::TypedValue> results; - interpreter()(query, dba, {}, false).PullAll(results); + interpreter()(query, *dba, {}, false).PullAll(results); } } diff --git a/tests/benchmark/query/planner.cpp b/tests/benchmark/query/planner.cpp index 052630f9c..203a77e84 100644 --- a/tests/benchmark/query/planner.cpp +++ b/tests/benchmark/query/planner.cpp @@ -32,7 +32,7 @@ static void AddChainedMatches(int num_matches, query::AstStorage &storage) { static void BM_PlanChainedMatches(benchmark::State &state) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); while (state.KeepRunning()) { state.PauseTiming(); query::AstStorage storage; @@ -41,7 +41,7 @@ static void BM_PlanChainedMatches(benchmark::State &state) { query::SymbolTable symbol_table; query::SymbolGenerator symbol_generator(symbol_table); storage.query()->Accept(symbol_generator); - auto ctx = query::plan::MakePlanningContext(storage, symbol_table, dba); + auto ctx = query::plan::MakePlanningContext(storage, symbol_table, *dba); state.ResumeTiming(); auto query_parts = query::plan::CollectQueryParts(symbol_table, storage); if (query_parts.query_parts.size() == 0) { @@ -85,18 +85,18 @@ static void AddIndexedMatches( static auto CreateIndexedVertices(int index_count, int vertex_count, database::GraphDb &db) { - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); - database::GraphDbAccessor(db).BuildIndex(label, prop); - database::GraphDbAccessor dba(db); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); + db.Access()->BuildIndex(label, prop); + auto dba = db.Access(); for (int vi = 0; vi < vertex_count; ++vi) { for (int index = 0; index < index_count; ++index) { - auto vertex = dba.InsertVertex(); + auto vertex = dba->InsertVertex(); vertex.add_label(label); vertex.PropsSet(prop, index); } } - dba.Commit(); + dba->Commit(); return std::make_pair(label, prop); } @@ -107,7 +107,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); - database::GraphDbAccessor dba(db); + auto dba = db.Access(); Parameters parameters; while (state.KeepRunning()) { state.PauseTiming(); @@ -118,7 +118,7 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) { query::SymbolGenerator symbol_generator(symbol_table); storage.query()->Accept(symbol_generator); state.ResumeTiming(); - auto ctx = query::plan::MakePlanningContext(storage, symbol_table, dba); + auto ctx = query::plan::MakePlanningContext(storage, symbol_table, *dba); auto query_parts = query::plan::CollectQueryParts(symbol_table, storage); if (query_parts.query_parts.size() == 0) { std::exit(EXIT_FAILURE); @@ -127,7 +127,7 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) { auto plans = query::plan::MakeLogicalPlanForSingleQuery< query::plan::VariableStartPlanner>(single_query_parts, ctx); for (auto plan : plans) { - query::plan::EstimatePlanCost(dba, parameters, *plan); + query::plan::EstimatePlanCost(*dba, parameters, *plan); } } } @@ -140,8 +140,8 @@ static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts( int index_count = state.range(0); int vertex_count = state.range(1); std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db); - database::GraphDbAccessor dba(db); - auto vertex_counts = query::plan::MakeVertexCountCache(dba); + auto dba = db.Access(); + auto vertex_counts = query::plan::MakeVertexCountCache(*dba); Parameters parameters; while (state.KeepRunning()) { state.PauseTiming(); diff --git a/tests/manual/distributed_common.hpp b/tests/manual/distributed_common.hpp index e70f3ef4a..5923660f1 100644 --- a/tests/manual/distributed_common.hpp +++ b/tests/manual/distributed_common.hpp @@ -64,10 +64,10 @@ class Cluster { auto Execute(const std::string &query, std::map<std::string, query::TypedValue> params = {}) { - database::GraphDbAccessor dba(*master_); + auto dba = master_->Access(); ResultStreamFaker<query::TypedValue> result; - interpreter_->operator()(query, dba, params, false).PullAll(result); - dba.Commit(); + interpreter_->operator()(query, *dba, params, false).PullAll(result); + dba->Commit(); return result.GetResults(); }; diff --git a/tests/manual/query_planner.cpp b/tests/manual/query_planner.cpp index 20c775478..855a242e6 100644 --- a/tests/manual/query_planner.cpp +++ b/tests/manual/query_planner.cpp @@ -832,10 +832,10 @@ int main(int argc, char *argv[]) { std::exit(EXIT_FAILURE); } database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); Timer planning_timer; InteractiveDbAccessor interactive_db( - dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0, + *dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0, planning_timer); if (!in_db_filename.empty()) { std::ifstream db_file(in_db_filename); @@ -846,7 +846,7 @@ int main(int argc, char *argv[]) { if (!line || *line == "quit") break; if (line->empty()) continue; try { - auto ast = MakeAst(*line, dba); + auto ast = MakeAst(*line, *dba); auto symbol_table = MakeSymbolTable(ast); planning_timer.Start(); auto plans = MakeLogicalPlans(ast, symbol_table, interactive_db); @@ -856,7 +856,7 @@ int main(int argc, char *argv[]) { << std::chrono::duration<double, std::milli>(planning_time).count() << "ms" << std::endl; std::cout << "Generated " << plans.size() << " plans" << std::endl; - ExaminePlans(dba, symbol_table, plans); + ExaminePlans(*dba, symbol_table, plans); } catch (const utils::BasicException &e) { std::cout << "Error: " << e.what() << std::endl; } diff --git a/tests/manual/single_query.cpp b/tests/manual/single_query.cpp index 48d186514..21b7397bc 100644 --- a/tests/manual/single_query.cpp +++ b/tests/manual/single_query.cpp @@ -12,9 +12,9 @@ int main(int argc, char *argv[]) { exit(1); } database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); ResultStreamFaker<query::TypedValue> stream; - auto results = query::Interpreter{db}(argv[1], dba, {}, false); + auto results = query::Interpreter{db}(argv[1], *dba, {}, false); stream.Header(results.header()); results.PullAll(stream); stream.Summary(results.summary()); diff --git a/tests/property_based/random_graph.cpp b/tests/property_based/random_graph.cpp index fe11e6854..38f3bb502 100644 --- a/tests/property_based/random_graph.cpp +++ b/tests/property_based/random_graph.cpp @@ -29,11 +29,11 @@ RC_GTEST_PROP(RandomGraph, RandomGraph, (std::vector<std::string> vertex_labels, std::unordered_map<VertexAccessor, std::string> vertex_label_map; std::unordered_map<EdgeAccessor, std::string> edge_type_map; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); for (auto label : vertex_labels) { - auto vertex_accessor = dba.InsertVertex(); - vertex_accessor.add_label(dba.Label(label)); + auto vertex_accessor = dba->InsertVertex(); + vertex_accessor.add_label(dba->Label(label)); vertex_label_map.insert({vertex_accessor, label}); vertices.push_back(vertex_accessor); } @@ -41,23 +41,23 @@ RC_GTEST_PROP(RandomGraph, RandomGraph, (std::vector<std::string> vertex_labels, for (auto type : edge_types) { auto from = vertices[*rc::gen::inRange(0, vertices_num)]; auto to = vertices[*rc::gen::inRange(0, vertices_num)]; - auto edge_accessor = dba.InsertEdge(from, to, dba.EdgeType(type)); + auto edge_accessor = dba->InsertEdge(from, to, dba->EdgeType(type)); edge_type_map.insert({edge_accessor, type}); } - dba.AdvanceCommand(); + dba->AdvanceCommand(); int edges_num_check = 0; int vertices_num_check = 0; - for (const auto &vertex : dba.Vertices(false)) { + for (const auto &vertex : dba->Vertices(false)) { auto label = vertex_label_map.at(vertex); RC_ASSERT(vertex.labels().size() == 1); - RC_ASSERT(dba.LabelName(vertex.labels()[0]) == label); + RC_ASSERT(dba->LabelName(vertex.labels()[0]) == label); vertices_num_check++; } - for (const auto &edge : dba.Edges(false)) { + for (const auto &edge : dba->Edges(false)) { auto type = edge_type_map.at(edge); - RC_ASSERT(dba.EdgeTypeName(edge.EdgeType()) == type); + RC_ASSERT(dba->EdgeTypeName(edge.EdgeType()) == type); edges_num_check++; } RC_ASSERT(vertices_num_check == vertices_num); diff --git a/tests/unit/bolt_encoder.cpp b/tests/unit/bolt_encoder.cpp index c19505c99..1c69794ae 100644 --- a/tests/unit/bolt_encoder.cpp +++ b/tests/unit/bolt_encoder.cpp @@ -165,24 +165,24 @@ TEST(BoltEncoder, VertexAndEdge) { // create vertex database::SingleNode db; - database::GraphDbAccessor db_accessor(db); - auto va1 = db_accessor.InsertVertex(); - auto va2 = db_accessor.InsertVertex(); - auto l1 = db_accessor.Label("label1"); - auto l2 = db_accessor.Label("label2"); + auto db_accessor = db.Access(); + auto va1 = db_accessor->InsertVertex(); + auto va2 = db_accessor->InsertVertex(); + auto l1 = db_accessor->Label("label1"); + auto l2 = db_accessor->Label("label2"); va1.add_label(l1); va1.add_label(l2); - auto p1 = db_accessor.Property("prop1"); - auto p2 = db_accessor.Property("prop2"); + auto p1 = db_accessor->Property("prop1"); + auto p2 = db_accessor->Property("prop2"); PropertyValue pv1(12), pv2(200); va1.PropsSet(p1, pv1); va1.PropsSet(p2, pv2); // create edge - auto et = db_accessor.EdgeType("edgetype"); - auto ea = db_accessor.InsertEdge(va1, va2, et); - auto p3 = db_accessor.Property("prop3"); - auto p4 = db_accessor.Property("prop4"); + auto et = db_accessor->EdgeType("edgetype"); + auto ea = db_accessor->InsertEdge(va1, va2, et); + auto p3 = db_accessor->Property("prop3"); + auto p4 = db_accessor->Property("prop4"); PropertyValue pv3(42), pv4(1234); ea.PropsSet(p3, pv3); ea.PropsSet(p4, pv4); diff --git a/tests/unit/cypher_main_visitor.cpp b/tests/unit/cypher_main_visitor.cpp index 24ba8aec7..d4e77e0bf 100644 --- a/tests/unit/cypher_main_visitor.cpp +++ b/tests/unit/cypher_main_visitor.cpp @@ -35,12 +35,12 @@ class Base { public: explicit Base(const std::string &query) : query_string_(query) {} database::SingleNode db_; - database::GraphDbAccessor db_accessor_{db_}; - Context context_{db_accessor_}; + std::unique_ptr<database::GraphDbAccessor> db_accessor_{db_.Access()}; + Context context_{*db_accessor_}; std::string query_string_; auto Prop(const std::string &prop_name) { - return db_accessor_.Property(prop_name); + return db_accessor_->Property(prop_name); } auto PropPair(const std::string &prop_name) { @@ -203,7 +203,7 @@ TYPED_TEST(CypherMainVisitorTest, PropertyLookup) { ASSERT_TRUE(identifier); ASSERT_EQ(identifier->name_, "n"); ASSERT_EQ(property_lookup->property_, - ast_generator.db_accessor_.Property("x")); + ast_generator.db_accessor_->Property("x")); } TYPED_TEST(CypherMainVisitorTest, LabelsTest) { @@ -220,8 +220,8 @@ TYPED_TEST(CypherMainVisitorTest, LabelsTest) { ASSERT_TRUE(identifier); ASSERT_EQ(identifier->name_, "n"); ASSERT_THAT(labels_test->labels_, - ElementsAre(ast_generator.db_accessor_.Label("x"), - ast_generator.db_accessor_.Label("y"))); + ElementsAre(ast_generator.db_accessor_->Label("x"), + ast_generator.db_accessor_->Label("y"))); } TYPED_TEST(CypherMainVisitorTest, EscapedLabel) { @@ -236,7 +236,7 @@ TYPED_TEST(CypherMainVisitorTest, EscapedLabel) { auto identifier = dynamic_cast<Identifier *>(labels_test->expression_); ASSERT_EQ(identifier->name_, "n"); ASSERT_THAT(labels_test->labels_, - ElementsAre(ast_generator.db_accessor_.Label("l-$\"'ab`e``l"))); + ElementsAre(ast_generator.db_accessor_->Label("l-$\"'ab`e``l"))); } TYPED_TEST(CypherMainVisitorTest, KeywordLabel) { @@ -251,7 +251,7 @@ TYPED_TEST(CypherMainVisitorTest, KeywordLabel) { auto identifier = dynamic_cast<Identifier *>(labels_test->expression_); ASSERT_EQ(identifier->name_, "n"); ASSERT_THAT(labels_test->labels_, - ElementsAre(ast_generator.db_accessor_.Label("DEletE"))); + ElementsAre(ast_generator.db_accessor_->Label("DEletE"))); } TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) { @@ -266,7 +266,7 @@ TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) { auto identifier = dynamic_cast<Identifier *>(labels_test->expression_); EXPECT_EQ(identifier->name_, "n"); ASSERT_THAT(labels_test->labels_, - ElementsAre(ast_generator.db_accessor_.Label("a"))); + ElementsAre(ast_generator.db_accessor_->Label("a"))); } TYPED_TEST(CypherMainVisitorTest, ReturnNoDistinctNoBagSemantics) { @@ -946,10 +946,10 @@ TYPED_TEST(CypherMainVisitorTest, NodePattern) { EXPECT_EQ(node->identifier_->name_, CypherMainVisitor::kAnonPrefix + std::to_string(1)); EXPECT_FALSE(node->identifier_->user_declared_); - EXPECT_THAT(node->labels_, - UnorderedElementsAre(ast_generator.db_accessor_.Label("label1"), - ast_generator.db_accessor_.Label("label2"), - ast_generator.db_accessor_.Label("label3"))); + EXPECT_THAT(node->labels_, UnorderedElementsAre( + ast_generator.db_accessor_->Label("label1"), + ast_generator.db_accessor_->Label("label2"), + ast_generator.db_accessor_->Label("label3"))); std::map<std::pair<std::string, storage::Property>, int64_t> properties; for (auto x : node->properties_) { TypedValue value = LiteralValue(ast_generator.context_, x.second); @@ -1048,8 +1048,8 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternDetails) { EXPECT_EQ(edge->direction_, EdgeAtom::Direction::IN); EXPECT_THAT( edge->edge_types_, - UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"), - ast_generator.db_accessor_.EdgeType("type2"))); + UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"), + ast_generator.db_accessor_->EdgeType("type2"))); std::map<std::pair<std::string, storage::Property>, int64_t> properties; for (auto x : edge->properties_) { TypedValue value = LiteralValue(ast_generator.context_, x.second); @@ -1205,7 +1205,7 @@ TYPED_TEST(CypherMainVisitorTest, CheckLiteral(ast_generator.context_, edge->properties_[ast_generator.PropPair("prop")], 42); ASSERT_EQ(edge->edge_types_.size(), 1U); - auto edge_type = ast_generator.db_accessor_.EdgeType("edge_type"); + auto edge_type = ast_generator.db_accessor_->EdgeType("edge_type"); EXPECT_EQ(edge->edge_types_[0], edge_type); } @@ -1341,7 +1341,7 @@ TYPED_TEST(CypherMainVisitorTest, Set) { ASSERT_TRUE(identifier1); ASSERT_EQ(identifier1->name_, "a"); ASSERT_EQ(set_property->property_lookup_->property_, - ast_generator.db_accessor_.Property("x")); + ast_generator.db_accessor_->Property("x")); auto *identifier2 = dynamic_cast<Identifier *>(set_property->expression_); ASSERT_EQ(identifier2->name_, "b"); } @@ -1376,8 +1376,8 @@ TYPED_TEST(CypherMainVisitorTest, Set) { ASSERT_TRUE(set_labels->identifier_); ASSERT_EQ(set_labels->identifier_->name_, "g"); ASSERT_THAT(set_labels->labels_, - UnorderedElementsAre(ast_generator.db_accessor_.Label("h"), - ast_generator.db_accessor_.Label("i"))); + UnorderedElementsAre(ast_generator.db_accessor_->Label("h"), + ast_generator.db_accessor_->Label("i"))); } } @@ -1398,7 +1398,7 @@ TYPED_TEST(CypherMainVisitorTest, Remove) { ASSERT_TRUE(identifier1); ASSERT_EQ(identifier1->name_, "a"); ASSERT_EQ(remove_property->property_lookup_->property_, - ast_generator.db_accessor_.Property("x")); + ast_generator.db_accessor_->Property("x")); } { auto *remove_labels = @@ -1407,8 +1407,8 @@ TYPED_TEST(CypherMainVisitorTest, Remove) { ASSERT_TRUE(remove_labels->identifier_); ASSERT_EQ(remove_labels->identifier_->name_, "g"); ASSERT_THAT(remove_labels->labels_, - UnorderedElementsAre(ast_generator.db_accessor_.Label("h"), - ast_generator.db_accessor_.Label("i"))); + UnorderedElementsAre(ast_generator.db_accessor_->Label("h"), + ast_generator.db_accessor_->Label("i"))); } } @@ -1601,9 +1601,9 @@ TYPED_TEST(CypherMainVisitorTest, CreateIndex) { ASSERT_EQ(single_query->clauses_.size(), 1U); auto *create_index = dynamic_cast<CreateIndex *>(single_query->clauses_[0]); ASSERT_TRUE(create_index); - ASSERT_EQ(create_index->label_, ast_generator.db_accessor_.Label("mirko")); + ASSERT_EQ(create_index->label_, ast_generator.db_accessor_->Label("mirko")); ASSERT_EQ(create_index->property_, - ast_generator.db_accessor_.Property("slavko")); + ast_generator.db_accessor_->Property("slavko")); } TYPED_TEST(CypherMainVisitorTest, ReturnAll) { @@ -1700,8 +1700,8 @@ TYPED_TEST(CypherMainVisitorTest, MatchBfsReturn) { EXPECT_EQ(bfs->direction_, EdgeAtom::Direction::OUT); EXPECT_THAT( bfs->edge_types_, - UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"), - ast_generator.db_accessor_.EdgeType("type2"))); + UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"), + ast_generator.db_accessor_->EdgeType("type2"))); EXPECT_EQ(bfs->identifier_->name_, "r"); EXPECT_EQ(bfs->filter_lambda_.inner_edge->name_, "e"); EXPECT_TRUE(bfs->filter_lambda_.inner_edge->user_declared_); @@ -1748,8 +1748,8 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestReturn) { EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT); EXPECT_THAT( shortest->edge_types_, - UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"), - ast_generator.db_accessor_.EdgeType("type2"))); + UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"), + ast_generator.db_accessor_->EdgeType("type2"))); CheckLiteral(ast_generator.context_, shortest->upper_bound_, 10); EXPECT_FALSE(shortest->lower_bound_); EXPECT_EQ(shortest->identifier_->name_, "r"); @@ -1788,8 +1788,8 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestNoFilterReturn) { EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT); EXPECT_THAT( shortest->edge_types_, - UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"), - ast_generator.db_accessor_.EdgeType("type2"))); + UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"), + ast_generator.db_accessor_->EdgeType("type2"))); CheckLiteral(ast_generator.context_, shortest->upper_bound_, 10); EXPECT_FALSE(shortest->lower_bound_); EXPECT_EQ(shortest->identifier_->name_, "r"); diff --git a/tests/unit/database_key_index.cpp b/tests/unit/database_key_index.cpp index c87b452a5..394329377 100644 --- a/tests/unit/database_key_index.cpp +++ b/tests/unit/database_key_index.cpp @@ -15,7 +15,7 @@ using testing::UnorderedElementsAreArray; TEST(LabelsIndex, UniqueInsert) { database::KeyIndex<storage::Label, Vertex> index; database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); tx::SingleNodeEngine engine; auto t1 = engine.Begin(); @@ -23,28 +23,28 @@ TEST(LabelsIndex, UniqueInsert) { engine.Commit(*t1); auto t2 = engine.Begin(); - vlist.find(*t2)->labels_.push_back(dba.Label("1")); - index.Update(dba.Label("1"), &vlist, vlist.find(*t2)); + vlist.find(*t2)->labels_.push_back(dba->Label("1")); + index.Update(dba->Label("1"), &vlist, vlist.find(*t2)); // Try multiple inserts - index.Update(dba.Label("1"), &vlist, vlist.find(*t2)); + index.Update(dba->Label("1"), &vlist, vlist.find(*t2)); - vlist.find(*t2)->labels_.push_back(dba.Label("2")); - index.Update(dba.Label("2"), &vlist, vlist.find(*t2)); + vlist.find(*t2)->labels_.push_back(dba->Label("2")); + index.Update(dba->Label("2"), &vlist, vlist.find(*t2)); - vlist.find(*t2)->labels_.push_back(dba.Label("3")); - index.Update(dba.Label("3"), &vlist, vlist.find(*t2)); + vlist.find(*t2)->labels_.push_back(dba->Label("3")); + index.Update(dba->Label("3"), &vlist, vlist.find(*t2)); engine.Commit(*t2); - EXPECT_EQ(index.Count(dba.Label("1")), 1); - EXPECT_EQ(index.Count(dba.Label("2")), 1); - EXPECT_EQ(index.Count(dba.Label("3")), 1); + EXPECT_EQ(index.Count(dba->Label("1")), 1); + EXPECT_EQ(index.Count(dba->Label("2")), 1); + EXPECT_EQ(index.Count(dba->Label("3")), 1); } // Check if index filters duplicates. TEST(LabelsIndex, UniqueFilter) { database::SingleNode db; database::KeyIndex<storage::Label, Vertex> index; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); tx::SingleNodeEngine engine; auto t1 = engine.Begin(); @@ -55,7 +55,7 @@ TEST(LabelsIndex, UniqueFilter) { auto r1v2 = vlist2.find(*t1); EXPECT_NE(vlist1.find(*t1), nullptr); - auto label1 = dba.Label("1"); + auto label1 = dba->Label("1"); vlist1.find(*t1)->labels_.push_back(label1); vlist2.find(*t1)->labels_.push_back(label1); index.Update(label1, &vlist1, r1v1); @@ -84,7 +84,7 @@ TEST(LabelsIndex, UniqueFilter) { TEST(LabelsIndex, Refresh) { database::KeyIndex<storage::Label, Vertex> index; database::SingleNode db; - database::GraphDbAccessor access(db); + auto access = db.Access(); tx::SingleNodeEngine engine; // add two vertices to database @@ -98,7 +98,7 @@ TEST(LabelsIndex, Refresh) { EXPECT_NE(v1r1, nullptr); EXPECT_NE(v2r1, nullptr); - auto label = access.Label("label"); + auto label = access->Label("label"); v1r1->labels_.push_back(label); v2r1->labels_.push_back(label); index.Update(label, &vlist1, v1r1); @@ -123,10 +123,10 @@ TEST(LabelsIndex, Refresh) { // Transaction hasn't ended and so the vertex is not visible. TEST(LabelsIndexDb, AddGetZeroLabels) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto vertex = dba.InsertVertex(); - vertex.add_label(dba.Label("test")); - auto collection = dba.Vertices(dba.Label("test"), false); + auto dba = db.Access(); + auto vertex = dba->InsertVertex(); + vertex.add_label(dba->Label("test")); + auto collection = dba->Vertices(dba->Label("test"), false); std::vector<VertexAccessor> collection_vector(collection.begin(), collection.end()); EXPECT_EQ(collection_vector.size(), (size_t)0); @@ -137,61 +137,61 @@ TEST(LabelsIndexDb, AddGetZeroLabels) { TEST(LabelsIndexDb, AddGetRemoveLabel) { database::SingleNode db; { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto vertex1 = dba.InsertVertex(); - vertex1.add_label(dba.Label("test")); + auto vertex1 = dba->InsertVertex(); + vertex1.add_label(dba->Label("test")); - auto vertex2 = dba.InsertVertex(); - vertex2.add_label(dba.Label("test2")); + auto vertex2 = dba->InsertVertex(); + vertex2.add_label(dba->Label("test2")); - auto vertex3 = dba.InsertVertex(); - vertex3.add_label(dba.Label("test")); + auto vertex3 = dba->InsertVertex(); + vertex3.add_label(dba->Label("test")); - dba.Commit(); + dba->Commit(); } // Finish transaction. { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto filtered = dba.Vertices(dba.Label("test"), false); + auto filtered = dba->Vertices(dba->Label("test"), false); std::vector<VertexAccessor> collection(filtered.begin(), filtered.end()); - auto vertices = dba.Vertices(false); + auto vertices = dba->Vertices(false); std::vector<VertexAccessor> expected_collection; for (auto vertex : vertices) { - if (vertex.has_label(dba.Label("test"))) { + if (vertex.has_label(dba->Label("test"))) { expected_collection.push_back(vertex); } else { - EXPECT_TRUE(vertex.has_label(dba.Label("test2"))); + EXPECT_TRUE(vertex.has_label(dba->Label("test2"))); } } EXPECT_EQ(expected_collection.size(), collection.size()); - EXPECT_TRUE(collection[0].has_label(dba.Label("test"))); - EXPECT_TRUE(collection[1].has_label(dba.Label("test"))); - EXPECT_FALSE(collection[0].has_label(dba.Label("test2"))); - EXPECT_FALSE(collection[1].has_label(dba.Label("test2"))); - dba.RemoveVertex(collection[0]); // Remove from database and test if - // index won't return it. + EXPECT_TRUE(collection[0].has_label(dba->Label("test"))); + EXPECT_TRUE(collection[1].has_label(dba->Label("test"))); + EXPECT_FALSE(collection[0].has_label(dba->Label("test2"))); + EXPECT_FALSE(collection[1].has_label(dba->Label("test2"))); + dba->RemoveVertex(collection[0]); // Remove from database and test if + // index won't return it. // Remove label from the vertex and add new label. - collection[1].remove_label(dba.Label("test")); - collection[1].add_label(dba.Label("test2")); - dba.Commit(); + collection[1].remove_label(dba->Label("test")); + collection[1].add_label(dba->Label("test2")); + dba->Commit(); } { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto filtered = dba.Vertices(dba.Label("test"), false); + auto filtered = dba->Vertices(dba->Label("test"), false); std::vector<VertexAccessor> collection(filtered.begin(), filtered.end()); - auto vertices = dba.Vertices(false); + auto vertices = dba->Vertices(false); std::vector<VertexAccessor> expected_collection; for (auto vertex : vertices) { - if (vertex.has_label(dba.Label("test"))) { + if (vertex.has_label(dba->Label("test"))) { expected_collection.push_back(vertex); } else { - EXPECT_TRUE(vertex.has_label(dba.Label("test2"))); + EXPECT_TRUE(vertex.has_label(dba->Label("test2"))); } } diff --git a/tests/unit/database_label_property_index.cpp b/tests/unit/database_label_property_index.cpp index a22c9e1c9..b6400e370 100644 --- a/tests/unit/database_label_property_index.cpp +++ b/tests/unit/database_label_property_index.cpp @@ -12,12 +12,12 @@ using namespace database; class LabelPropertyIndexComplexTest : public ::testing::Test { protected: virtual void SetUp() { - GraphDbAccessor accessor(db_); + auto accessor = db_.Access(); - label = accessor.Label("label"); - property = accessor.Property("property"); - label2 = accessor.Label("label2"); - property2 = accessor.Property("property2"); + label = accessor->Label("label"); + property = accessor->Property("property"); + label2 = accessor->Label("label2"); + property2 = accessor->Property("property2"); key = new LabelPropertyIndex::Key(label, property); EXPECT_EQ(index.CreateIndex(*key), true); @@ -59,9 +59,9 @@ class LabelPropertyIndexComplexTest : public ::testing::Test { TEST(LabelPropertyIndex, CreateIndex) { SingleNode db; - GraphDbAccessor accessor(db); - LabelPropertyIndex::Key key(accessor.Label("test"), - accessor.Property("test2")); + auto accessor = db.Access(); + LabelPropertyIndex::Key key(accessor->Label("test"), + accessor->Property("test2")); LabelPropertyIndex index; EXPECT_EQ(index.CreateIndex(key), true); EXPECT_EQ(index.CreateIndex(key), false); @@ -69,9 +69,9 @@ TEST(LabelPropertyIndex, CreateIndex) { TEST(LabelPropertyIndex, DeleteIndex) { SingleNode db; - GraphDbAccessor accessor(db); - LabelPropertyIndex::Key key(accessor.Label("test"), - accessor.Property("test2")); + auto accessor = db.Access(); + LabelPropertyIndex::Key key(accessor->Label("test"), + accessor->Property("test2")); LabelPropertyIndex index; EXPECT_EQ(index.CreateIndex(key), true); EXPECT_EQ(index.CreateIndex(key), false); @@ -81,9 +81,9 @@ TEST(LabelPropertyIndex, DeleteIndex) { TEST(LabelPropertyIndex, IndexExistance) { SingleNode db; - GraphDbAccessor accessor(db); - LabelPropertyIndex::Key key(accessor.Label("test"), - accessor.Property("test2")); + auto accessor = db.Access(); + LabelPropertyIndex::Key key(accessor->Label("test"), + accessor->Property("test2")); LabelPropertyIndex index; EXPECT_EQ(index.CreateIndex(key), true); // Index doesn't exist - and can't be used untill it's been notified as built. @@ -94,9 +94,9 @@ TEST(LabelPropertyIndex, IndexExistance) { TEST(LabelPropertyIndex, Count) { SingleNode db; - GraphDbAccessor accessor(db); - auto label = accessor.Label("label"); - auto property = accessor.Property("property"); + auto accessor = db.Access(); + auto label = accessor->Label("label"); + auto property = accessor->Property("property"); LabelPropertyIndex::Key key(label, property); LabelPropertyIndex index; ::testing::FLAGS_gtest_death_test_style = "threadsafe"; diff --git a/tests/unit/database_transaction_timeout.cpp b/tests/unit/database_transaction_timeout.cpp index 90c7cbf8e..f1ec5ef15 100644 --- a/tests/unit/database_transaction_timeout.cpp +++ b/tests/unit/database_transaction_timeout.cpp @@ -14,19 +14,19 @@ TEST(TransactionTimeout, TransactionTimeout) { auto interpret = [&](auto &dba, const std::string &query) { ResultStreamFaker<query::TypedValue> stream; interpreter(query, dba, {}, false).PullAll(stream); - }; { - database::GraphDbAccessor dba(db); - interpret(dba, "MATCH (n) RETURN n"); + auto dba = db.Access(); + interpret(*dba, "MATCH (n) RETURN n"); } { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); std::this_thread::sleep_for(std::chrono::seconds(5)); - ASSERT_THROW(interpret(dba, "MATCH (n) RETURN n"), query::HintedAbortError); + ASSERT_THROW(interpret(*dba, "MATCH (n) RETURN n"), + query::HintedAbortError); } { - database::GraphDbAccessor dba(db); - interpret(dba, "MATCH (n) RETURN n"); + auto dba = db.Access(); + interpret(*dba, "MATCH (n) RETURN n"); } } diff --git a/tests/unit/distributed_bfs.cpp b/tests/unit/distributed_bfs.cpp index 9ab941aef..861258b3b 100644 --- a/tests/unit/distributed_bfs.cpp +++ b/tests/unit/distributed_bfs.cpp @@ -33,18 +33,18 @@ class BfsTest : public DistributedGraphDbTest { }; TEST_F(BfsTest, Expansion) { - GraphDbAccessor dba{master()}; + auto dba = master().Access(); auto &clients = master().bfs_subcursor_clients(); auto subcursor_ids = clients.CreateBfsSubcursors( - dba.transaction_id(), query::EdgeAtom::Direction::BOTH, - {dba.EdgeType("Edge")}, query::GraphView::OLD); + dba->transaction_id(), query::EdgeAtom::Direction::BOTH, + {dba->EdgeType("Edge")}, query::GraphView::OLD); clients.RegisterSubcursors(subcursor_ids); clients.SetSource(subcursor_ids, vertices[0]); auto pull = [&clients, &subcursor_ids, &dba](int worker_id) { - return clients.Pull(worker_id, subcursor_ids[worker_id], &dba); + return clients.Pull(worker_id, subcursor_ids[worker_id], dba.get()); }; EXPECT_EQ(pull(0), std::experimental::nullopt); @@ -87,17 +87,17 @@ TEST_F(BfsTest, Expansion) { distributed::PathSegment ps; - ps = clients.ReconstructPath(subcursor_ids, vertices[3], &dba); + ps = clients.ReconstructPath(subcursor_ids, vertices[3], dba.get()); ASSERT_EQ(ps.next_vertex, vertices[4]); ASSERT_EQ(ps.next_edge, std::experimental::nullopt); compare(ps.edges, {{3, 4}}); - ps = clients.ReconstructPath(subcursor_ids, vertices[4], &dba); + ps = clients.ReconstructPath(subcursor_ids, vertices[4], dba.get()); EXPECT_EQ(ps.next_vertex, std::experimental::nullopt); EXPECT_EQ(ps.next_edge, (edges[{0, 1}])); compare(ps.edges, {{2, 4}, {1, 2}}); - ps = clients.ReconstructPath(subcursor_ids, edges[{0, 1}], &dba); + ps = clients.ReconstructPath(subcursor_ids, edges[{0, 1}], dba.get()); EXPECT_EQ(ps.next_vertex, std::experimental::nullopt); EXPECT_EQ(ps.next_edge, std::experimental::nullopt); compare(ps.edges, {{0, 1}}); diff --git a/tests/unit/distributed_common.hpp b/tests/unit/distributed_common.hpp index 1109d1af9..553feb3c1 100644 --- a/tests/unit/distributed_common.hpp +++ b/tests/unit/distributed_common.hpp @@ -98,9 +98,6 @@ class DistributedGraphDbTest : public ::testing::Test { } database::Master &master() { return *master_; } - auto &master_tx_engine() { - return dynamic_cast<tx::MasterEngine &>(master_->tx_engine()); - } database::Worker &worker(int worker_id) { return workers_[worker_id - 1]->worker_; @@ -109,9 +106,9 @@ class DistributedGraphDbTest : public ::testing::Test { /// Inserts a vertex and returns it's global address. Does it in a new /// transaction. storage::VertexAddress InsertVertex(database::GraphDb &db) { - database::GraphDbAccessor dba{db}; - auto r_val = dba.InsertVertex().GlobalAddress(); - dba.Commit(); + auto dba = db.Access(); + auto r_val = dba->InsertVertex().GlobalAddress(); + dba->Commit(); return r_val; } @@ -121,27 +118,27 @@ class DistributedGraphDbTest : public ::testing::Test { const std::string &edge_type_name) { CHECK(from_addr.is_remote() && to_addr.is_remote()) << "Distributed test InsertEdge only takes global addresses"; - database::GraphDbAccessor dba{master()}; - VertexAccessor from{from_addr, dba}; - VertexAccessor to{to_addr, dba}; + auto dba = master().Access(); + VertexAccessor from{from_addr, *dba}; + VertexAccessor to{to_addr, *dba}; auto r_val = - dba.InsertEdge(from, to, dba.EdgeType(edge_type_name)).GlobalAddress(); - master().updates_server().Apply(dba.transaction_id()); - worker(1).updates_server().Apply(dba.transaction_id()); - worker(2).updates_server().Apply(dba.transaction_id()); - dba.Commit(); + dba->InsertEdge(from, to, dba->EdgeType(edge_type_name)).GlobalAddress(); + master().updates_server().Apply(dba->transaction_id()); + worker(1).updates_server().Apply(dba->transaction_id()); + worker(2).updates_server().Apply(dba->transaction_id()); + dba->Commit(); return r_val; } auto VertexCount(database::GraphDb &db) { - database::GraphDbAccessor dba{db}; - auto vertices = dba.Vertices(false); + auto dba = db.Access(); + auto vertices = dba->Vertices(false); return std::distance(vertices.begin(), vertices.end()); }; auto EdgeCount(database::GraphDb &db) { - database::GraphDbAccessor dba(db); - auto edges = dba.Edges(false); + auto dba = db.Access(); + auto edges = dba->Edges(false); return std::distance(edges.begin(), edges.end()); }; diff --git a/tests/unit/distributed_data_exchange.cpp b/tests/unit/distributed_data_exchange.cpp index 6cc136786..86155b322 100644 --- a/tests/unit/distributed_data_exchange.cpp +++ b/tests/unit/distributed_data_exchange.cpp @@ -21,51 +21,51 @@ TEST_F(DistributedDataExchangeTest, RemoteDataGetting) { gid::Gid v1_id, v2_id, e1_id; { - GraphDbAccessor dba{master()}; - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto e1 = dba.InsertEdge(v1, v2, dba.EdgeType("et")); + auto dba = master().Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto e1 = dba->InsertEdge(v1, v2, dba->EdgeType("et")); // Set some data so we see we're getting the right stuff. - v1.PropsSet(dba.Property("p1"), 42); - v1.add_label(dba.Label("label")); - v2.PropsSet(dba.Property("p2"), "value"); - e1.PropsSet(dba.Property("p3"), true); + v1.PropsSet(dba->Property("p1"), 42); + v1.add_label(dba->Label("label")); + v2.PropsSet(dba->Property("p2"), "value"); + e1.PropsSet(dba->Property("p3"), true); v1_id = v1.gid(); v2_id = v2.gid(); e1_id = e1.gid(); - dba.Commit(); + dba->Commit(); } // The master must start a transaction before workers can work in it. - GraphDbAccessor master_dba{master()}; + auto master_dba = master().Access(); { - GraphDbAccessor w1_dba{worker(1), master_dba.transaction_id()}; - VertexAccessor v1_in_w1{{v1_id, 0}, w1_dba}; + auto w1_dba = worker(1).Access(master_dba->transaction_id()); + VertexAccessor v1_in_w1{{v1_id, 0}, *w1_dba}; EXPECT_NE(v1_in_w1.GetOld(), nullptr); EXPECT_EQ(v1_in_w1.GetNew(), nullptr); - EXPECT_EQ(v1_in_w1.PropsAt(w1_dba.Property("p1")).Value<int64_t>(), 42); - EXPECT_TRUE(v1_in_w1.has_label(w1_dba.Label("label"))); + EXPECT_EQ(v1_in_w1.PropsAt(w1_dba->Property("p1")).Value<int64_t>(), 42); + EXPECT_TRUE(v1_in_w1.has_label(w1_dba->Label("label"))); } { - GraphDbAccessor w2_dba{worker(2), master_dba.transaction_id()}; - VertexAccessor v2_in_w2{{v2_id, 0}, w2_dba}; + auto w2_dba = worker(2).Access(master_dba->transaction_id()); + VertexAccessor v2_in_w2{{v2_id, 0}, *w2_dba}; EXPECT_NE(v2_in_w2.GetOld(), nullptr); EXPECT_EQ(v2_in_w2.GetNew(), nullptr); - EXPECT_EQ(v2_in_w2.PropsAt(w2_dba.Property("p2")).Value<std::string>(), + EXPECT_EQ(v2_in_w2.PropsAt(w2_dba->Property("p2")).Value<std::string>(), "value"); - EXPECT_FALSE(v2_in_w2.has_label(w2_dba.Label("label"))); + EXPECT_FALSE(v2_in_w2.has_label(w2_dba->Label("label"))); - VertexAccessor v1_in_w2{{v1_id, 0}, w2_dba}; - EdgeAccessor e1_in_w2{{e1_id, 0}, w2_dba}; + VertexAccessor v1_in_w2{{v1_id, 0}, *w2_dba}; + EdgeAccessor e1_in_w2{{e1_id, 0}, *w2_dba}; EXPECT_EQ(e1_in_w2.from(), v1_in_w2); EXPECT_EQ(e1_in_w2.to(), v2_in_w2); - EXPECT_EQ(e1_in_w2.EdgeType(), w2_dba.EdgeType("et")); - EXPECT_EQ(e1_in_w2.PropsAt(w2_dba.Property("p3")).Value<bool>(), true); + EXPECT_EQ(e1_in_w2.EdgeType(), w2_dba->EdgeType("et")); + EXPECT_EQ(e1_in_w2.PropsAt(w2_dba->Property("p3")).Value<bool>(), true); } } @@ -76,7 +76,7 @@ TEST_F(DistributedDataExchangeTest, RemoteExpansion) { InsertEdge(from, to, "et"); { // Expand on the master for three hops. Collect vertex gids. - GraphDbAccessor dba{master()}; + auto dba = master().Access(); std::vector<VertexAccessor> visited; auto expand = [](auto &v) { @@ -86,7 +86,7 @@ TEST_F(DistributedDataExchangeTest, RemoteExpansion) { }; // Do a few hops back and forth, all on the master. - VertexAccessor v{from, dba}; + VertexAccessor v{from, *dba}; for (int i = 0; i < 5; ++i) { v = expand(v); EXPECT_FALSE(v.address().is_local()); @@ -101,13 +101,13 @@ TEST_F(DistributedDataExchangeTest, VertexCountsEqual) { for (int i = 0; i < 9; ++i) InsertVertex(worker(2)); { - GraphDbAccessor accessor(master()); + auto accessor = master().Access(); auto m_cnt = - master().data_clients().VertexCounts(accessor.transaction().id_); + master().data_clients().VertexCounts(accessor->transaction().id_); auto w1_cnt = - worker(1).data_clients().VertexCounts(accessor.transaction().id_); + worker(1).data_clients().VertexCounts(accessor->transaction().id_); auto w2_cnt = - worker(2).data_clients().VertexCounts(accessor.transaction().id_); + worker(2).data_clients().VertexCounts(accessor->transaction().id_); auto check = [&m_cnt, &w1_cnt, &w2_cnt](int key, int value) { return m_cnt[key] == w1_cnt[key] && w1_cnt[key] == w2_cnt[key] && @@ -122,17 +122,17 @@ TEST_F(DistributedDataExchangeTest, VertexCountsEqual) { TEST_F(DistributedDataExchangeTest, VertexCountsTransactional) { { - GraphDbAccessor accessor(master()); + auto accessor = master().Access(); InsertVertex(master()); EXPECT_EQ(master().data_clients().VertexCounts( - accessor.transaction().id_)[master().WorkerId()], + accessor->transaction().id_)[master().WorkerId()], 0); } // Transaction after insert which should now see the insertion { - GraphDbAccessor accessor(master()); + auto accessor = master().Access(); EXPECT_EQ(master().data_clients().VertexCounts( - accessor.transaction().id_)[master().WorkerId()], + accessor->transaction().id_)[master().WorkerId()], 1); } } diff --git a/tests/unit/distributed_durability.cpp b/tests/unit/distributed_durability.cpp index d25edb1ed..6025e99de 100644 --- a/tests/unit/distributed_durability.cpp +++ b/tests/unit/distributed_durability.cpp @@ -47,21 +47,21 @@ class DistributedDurability : public DistributedGraphDbTest { private: void AddVertex(database::GraphDb &db, const std::string &label) { - database::GraphDbAccessor dba(db); - auto vertex = dba.InsertVertex(); - vertex.add_label(dba.Label(label)); - dba.Commit(); + auto dba = db.Access(); + auto vertex = dba->InsertVertex(); + vertex.add_label(dba->Label(label)); + dba->Commit(); } void CheckVertex(database::GraphDb &db, int expected_count, const std::string &label) { - database::GraphDbAccessor dba(db); - auto it = dba.Vertices(false); + auto dba = db.Access(); + auto it = dba->Vertices(false); std::vector<VertexAccessor> vertices{it.begin(), it.end()}; EXPECT_EQ(vertices.size(), expected_count); for (auto &vertex : vertices) { ASSERT_EQ(vertex.labels().size(), 1); - EXPECT_EQ(vertex.labels()[0], dba.Label(label)); + EXPECT_EQ(vertex.labels()[0], dba->Label(label)); } } }; @@ -71,8 +71,8 @@ TEST_F(DistributedDurability, MakeSnapshot) { // of it { AddVertices(); - database::GraphDbAccessor dba(master()); - master().MakeSnapshot(dba); + auto dba = master().Access(); + master().MakeSnapshot(*dba); } // Recover the graph and check if it's the same as before { @@ -101,15 +101,15 @@ TEST_F(DistributedDurability, RecoveryFromSameSnapshot) { { AddVertices(); // Make snapshot on one worker, expect it won't recover from that. - database::GraphDbAccessor dba(worker(1)); - worker(1).MakeSnapshot(dba); + auto dba = worker(1).Access(); + worker(1).MakeSnapshot(*dba); } { RestartWithRecovery(); CheckVertices(0); AddVertices(); - database::GraphDbAccessor dba(master()); - master().MakeSnapshot(dba); + auto dba = master().Access(); + master().MakeSnapshot(*dba); } { RestartWithRecovery(); @@ -117,8 +117,8 @@ TEST_F(DistributedDurability, RecoveryFromSameSnapshot) { AddVertices(); CheckVertices(2); // Make snapshot on one worker, expect it won't recover from that. - database::GraphDbAccessor dba(worker(1)); - worker(1).MakeSnapshot(dba); + auto dba = worker(1).Access(); + worker(1).MakeSnapshot(*dba); } { RestartWithRecovery(); @@ -130,8 +130,8 @@ TEST_F(DistributedDurability, RecoveryFailure) { { AddVertices(); // Make a snapshot on the master without the right snapshots on workers. - database::GraphDbAccessor dba(master()); - bool status = durability::MakeSnapshot(master(), dba, tmp_dir_, 100); + auto dba = master().Access(); + bool status = durability::MakeSnapshot(master(), *dba, tmp_dir_, 100); ASSERT_TRUE(status); } ::testing::FLAGS_gtest_death_test_style = "threadsafe"; @@ -171,8 +171,8 @@ void CheckDeltas(fs::path wal_dir, database::StateDelta::Type op) { TEST_F(DistributedDurability, WriteCommittedTx) { RestartWithWal(); - database::GraphDbAccessor dba(master()); - dba.Commit(); + auto dba = master().Access(); + dba->Commit(); FlushAllWal(); CheckDeltas(tmp_dir_ / durability::kWalDir, database::StateDelta::Type::TRANSACTION_COMMIT); @@ -180,8 +180,8 @@ TEST_F(DistributedDurability, WriteCommittedTx) { TEST_F(DistributedDurability, WriteAbortedTx) { RestartWithWal(); - database::GraphDbAccessor dba(master()); - dba.Abort(); + auto dba = master().Access(); + dba->Abort(); FlushAllWal(); CheckDeltas(tmp_dir_ / durability::kWalDir, database::StateDelta::Type::TRANSACTION_ABORT); diff --git a/tests/unit/distributed_dynamic_graph_partitioner.cpp b/tests/unit/distributed_dynamic_graph_partitioner.cpp index c5a8f865e..962252380 100644 --- a/tests/unit/distributed_dynamic_graph_partitioner.cpp +++ b/tests/unit/distributed_dynamic_graph_partitioner.cpp @@ -32,8 +32,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, CountLabels) { for (int i = 0; i < 6; ++i) InsertEdge(vc, va, "edge"); DynamicGraphPartitioner dgp(&master()); - GraphDbAccessor dba(master()); - VertexAccessor v(va, dba); + auto dba = master().Access(); + VertexAccessor v(va, *dba); auto count_labels = dgp.CountLabels(v); // Self loops counted twice @@ -53,8 +53,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMoveVertex) { for (int i = 0; i < 100; ++i) InsertEdge(va, vb, "edge"); DynamicGraphPartitioner dgp(&master()); - GraphDbAccessor dba(master()); - auto migrations = dgp.FindMigrations(dba); + auto dba = master().Access(); + auto migrations = dgp.FindMigrations(*dba); // Expect `va` to try to move to another worker, the one connected to it ASSERT_EQ(migrations.size(), 1); EXPECT_EQ(migrations[0].second, worker(1).WorkerId()); @@ -68,8 +68,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsNoChange) { // Everything is balanced, there should be no movement DynamicGraphPartitioner dgp(&master()); - GraphDbAccessor dba(master()); - auto migrations = dgp.FindMigrations(dba); + auto dba = master().Access(); + auto migrations = dgp.FindMigrations(*dba); EXPECT_EQ(migrations.size(), 0); } @@ -86,9 +86,9 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMultipleAndLimit) { for (int i = 0; i < 100; ++i) InsertEdge(va, vc, "edge"); for (int i = 0; i < 100; ++i) InsertEdge(vb, vc, "edge"); DynamicGraphPartitioner dgp(&master()); - GraphDbAccessor dba(master()); + auto dba = master().Access(); { - auto migrations = dgp.FindMigrations(dba); + auto migrations = dgp.FindMigrations(*dba); // Expect vertices to try to move to another worker ASSERT_EQ(migrations.size(), 2); } @@ -96,7 +96,7 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMultipleAndLimit) { // See if flag affects number of returned results { FLAGS_dgp_max_batch_size = 1; - auto migrations = dgp.FindMigrations(dba); + auto migrations = dgp.FindMigrations(*dba); // Expect vertices to try to move to another worker ASSERT_EQ(migrations.size(), 1); } @@ -151,8 +151,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, Run) { return cnt; }; - GraphDbAccessor dba_m(master()); - GraphDbAccessor dba_w1(worker(1)); - EXPECT_EQ(CountRemotes(dba_m), 50); - EXPECT_EQ(CountRemotes(dba_w1), 50); + auto dba_m = master().Access(); + auto dba_w1 = worker(1).Access(); + EXPECT_EQ(CountRemotes(*dba_m), 50); + EXPECT_EQ(CountRemotes(*dba_w1), 50); } diff --git a/tests/unit/distributed_gc.cpp b/tests/unit/distributed_gc.cpp index 44c742727..7d0e4188b 100644 --- a/tests/unit/distributed_gc.cpp +++ b/tests/unit/distributed_gc.cpp @@ -8,13 +8,13 @@ class DistributedGcTest : public DistributedGraphDbTest { }; TEST_F(DistributedGcTest, GarbageCollect) { - database::GraphDbAccessor dba{master()}; - auto tx = dba.transaction_id(); - dba.Commit(); + auto dba = master().Access(); + auto tx = dba->transaction_id(); + dba->Commit(); // Create multiple transactions so that the commit log can be cleared for (int i = 0; i < tx::CommitLog::kBitsetBlockSize; ++i) { - database::GraphDbAccessor dba{master()}; + auto dba = master().Access(); } master().CollectGarbage(); @@ -22,9 +22,9 @@ TEST_F(DistributedGcTest, GarbageCollect) { worker(2).CollectGarbage(); EXPECT_EQ(master().tx_engine().Info(tx).is_committed(), true); - database::GraphDbAccessor dba2{master()}; - auto tx_last = dba2.transaction_id(); - dba2.Commit(); + auto dba2 = master().Access(); + auto tx_last = dba2->transaction_id(); + dba2->Commit(); worker(1).CollectGarbage(); worker(2).CollectGarbage(); @@ -39,17 +39,17 @@ TEST_F(DistributedGcTest, GarbageCollect) { } TEST_F(DistributedGcTest, GarbageCollectBlocked) { - database::GraphDbAccessor dba{master()}; - auto tx = dba.transaction_id(); - dba.Commit(); + auto dba = master().Access(); + auto tx = dba->transaction_id(); + dba->Commit(); // Block garbage collection because this is a still alive transaction on the // worker - database::GraphDbAccessor dba3{worker(1)}; + auto dba3 = worker(1).Access(); // Create multiple transactions so that the commit log can be cleared for (int i = 0; i < tx::CommitLog::kBitsetBlockSize; ++i) { - database::GraphDbAccessor dba{master()}; + auto dba = master().Access(); } // Query for a large id so that the commit log new block is created @@ -60,9 +60,9 @@ TEST_F(DistributedGcTest, GarbageCollectBlocked) { worker(2).CollectGarbage(); EXPECT_EQ(master().tx_engine().Info(tx).is_committed(), true); - database::GraphDbAccessor dba2{master()}; - auto tx_last = dba2.transaction_id(); - dba2.Commit(); + auto dba2 = master().Access(); + auto tx_last = dba2->transaction_id(); + dba2->Commit(); worker(1).CollectGarbage(); worker(2).CollectGarbage(); diff --git a/tests/unit/distributed_graph_db.cpp b/tests/unit/distributed_graph_db.cpp index 039df4290..464119e45 100644 --- a/tests/unit/distributed_graph_db.cpp +++ b/tests/unit/distributed_graph_db.cpp @@ -49,8 +49,8 @@ TEST_F(DistributedGraphDb, Coordination) { } TEST_F(DistributedGraphDb, TxEngine) { - auto *tx1 = master_tx_engine().Begin(); - auto *tx2 = master_tx_engine().Begin(); + auto *tx1 = master().tx_engine().Begin(); + auto *tx2 = master().tx_engine().Begin(); EXPECT_EQ(tx2->snapshot().size(), 1); EXPECT_EQ( worker(1).tx_engine().RunningTransaction(tx1->id_)->snapshot().size(), 0); @@ -134,43 +134,43 @@ TEST_F(DistributedGraphDb, BuildIndexDistributed) { storage::Property property; { - GraphDbAccessor dba0{master()}; - label = dba0.Label("label"); - property = dba0.Property("property"); - auto tx_id = dba0.transaction_id(); + auto dba0 = master().Access(); + label = dba0->Label("label"); + property = dba0->Property("property"); + auto tx_id = dba0->transaction_id(); - GraphDbAccessor dba1{worker(1), tx_id}; - GraphDbAccessor dba2{worker(2), tx_id}; + auto dba1 = worker(1).Access(tx_id); + auto dba2 = worker(2).Access(tx_id); auto add_vertex = [label, property](GraphDbAccessor &dba) { auto vertex = dba.InsertVertex(); vertex.add_label(label); vertex.PropsSet(property, 1); }; - for (int i = 0; i < 100; ++i) add_vertex(dba0); - for (int i = 0; i < 50; ++i) add_vertex(dba1); - for (int i = 0; i < 300; ++i) add_vertex(dba2); - dba0.Commit(); + for (int i = 0; i < 100; ++i) add_vertex(*dba0); + for (int i = 0; i < 50; ++i) add_vertex(*dba1); + for (int i = 0; i < 300; ++i) add_vertex(*dba2); + dba0->Commit(); } { - GraphDbAccessor dba{master()}; - dba.BuildIndex(label, property); - EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property)); - EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 100); + auto dba = master().Access(); + dba->BuildIndex(label, property); + EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property)); + EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 100); } - GraphDbAccessor dba_master{master()}; + auto dba_master = master().Access(); { - GraphDbAccessor dba{worker(1), dba_master.transaction_id()}; - EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property)); - EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 50); + auto dba = worker(1).Access(dba_master->transaction_id()); + EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property)); + EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 50); } { - GraphDbAccessor dba{worker(2), dba_master.transaction_id()}; - EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property)); - EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 300); + auto dba = worker(2).Access(dba_master->transaction_id()); + EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property)); + EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 300); } } @@ -178,9 +178,9 @@ TEST_F(DistributedGraphDb, BuildIndexConcurrentInsert) { storage::Label label; storage::Property property; - GraphDbAccessor dba0{master()}; - label = dba0.Label("label"); - property = dba0.Property("property"); + auto dba0 = master().Access(); + label = dba0->Label("label"); + property = dba0->Property("property"); int cnt = 0; auto add_vertex = [label, property, &cnt](GraphDbAccessor &dba) { @@ -188,40 +188,40 @@ TEST_F(DistributedGraphDb, BuildIndexConcurrentInsert) { vertex.add_label(label); vertex.PropsSet(property, ++cnt); }; - dba0.Commit(); + dba0->Commit(); auto worker_insert = std::thread([this, &add_vertex]() { for (int i = 0; i < 10000; ++i) { - GraphDbAccessor dba1{worker(1)}; - add_vertex(dba1); - dba1.Commit(); + auto dba1 = worker(1).Access(); + add_vertex(*dba1); + dba1->Commit(); } }); std::this_thread::sleep_for(0.5s); { - GraphDbAccessor dba{master()}; - dba.BuildIndex(label, property); - EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property)); + auto dba = master().Access(); + dba->BuildIndex(label, property); + EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property)); } worker_insert.join(); { - GraphDbAccessor dba{worker(1)}; - EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property)); - EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 10000); + auto dba = worker(1).Access(); + EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property)); + EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 10000); } } TEST_F(DistributedGraphDb, WorkerOwnedDbAccessors) { - GraphDbAccessor dba_w1(worker(1)); - auto v = dba_w1.InsertVertex(); - auto prop = dba_w1.Property("p"); + auto dba_w1 = worker(1).Access(); + auto v = dba_w1->InsertVertex(); + auto prop = dba_w1->Property("p"); v.PropsSet(prop, 42); auto v_ga = v.GlobalAddress(); - dba_w1.Commit(); + dba_w1->Commit(); - GraphDbAccessor dba_w2(worker(2)); - VertexAccessor v_in_w2{v_ga, dba_w2}; + auto dba_w2 = worker(2).Access(); + VertexAccessor v_in_w2{v_ga, *dba_w2}; EXPECT_EQ(v_in_w2.PropsAt(prop).Value<int64_t>(), 42); } diff --git a/tests/unit/distributed_interpretation.cpp b/tests/unit/distributed_interpretation.cpp index beea9794a..dabac4d39 100644 --- a/tests/unit/distributed_interpretation.cpp +++ b/tests/unit/distributed_interpretation.cpp @@ -48,9 +48,9 @@ class DistributedInterpretationTest : public DistributedGraphDbTest { } auto Run(const std::string &query) { - GraphDbAccessor dba(master()); - auto results = RunWithDba(query, dba); - dba.Commit(); + auto dba = master().Access(); + auto results = RunWithDba(query, *dba); + dba->Commit(); return results; } @@ -164,20 +164,20 @@ TEST_F(DistributedInterpretationTest, Cartesian) { // Create some data on the master and both workers. storage::Property prop; { - GraphDbAccessor dba{master()}; - auto tx_id = dba.transaction_id(); - GraphDbAccessor dba1{worker(1), tx_id}; - GraphDbAccessor dba2{worker(2), tx_id}; - prop = dba.Property("prop"); + auto dba = master().Access(); + auto tx_id = dba->transaction_id(); + auto dba1 = worker(1).Access(tx_id); + auto dba2 = worker(2).Access(tx_id); + prop = dba->Property("prop"); auto add_data = [prop](GraphDbAccessor &dba, int value) { dba.InsertVertex().PropsSet(prop, value); }; - for (int i = 0; i < 10; ++i) add_data(dba, i); - for (int i = 10; i < 20; ++i) add_data(dba1, i); - for (int i = 20; i < 30; ++i) add_data(dba2, i); + for (int i = 0; i < 10; ++i) add_data(*dba, i); + for (int i = 10; i < 20; ++i) add_data(*dba1, i); + for (int i = 20; i < 30; ++i) add_data(*dba2, i); - dba.Commit(); + dba->Commit(); } std::vector<std::vector<int64_t>> expected; @@ -214,14 +214,14 @@ class TestQueryWaitsOnFutures : public DistributedInterpretationTest { TEST_F(TestQueryWaitsOnFutures, Test) { const int kVertexCount = 10; auto make_fully_connected = [](database::GraphDb &db) { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); std::vector<VertexAccessor> vertices; for (int i = 0; i < kVertexCount; ++i) - vertices.emplace_back(dba.InsertVertex()); - auto et = dba.EdgeType("et"); + vertices.emplace_back(dba->InsertVertex()); + auto et = dba->EdgeType("et"); for (auto &from : vertices) - for (auto &to : vertices) dba.InsertEdge(from, to, et); - dba.Commit(); + for (auto &to : vertices) dba->InsertEdge(from, to, et); + dba->Commit(); }; make_fully_connected(worker(1)); @@ -292,25 +292,25 @@ TEST_F(DistributedInterpretationTest, OngoingProduceKeyTest) { InsertVertex(worker(2)); } - GraphDbAccessor dba(master()); - auto count1 = RunWithDba("MATCH (n) RETURN count(n)", dba); - dba.AdvanceCommand(); - auto count2 = RunWithDba("MATCH (n) RETURN count(n)", dba); + auto dba = master().Access(); + auto count1 = RunWithDba("MATCH (n) RETURN count(n)", *dba); + dba->AdvanceCommand(); + auto count2 = RunWithDba("MATCH (n) RETURN count(n)", *dba); ASSERT_EQ(count1[0][0].ValueInt(), 3 * worker_count); ASSERT_EQ(count2[0][0].ValueInt(), 3 * worker_count); } TEST_F(DistributedInterpretationTest, AdvanceCommandOnWorkers) { - GraphDbAccessor dba(master()); - RunWithDba("UNWIND RANGE(1, 10) as x CREATE (:A {id: x})", dba); - dba.AdvanceCommand(); + auto dba = master().Access(); + RunWithDba("UNWIND RANGE(1, 10) as x CREATE (:A {id: x})", *dba); + dba->AdvanceCommand(); // Advance commands on workers also. auto futures = master().pull_clients().NotifyAllTransactionCommandAdvanced( - dba.transaction_id()); + dba->transaction_id()); for (auto &future : futures) future.wait(); - auto count = RunWithDba("MATCH (n) RETURN count(n)", dba); + auto count = RunWithDba("MATCH (n) RETURN count(n)", *dba); ASSERT_EQ(count[0][0].ValueInt(), 10); } diff --git a/tests/unit/distributed_query_plan.cpp b/tests/unit/distributed_query_plan.cpp index 0ca1da1d2..8cf355c70 100644 --- a/tests/unit/distributed_query_plan.cpp +++ b/tests/unit/distributed_query_plan.cpp @@ -38,8 +38,8 @@ class DistributedQueryPlan : public DistributedGraphDbTest { }; TEST_F(DistributedQueryPlan, PullProduceRpc) { - GraphDbAccessor dba{master()}; - Context ctx{dba}; + auto dba = master().Access(); + Context ctx{*dba}; SymbolGenerator symbol_generator{ctx.symbol_table_}; AstStorage storage; @@ -56,13 +56,13 @@ TEST_F(DistributedQueryPlan, PullProduceRpc) { auto produce = MakeProduce(unwind, x_ne); // Test that the plan works locally. - auto results = CollectProduce(produce.get(), ctx.symbol_table_, dba); + auto results = CollectProduce(produce.get(), ctx.symbol_table_, *dba); ASSERT_EQ(results.size(), 5); const int plan_id = 42; master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_); - tx::CommandId command_id = dba.transaction().cid(); + tx::CommandId command_id = dba->transaction().cid(); Parameters params; std::vector<query::Symbol> symbols{ctx.symbol_table_[*x_ne]}; auto remote_pull = [this, &command_id, ¶ms, &symbols]( @@ -86,17 +86,17 @@ TEST_F(DistributedQueryPlan, PullProduceRpc) { EXPECT_EQ(batch.frames[1][0].ValueInt(), 2); }; - GraphDbAccessor dba_1{master()}; - GraphDbAccessor dba_2{master()}; + auto dba_1 = master().Access(); + auto dba_2 = master().Access(); for (int worker_id : {1, 2}) { // TODO flor, proper test async here. - auto tx1_batch1 = remote_pull(dba_1, worker_id).get(); + auto tx1_batch1 = remote_pull(*dba_1, worker_id).get(); expect_first_batch(tx1_batch1); - auto tx2_batch1 = remote_pull(dba_2, worker_id).get(); + auto tx2_batch1 = remote_pull(*dba_2, worker_id).get(); expect_first_batch(tx2_batch1); - auto tx2_batch2 = remote_pull(dba_2, worker_id).get(); + auto tx2_batch2 = remote_pull(*dba_2, worker_id).get(); expect_second_batch(tx2_batch2); - auto tx1_batch2 = remote_pull(dba_1, worker_id).get(); + auto tx1_batch2 = remote_pull(*dba_1, worker_id).get(); expect_second_batch(tx1_batch2); } } @@ -107,8 +107,8 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) { // sequence ID, so we can check we retrieved all. storage::Property prop; { - GraphDbAccessor dba{master()}; - prop = dba.Property("prop"); + auto dba = master().Access(); + prop = dba->Property("prop"); auto create_data = [prop](GraphDbAccessor &dba, int worker_id) { auto v1 = dba.InsertVertex(); v1.PropsSet(prop, worker_id * 10); @@ -117,16 +117,16 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) { auto e12 = dba.InsertEdge(v1, v2, dba.EdgeType("et")); e12.PropsSet(prop, worker_id * 10 + 2); }; - create_data(dba, 0); - GraphDbAccessor dba_w1{worker(1), dba.transaction_id()}; - create_data(dba_w1, 1); - GraphDbAccessor dba_w2{worker(2), dba.transaction_id()}; - create_data(dba_w2, 2); - dba.Commit(); + create_data(*dba, 0); + auto dba_w1 = worker(1).Access(dba->transaction_id()); + create_data(*dba_w1, 1); + auto dba_w2 = worker(2).Access(dba->transaction_id()); + create_data(*dba_w2, 2); + dba->Commit(); } - GraphDbAccessor dba{master()}; - Context ctx{dba}; + auto dba = master().Access(); + Context ctx{*dba}; SymbolGenerator symbol_generator{ctx.symbol_table_}; AstStorage storage; @@ -175,13 +175,13 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) { }; // Test that the plan works locally. - auto results = CollectProduce(produce.get(), ctx.symbol_table_, dba); + auto results = CollectProduce(produce.get(), ctx.symbol_table_, *dba); check_result(0, results); const int plan_id = 42; master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_); - tx::CommandId command_id = dba.transaction().cid(); + tx::CommandId command_id = dba->transaction().cid(); Parameters params; std::vector<query::Symbol> symbols{ctx.symbol_table_[*return_n_r], ctx.symbol_table_[*return_m], p_sym}; @@ -190,8 +190,8 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) { return master().pull_clients().Pull(&dba, worker_id, plan_id, command_id, params, symbols, 0, false, 3); }; - auto future_w1_results = remote_pull(dba, 1); - auto future_w2_results = remote_pull(dba, 2); + auto future_w1_results = remote_pull(*dba, 1); + auto future_w2_results = remote_pull(*dba, 2); check_result(1, future_w1_results.get().frames); check_result(2, future_w2_results.get().frames); } @@ -204,8 +204,8 @@ TEST_F(DistributedQueryPlan, Synchronize) { // Query: MATCH (n)--(m) SET m.prop = 2 RETURN n.prop // This query ensures that a remote update gets applied and the local stuff // gets reconstructed. - auto &db = master(); - GraphDbAccessor dba{db}; + auto dba_ptr = master().Access(); + auto &dba = *dba_ptr; Context ctx{dba}; SymbolGenerator symbol_generator{ctx.symbol_table_}; AstStorage storage; @@ -252,9 +252,8 @@ TEST_F(DistributedQueryPlan, Synchronize) { TEST_F(DistributedQueryPlan, Create) { // Query: UNWIND range(0, 1000) as x CREATE () - auto &db = master(); - GraphDbAccessor dba{db}; - Context ctx{dba}; + auto dba = master().Access(); + Context ctx{*dba}; SymbolGenerator symbol_generator{ctx.symbol_table_}; AstStorage storage; auto range = FN("range", LITERAL(0), LITERAL(1000)); @@ -264,8 +263,8 @@ TEST_F(DistributedQueryPlan, Create) { ctx.symbol_table_[*node->identifier_] = ctx.symbol_table_.CreateSymbol("n", true); auto create = std::make_shared<query::plan::CreateNode>(unwind, node, true); - PullAll(create, dba, ctx.symbol_table_); - dba.Commit(); + PullAll(create, *dba, ctx.symbol_table_); + dba->Commit(); EXPECT_GT(VertexCount(master()), 200); EXPECT_GT(VertexCount(worker(1)), 200); @@ -276,11 +275,11 @@ TEST_F(DistributedQueryPlan, PullRemoteOrderBy) { // Create some data on the master and both workers. storage::Property prop; { - GraphDbAccessor dba{master()}; - auto tx_id = dba.transaction_id(); - GraphDbAccessor dba1{worker(1), tx_id}; - GraphDbAccessor dba2{worker(2), tx_id}; - prop = dba.Property("prop"); + auto dba = master().Access(); + auto tx_id = dba->transaction_id(); + auto dba1 = worker(1).Access(tx_id); + auto dba2 = worker(2).Access(tx_id); + prop = dba->Property("prop"); auto add_data = [prop](GraphDbAccessor &dba, int value) { dba.InsertVertex().PropsSet(prop, value); }; @@ -289,15 +288,15 @@ TEST_F(DistributedQueryPlan, PullRemoteOrderBy) { for (int i = 0; i < 300; ++i) data.push_back(i); std::random_shuffle(data.begin(), data.end()); - for (int i = 0; i < 100; ++i) add_data(dba, data[i]); - for (int i = 100; i < 200; ++i) add_data(dba1, data[i]); - for (int i = 200; i < 300; ++i) add_data(dba2, data[i]); + for (int i = 0; i < 100; ++i) add_data(*dba, data[i]); + for (int i = 100; i < 200; ++i) add_data(*dba1, data[i]); + for (int i = 200; i < 300; ++i) add_data(*dba2, data[i]); - dba.Commit(); + dba->Commit(); } - auto &db = master(); - GraphDbAccessor dba{db}; + auto dba_ptr = master().Access(); + auto &dba = *dba_ptr; Context ctx{dba}; SymbolGenerator symbol_generator{ctx.symbol_table_}; AstStorage storage; @@ -341,8 +340,8 @@ TEST_F(DistributedTransactionTimeout, Timeout) { InsertVertex(worker(1)); InsertVertex(worker(1)); - GraphDbAccessor dba{master()}; - Context ctx{dba}; + auto dba = master().Access(); + Context ctx{*dba}; SymbolGenerator symbol_generator{ctx.symbol_table_}; AstStorage storage; @@ -356,14 +355,14 @@ TEST_F(DistributedTransactionTimeout, Timeout) { const int plan_id = 42; master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_); - tx::CommandId command_id = dba.transaction().cid(); + tx::CommandId command_id = dba->transaction().cid(); Parameters params; std::vector<query::Symbol> symbols{ctx.symbol_table_[*output]}; auto remote_pull = [this, &command_id, ¶ms, &symbols, &dba]() { return master() .pull_clients() - .Pull(&dba, 1, plan_id, command_id, params, symbols, 0, false, 1) + .Pull(dba.get(), 1, plan_id, command_id, params, symbols, 0, false, 1) .get() .pull_state; }; diff --git a/tests/unit/distributed_reset.cpp b/tests/unit/distributed_reset.cpp index ba93b4e77..0a3dbaf1b 100644 --- a/tests/unit/distributed_reset.cpp +++ b/tests/unit/distributed_reset.cpp @@ -16,10 +16,10 @@ TEST_F(DistributedReset, ResetTest) { auto pull_remote = std::make_shared<query::plan::PullRemote>( once, 42, std::vector<query::Symbol>()); master().plan_dispatcher().DispatchPlan(42, once, symbol_table); - database::GraphDbAccessor dba{master()}; + auto dba = master().Access(); query::Frame frame(0); - query::Context context(dba); - auto pull_remote_cursor = pull_remote->MakeCursor(dba); + query::Context context(*dba); + auto pull_remote_cursor = pull_remote->MakeCursor(*dba); for (int i = 0; i < 3; ++i) { EXPECT_TRUE(pull_remote_cursor->Pull(frame, context)); diff --git a/tests/unit/distributed_updates.cpp b/tests/unit/distributed_updates.cpp index 00a5e314a..423b2b628 100644 --- a/tests/unit/distributed_updates.cpp +++ b/tests/unit/distributed_updates.cpp @@ -24,14 +24,13 @@ class DistributedUpdateTest : public DistributedGraphDbTest { void SetUp() override { DistributedGraphDbTest::SetUp(); - database::GraphDbAccessor dba_tx1{worker(1)}; - auto v = dba_tx1.InsertVertex(); + auto dba_tx1 = worker(1).Access(); + auto v = dba_tx1->InsertVertex(); auto v_ga = v.GlobalAddress(); - dba_tx1.Commit(); + dba_tx1->Commit(); - dba1 = std::make_unique<database::GraphDbAccessor>(worker(1)); - dba2 = std::make_unique<database::GraphDbAccessor>(worker(2), - dba1->transaction_id()); + dba1 = worker(1).Access(); + dba2 = worker(2).Access(dba1->transaction_id()); v1_dba1 = std::make_unique<VertexAccessor>(v_ga, *dba1); v1_dba2 = std::make_unique<VertexAccessor>(v_ga, *dba2); @@ -77,14 +76,14 @@ class DistributedGraphDbSimpleUpdatesTest : public DistributedGraphDbTest { TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertex) { gid::Gid gid; { - database::GraphDbAccessor dba{worker(1)}; - auto v = dba.InsertVertexIntoRemote(2, {}, {}); + auto dba = worker(1).Access(); + auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {}); gid = v.gid(); - dba.Commit(); + dba->Commit(); } { - database::GraphDbAccessor dba{worker(2)}; - auto v = dba.FindVertexOptional(gid, false); + auto dba = worker(2).Access(); + auto v = dba->FindVertexOptional(gid, false); ASSERT_TRUE(v); } } @@ -93,17 +92,17 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithUpdate) { gid::Gid gid; storage::Property prop; { - database::GraphDbAccessor dba{worker(1)}; - auto v = dba.InsertVertexIntoRemote(2, {}, {}); + auto dba = worker(1).Access(); + auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {}); gid = v.gid(); - prop = dba.Property("prop"); + prop = dba->Property("prop"); v.PropsSet(prop, 42); - worker(2).updates_server().Apply(dba.transaction_id()); - dba.Commit(); + worker(2).updates_server().Apply(dba->transaction_id()); + dba->Commit(); } { - database::GraphDbAccessor dba{worker(2)}; - auto v = dba.FindVertexOptional(gid, false); + auto dba = worker(2).Access(); + auto v = dba->FindVertexOptional(gid, false); ASSERT_TRUE(v); EXPECT_EQ(v->PropsAt(prop).Value<int64_t>(), 42); } @@ -115,11 +114,12 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithData) { storage::Label l2; storage::Property prop; { - database::GraphDbAccessor dba{worker(1)}; - l1 = dba.Label("l1"); - l2 = dba.Label("l2"); - prop = dba.Property("prop"); - auto v = dba.InsertVertexIntoRemote(2, {l1, l2}, {{prop, 42}}); + auto dba = worker(1).Access(); + l1 = dba->Label("l1"); + l2 = dba->Label("l2"); + prop = dba->Property("prop"); + auto v = + database::InsertVertexIntoRemote(dba.get(), 2, {l1, l2}, {{prop, 42}}); gid = v.gid(); // Check local visibility before commit. @@ -127,12 +127,12 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithData) { EXPECT_TRUE(v.has_label(l2)); EXPECT_EQ(v.PropsAt(prop).Value<int64_t>(), 42); - worker(2).updates_server().Apply(dba.transaction_id()); - dba.Commit(); + worker(2).updates_server().Apply(dba->transaction_id()); + dba->Commit(); } { - database::GraphDbAccessor dba{worker(2)}; - auto v = dba.FindVertexOptional(gid, false); + auto dba = worker(2).Access(); + auto v = dba->FindVertexOptional(gid, false); ASSERT_TRUE(v); // Check remote data after commit. EXPECT_TRUE(v->has_label(l1)); @@ -148,23 +148,23 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, UpdateVertexRemoteAndLocal) { storage::Label l1; storage::Label l2; { - database::GraphDbAccessor dba{worker(1)}; - auto v = dba.InsertVertex(); + auto dba = worker(1).Access(); + auto v = dba->InsertVertex(); gid = v.gid(); - l1 = dba.Label("label1"); - l2 = dba.Label("label2"); - dba.Commit(); + l1 = dba->Label("label1"); + l2 = dba->Label("label2"); + dba->Commit(); } { - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - auto v_local = dba1.FindVertex(gid, false); - auto v_remote = VertexAccessor(storage::VertexAddress(gid, 1), dba0); + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto v_local = dba1->FindVertex(gid, false); + auto v_remote = VertexAccessor(storage::VertexAddress(gid, 1), *dba0); v_remote.add_label(l2); v_local.add_label(l1); - auto result = worker(1).updates_server().Apply(dba0.transaction_id()); + auto result = worker(1).updates_server().Apply(dba0->transaction_id()); EXPECT_EQ(result, distributed::UpdateResult::DONE); } } @@ -172,20 +172,20 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, UpdateVertexRemoteAndLocal) { TEST_F(DistributedGraphDbSimpleUpdatesTest, AddSameLabelRemoteAndLocal) { auto v_address = InsertVertex(worker(1)); { - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - auto v_local = dba1.FindVertex(v_address.gid(), false); - auto v_remote = VertexAccessor(v_address, dba0); - auto l1 = dba1.Label("label"); + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto v_local = dba1->FindVertex(v_address.gid(), false); + auto v_remote = VertexAccessor(v_address, *dba0); + auto l1 = dba1->Label("label"); v_remote.add_label(l1); v_local.add_label(l1); - worker(1).updates_server().Apply(dba0.transaction_id()); - dba0.Commit(); + worker(1).updates_server().Apply(dba0->transaction_id()); + dba0->Commit(); } { - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - auto v = dba1.FindVertex(v_address.gid(), false); + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto v = dba1->FindVertex(v_address.gid(), false); EXPECT_EQ(v.labels().size(), 1); } } @@ -194,44 +194,44 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, IndexGetsUpdatedRemotely) { storage::VertexAddress v_remote = InsertVertex(worker(1)); storage::Label label; { - database::GraphDbAccessor dba0{master()}; - label = dba0.Label("label"); - VertexAccessor va(v_remote, dba0); + auto dba0 = master().Access(); + label = dba0->Label("label"); + VertexAccessor va(v_remote, *dba0); va.add_label(label); - worker(1).updates_server().Apply(dba0.transaction_id()); - dba0.Commit(); + worker(1).updates_server().Apply(dba0->transaction_id()); + dba0->Commit(); } { - database::GraphDbAccessor dba1{worker(1)}; - auto vertices = dba1.Vertices(label, false); + auto dba1 = worker(1).Access(); + auto vertices = dba1->Vertices(label, false); EXPECT_EQ(std::distance(vertices.begin(), vertices.end()), 1); } } TEST_F(DistributedGraphDbSimpleUpdatesTest, DeleteVertexRemoteCommit) { auto v_address = InsertVertex(worker(1)); - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - auto v_remote = VertexAccessor(v_address, dba0); - dba0.RemoveVertex(v_remote); - EXPECT_TRUE(dba1.FindVertexOptional(v_address.gid(), true)); - EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()), + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto v_remote = VertexAccessor(v_address, *dba0); + dba0->RemoveVertex(v_remote); + EXPECT_TRUE(dba1->FindVertexOptional(v_address.gid(), true)); + EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()), distributed::UpdateResult::DONE); - EXPECT_FALSE(dba1.FindVertexOptional(v_address.gid(), true)); + EXPECT_FALSE(dba1->FindVertexOptional(v_address.gid(), true)); } TEST_F(DistributedGraphDbSimpleUpdatesTest, DeleteVertexRemoteBothDelete) { auto v_address = InsertVertex(worker(1)); { - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - auto v_local = dba1.FindVertex(v_address.gid(), false); - auto v_remote = VertexAccessor(v_address, dba0); - EXPECT_TRUE(dba1.RemoveVertex(v_local)); - EXPECT_TRUE(dba0.RemoveVertex(v_remote)); - EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()), + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto v_local = dba1->FindVertex(v_address.gid(), false); + auto v_remote = VertexAccessor(v_address, *dba0); + EXPECT_TRUE(dba1->RemoveVertex(v_local)); + EXPECT_TRUE(dba0->RemoveVertex(v_remote)); + EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()), distributed::UpdateResult::DONE); - EXPECT_FALSE(dba1.FindVertexOptional(v_address.gid(), true)); + EXPECT_FALSE(dba1->FindVertexOptional(v_address.gid(), true)); } } @@ -240,27 +240,27 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, DeleteVertexRemoteStillConnected) { auto e_address = InsertEdge(v_address, v_address, "edge"); { - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - auto v_remote = VertexAccessor(v_address, dba0); - dba0.RemoveVertex(v_remote); - EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()), + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto v_remote = VertexAccessor(v_address, *dba0); + dba0->RemoveVertex(v_remote); + EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()), distributed::UpdateResult::UNABLE_TO_DELETE_VERTEX_ERROR); - EXPECT_TRUE(dba1.FindVertexOptional(v_address.gid(), true)); + EXPECT_TRUE(dba1->FindVertexOptional(v_address.gid(), true)); } { - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - auto e_local = dba1.FindEdge(e_address.gid(), false); - auto v_local = dba1.FindVertex(v_address.gid(), false); - auto v_remote = VertexAccessor(v_address, dba0); + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto e_local = dba1->FindEdge(e_address.gid(), false); + auto v_local = dba1->FindVertex(v_address.gid(), false); + auto v_remote = VertexAccessor(v_address, *dba0); - dba1.RemoveEdge(e_local); - dba0.RemoveVertex(v_remote); + dba1->RemoveEdge(e_local); + dba0->RemoveVertex(v_remote); - EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()), + EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()), distributed::UpdateResult::DONE); - EXPECT_FALSE(dba1.FindVertexOptional(v_address.gid(), true)); + EXPECT_FALSE(dba1->FindVertexOptional(v_address.gid(), true)); } } @@ -282,28 +282,21 @@ class DistributedDetachDeleteTest : public DistributedGraphDbTest { template <typename TF> void Run(storage::VertexAddress v_address, TF check_func) { for (int i : {0, 1, 2}) { - database::GraphDbAccessor dba0{master()}; - database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()}; - database::GraphDbAccessor dba2{worker(2), dba0.transaction_id()}; + auto dba0 = master().Access(); + auto dba1 = worker(1).Access(dba0->transaction_id()); + auto dba2 = worker(2).Access(dba0->transaction_id()); - std::vector<std::reference_wrapper<database::GraphDbAccessor>> dba; - dba.emplace_back(dba0); - dba.emplace_back(dba1); - dba.emplace_back(dba2); + std::vector<std::reference_wrapper<database::GraphDbAccessor>> dba{ + *dba0, *dba1, *dba2}; + std::vector<database::DistributedGraphDb *> dbs{&master(), &worker(1), + &worker(2)}; auto &accessor = dba[i].get(); auto v_accessor = VertexAccessor(v_address, accessor); accessor.DetachRemoveVertex(v_accessor); - for (auto db_accessor : dba) { - distributed::UpdatesRpcServer *updates_server = nullptr; - auto *db = &db_accessor.get().db(); - if (auto *distributed_db = - dynamic_cast<database::DistributedGraphDb *>(db)) { - updates_server = &distributed_db->updates_server(); - } - ASSERT_TRUE(updates_server); - ASSERT_EQ(updates_server->Apply(dba[0].get().transaction_id()), + for (auto *db : dbs) { + ASSERT_EQ(db->updates_server().Apply(dba[0].get().transaction_id()), distributed::UpdateResult::DONE); } @@ -388,29 +381,29 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest { storage::VertexAddress to_addr) { CHECK(from_addr.is_remote() && to_addr.is_remote()) << "Local address given to CreateEdge"; - database::GraphDbAccessor dba{creator}; - auto edge_type = dba.EdgeType("et"); - VertexAccessor v1{from_addr, dba}; - VertexAccessor v2{to_addr, dba}; - auto edge = dba.InsertEdge(v1, v2, edge_type); + auto dba = creator.Access(); + auto edge_type = dba->EdgeType("et"); + VertexAccessor v1{from_addr, *dba}; + VertexAccessor v2{to_addr, *dba}; + auto edge = dba->InsertEdge(v1, v2, edge_type); e_ga = edge.GlobalAddress(); - for (auto &kv : props) edge.PropsSet(dba.Property(kv.first), kv.second); + for (auto &kv : props) edge.PropsSet(dba->Property(kv.first), kv.second); - master().updates_server().Apply(dba.transaction_id()); - worker(1).updates_server().Apply(dba.transaction_id()); - worker(2).updates_server().Apply(dba.transaction_id()); - dba.Commit(); + master().updates_server().Apply(dba->transaction_id()); + worker(1).updates_server().Apply(dba->transaction_id()); + worker(2).updates_server().Apply(dba->transaction_id()); + dba->Commit(); } void CheckState(database::GraphDb &db, bool edge_is_local, storage::VertexAddress from_addr, storage::VertexAddress to_addr) { - database::GraphDbAccessor dba{db}; + auto dba = db.Access(); // Check edge data. { - EdgeAccessor edge{e_ga, dba}; + EdgeAccessor edge{e_ga, *dba}; EXPECT_EQ(edge.address().is_local(), edge_is_local); EXPECT_EQ(edge.GlobalAddress(), e_ga); auto from = edge.from(); @@ -422,7 +415,7 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest { EXPECT_EQ(edge.Properties().size(), props.size()); for (auto &kv : props) { - auto equality = edge.PropsAt(dba.Property(kv.first)) == + auto equality = edge.PropsAt(dba->Property(kv.first)) == query::TypedValue(kv.second); EXPECT_TRUE(equality.IsBool() && equality.ValueBool()); } @@ -436,7 +429,7 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest { // Check `from` data. { - VertexAccessor from{from_addr, dba}; + VertexAccessor from{from_addr, *dba}; ASSERT_EQ(edges(from.out()).size(), 1); EXPECT_EQ(edges(from.out())[0].GlobalAddress(), e_ga); // In case of cycles we have 1 in the `in` edges. @@ -445,7 +438,7 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest { // Check `to` data. { - VertexAccessor to{to_addr, dba}; + VertexAccessor to{to_addr, *dba}; // In case of cycles we have 1 in the `out` edges. EXPECT_EQ(edges(to.out()).size(), from_addr == to_addr); ASSERT_EQ(edges(to.in()).size(), 1); @@ -505,13 +498,13 @@ class DistributedEdgeRemoveTest : public DistributedGraphDbTest { } void Delete(database::GraphDb &db) { - database::GraphDbAccessor dba{db}; - EdgeAccessor edge{edge_addr, dba}; - dba.RemoveEdge(edge); - master().updates_server().Apply(dba.transaction_id()); - worker(1).updates_server().Apply(dba.transaction_id()); - worker(2).updates_server().Apply(dba.transaction_id()); - dba.Commit(); + auto dba = db.Access(); + EdgeAccessor edge{edge_addr, *dba}; + dba->RemoveEdge(edge); + master().updates_server().Apply(dba->transaction_id()); + worker(1).updates_server().Apply(dba->transaction_id()); + worker(2).updates_server().Apply(dba->transaction_id()); + dba->Commit(); } template <typename TIterable> @@ -526,12 +519,12 @@ class DistributedEdgeRemoveTest : public DistributedGraphDbTest { ASSERT_EQ(EdgeCount(worker(1)), wid == 1); ASSERT_EQ(EdgeCount(worker(2)), wid == 2); - database::GraphDbAccessor dba{master()}; - VertexAccessor from{from_addr, dba}; + auto dba = master().Access(); + VertexAccessor from{from_addr, *dba}; EXPECT_EQ(Size(from.out()), 1); EXPECT_EQ(Size(from.in()), 0); - VertexAccessor to{to_addr, dba}; + VertexAccessor to{to_addr, *dba}; EXPECT_EQ(Size(to.out()), 0); EXPECT_EQ(Size(to.in()), 1); } @@ -541,13 +534,13 @@ class DistributedEdgeRemoveTest : public DistributedGraphDbTest { EXPECT_EQ(EdgeCount(worker(1)), 0); EXPECT_EQ(EdgeCount(worker(2)), 0); - database::GraphDbAccessor dba{master()}; + auto dba = master().Access(); - VertexAccessor from{from_addr, dba}; + VertexAccessor from{from_addr, *dba}; EXPECT_EQ(Size(from.out()), 0); EXPECT_EQ(Size(from.in()), 0); - VertexAccessor to{to_addr, dba}; + VertexAccessor to{to_addr, *dba}; EXPECT_EQ(Size(to.out()), 0); EXPECT_EQ(Size(to.in()), 0); } diff --git a/tests/unit/durability.cpp b/tests/unit/durability.cpp index b342bb652..3ce05d941 100644 --- a/tests/unit/durability.cpp +++ b/tests/unit/durability.cpp @@ -11,6 +11,7 @@ #include "glog/logging.h" #include "gtest/gtest.h" +#include "database/distributed_graph_db.hpp" #include "database/graph_db.hpp" #include "database/graph_db_accessor.hpp" #include "database/state_delta.hpp" @@ -165,12 +166,12 @@ class DbGenerator { /** Checks if the given databases have the same contents (indices, * vertices and edges). */ void CompareDbs(database::GraphDb &a, database::GraphDb &b) { - database::GraphDbAccessor dba_a(a); - database::GraphDbAccessor dba_b(b); + auto dba_a = a.Access(); + auto dba_b = b.Access(); { - auto index_a = dba_a.IndexInfo(); - auto index_b = dba_b.IndexInfo(); + auto index_a = dba_a->IndexInfo(); + auto index_b = dba_b->IndexInfo(); EXPECT_TRUE( index_a.size() == index_b.size() && std::is_permutation(index_a.begin(), index_a.end(), index_b.begin())) @@ -183,8 +184,8 @@ void CompareDbs(database::GraphDb &a, database::GraphDb &b) { std::vector<std::pair<std::string, query::TypedValue>> p1; std::vector<std::pair<std::string, query::TypedValue>> p2; - for (auto x : p1_id) p1.push_back({dba_a.PropertyName(x.first), x.second}); - for (auto x : p2_id) p2.push_back({dba_b.PropertyName(x.first), x.second}); + for (auto x : p1_id) p1.push_back({dba_a->PropertyName(x.first), x.second}); + for (auto x : p2_id) p2.push_back({dba_b->PropertyName(x.first), x.second}); // Don't use a binary predicate which depends on different value getters // semantics for two containers because is_permutation might call the @@ -200,37 +201,37 @@ void CompareDbs(database::GraphDb &a, database::GraphDb &b) { { int vertices_a_count = 0; - for (auto v_a : dba_a.Vertices(false)) { + for (auto v_a : dba_a->Vertices(false)) { vertices_a_count++; - auto v_b = dba_b.FindVertexOptional(v_a.gid(), false); + auto v_b = dba_b->FindVertexOptional(v_a.gid(), false); ASSERT_TRUE(v_b) << "Vertex not found, id: " << v_a.gid(); ASSERT_EQ(v_a.labels().size(), v_b->labels().size()); std::vector<std::string> v_a_labels; std::vector<std::string> v_b_labels; - for (auto x : v_a.labels()) v_a_labels.push_back(dba_a.LabelName(x)); - for (auto x : v_b->labels()) v_b_labels.push_back(dba_b.LabelName(x)); + for (auto x : v_a.labels()) v_a_labels.push_back(dba_a->LabelName(x)); + for (auto x : v_b->labels()) v_b_labels.push_back(dba_b->LabelName(x)); EXPECT_TRUE(std::is_permutation(v_a_labels.begin(), v_a_labels.end(), v_b_labels.begin())); EXPECT_TRUE(is_permutation_props(v_a.Properties(), v_b->Properties())); } - auto vertices_b = dba_b.Vertices(false); + auto vertices_b = dba_b->Vertices(false); EXPECT_EQ(std::distance(vertices_b.begin(), vertices_b.end()), vertices_a_count); } { int edges_a_count = 0; - for (auto e_a : dba_a.Edges(false)) { + for (auto e_a : dba_a->Edges(false)) { edges_a_count++; - auto e_b = dba_b.FindEdgeOptional(e_a.gid(), false); + auto e_b = dba_b->FindEdgeOptional(e_a.gid(), false); ASSERT_TRUE(e_b); ASSERT_TRUE(e_b) << "Edge not found, id: " << e_a.gid(); - EXPECT_EQ(dba_a.EdgeTypeName(e_a.EdgeType()), - dba_b.EdgeTypeName(e_b->EdgeType())); + EXPECT_EQ(dba_a->EdgeTypeName(e_a.EdgeType()), + dba_b->EdgeTypeName(e_b->EdgeType())); EXPECT_EQ(e_a.from().gid(), e_b->from().gid()); EXPECT_EQ(e_a.to().gid(), e_b->to().gid()); EXPECT_TRUE(is_permutation_props(e_a.Properties(), e_b->Properties())); } - auto edges_b = dba_b.Edges(false); + auto edges_b = dba_b->Edges(false); EXPECT_EQ(std::distance(edges_b.begin(), edges_b.end()), edges_a_count); } } @@ -276,9 +277,9 @@ void MakeDb(durability::WriteAheadLog &wal, database::GraphDbAccessor &dba, } void MakeDb(database::GraphDb &db, int scale, std::vector<int> indices = {}) { - database::GraphDbAccessor dba{db}; - MakeDb(db.wal(), dba, scale, indices); - dba.Commit(); + auto dba = db.Access(); + MakeDb(db.wal(), *dba, scale, indices); + dba->Commit(); } class Durability : public ::testing::Test { @@ -303,8 +304,8 @@ class Durability : public ::testing::Test { } void MakeSnapshot(database::GraphDb &db, int snapshot_max_retained = -1) { - database::GraphDbAccessor dba(db); - ASSERT_TRUE(durability::MakeSnapshot(db, dba, durability_dir_, + auto dba = db.Access(); + ASSERT_TRUE(durability::MakeSnapshot(db, *dba, durability_dir_, snapshot_max_retained)); } @@ -330,18 +331,18 @@ TEST_F(Durability, WalEncoding) { auto config = DbConfig(); config.durability_enabled = true; database::SingleNode db{config}; - database::GraphDbAccessor dba(db); - auto v0 = dba.InsertVertex(); + auto dba = db.Access(); + auto v0 = dba->InsertVertex(); ASSERT_EQ(v0.gid(), gid0); - v0.add_label(dba.Label("l0")); - v0.PropsSet(dba.Property("p0"), 42); - auto v1 = dba.InsertVertex(); + v0.add_label(dba->Label("l0")); + v0.PropsSet(dba->Property("p0"), 42); + auto v1 = dba->InsertVertex(); ASSERT_EQ(v1.gid(), gid1); - auto e0 = dba.InsertEdge(v0, v1, dba.EdgeType("et0")); + auto e0 = dba->InsertEdge(v0, v1, dba->EdgeType("et0")); ASSERT_EQ(e0.gid(), gid0); - e0.PropsSet(dba.Property("p0"), std::vector<PropertyValue>{1, 2, 3}); - dba.BuildIndex(dba.Label("l1"), dba.Property("p1")); - dba.Commit(); + e0.PropsSet(dba->Property("p0"), std::vector<PropertyValue>{1, 2, 3}); + dba->BuildIndex(dba->Label("l1"), dba->Property("p1")); + dba->Commit(); db.wal().Flush(); } @@ -394,26 +395,26 @@ TEST_F(Durability, SnapshotEncoding) { auto gid2 = generator.Next(); { database::SingleNode db{DbConfig()}; - database::GraphDbAccessor dba(db); - auto v0 = dba.InsertVertex(); + auto dba = db.Access(); + auto v0 = dba->InsertVertex(); ASSERT_EQ(v0.gid(), gid0); - v0.add_label(dba.Label("l0")); - v0.PropsSet(dba.Property("p0"), 42); - auto v1 = dba.InsertVertex(); + v0.add_label(dba->Label("l0")); + v0.PropsSet(dba->Property("p0"), 42); + auto v1 = dba->InsertVertex(); ASSERT_EQ(v1.gid(), gid1); - v1.add_label(dba.Label("l0")); - v1.add_label(dba.Label("l1")); - auto v2 = dba.InsertVertex(); + v1.add_label(dba->Label("l0")); + v1.add_label(dba->Label("l1")); + auto v2 = dba->InsertVertex(); ASSERT_EQ(v2.gid(), gid2); - v2.PropsSet(dba.Property("p0"), true); - v2.PropsSet(dba.Property("p1"), "Johnny"); - auto e0 = dba.InsertEdge(v0, v1, dba.EdgeType("et0")); + v2.PropsSet(dba->Property("p0"), true); + v2.PropsSet(dba->Property("p1"), "Johnny"); + auto e0 = dba->InsertEdge(v0, v1, dba->EdgeType("et0")); ASSERT_EQ(e0.gid(), gid0); - e0.PropsSet(dba.Property("p0"), std::vector<PropertyValue>{1, 2, 3}); - auto e1 = dba.InsertEdge(v2, v1, dba.EdgeType("et1")); + e0.PropsSet(dba->Property("p0"), std::vector<PropertyValue>{1, 2, 3}); + auto e1 = dba->InsertEdge(v2, v1, dba->EdgeType("et1")); ASSERT_EQ(e1.gid(), gid1); - dba.BuildIndex(dba.Label("l1"), dba.Property("p1")); - dba.Commit(); + dba->BuildIndex(dba->Label("l1"), dba->Property("p1")); + dba->Commit(); MakeSnapshot(db); } @@ -528,9 +529,9 @@ TEST_F(Durability, SnapshotNoVerticesIdRecovery) { // vertices which should make it not change any id after snapshot recovery, // but we still have to make sure that the id for generators is recovered { - database::GraphDbAccessor dba(db); - for (auto vertex : dba.Vertices(false)) dba.RemoveVertex(vertex); - dba.Commit(); + auto dba = db.Access(); + for (auto vertex : dba->Vertices(false)) dba->RemoveVertex(vertex); + dba->Commit(); } MakeSnapshot(db); @@ -634,34 +635,34 @@ TEST_F(Durability, SnapshotAndWalRecoveryAfterComplexTxSituation) { database::SingleNode db{config}; // The first transaction modifies and commits. - database::GraphDbAccessor dba_1{db}; - MakeDb(db.wal(), dba_1, 100); - dba_1.Commit(); + auto dba_1 = db.Access(); + MakeDb(db.wal(), *dba_1, 100); + dba_1->Commit(); // The second transaction will commit after snapshot. - database::GraphDbAccessor dba_2{db}; - MakeDb(db.wal(), dba_2, 100); + auto dba_2 = db.Access(); + MakeDb(db.wal(), *dba_2, 100); // The third transaction modifies and commits. - database::GraphDbAccessor dba_3{db}; - MakeDb(db.wal(), dba_3, 100); - dba_3.Commit(); + auto dba_3 = db.Access(); + MakeDb(db.wal(), *dba_3, 100); + dba_3->Commit(); MakeSnapshot(db); // Snapshooter takes the fourth transaction. - dba_2.Commit(); + dba_2->Commit(); // The fifth transaction starts and commits after snapshot. - database::GraphDbAccessor dba_5{db}; - MakeDb(db.wal(), dba_5, 100); - dba_5.Commit(); + auto dba_5 = db.Access(); + MakeDb(db.wal(), *dba_5, 100); + dba_5->Commit(); // The sixth transaction will not commit at all. - database::GraphDbAccessor dba_6{db}; - MakeDb(db.wal(), dba_6, 100); + auto dba_6 = db.Access(); + MakeDb(db.wal(), *dba_6, 100); auto VisibleVertexCount = [](database::GraphDb &db) { - database::GraphDbAccessor dba{db}; - auto vertices = dba.Vertices(false); + auto dba = db.Access(); + auto vertices = dba->Vertices(false); return std::distance(vertices.begin(), vertices.end()); }; ASSERT_EQ(VisibleVertexCount(db), 400); @@ -758,7 +759,8 @@ TEST_F(Durability, SnapshotOnExit) { TEST_F(Durability, WorkerIdRecovery) { auto config = DbConfig(); config.worker_id = 5; - database::SingleNode db{config}; + config.recovering_cluster_size = 1; + database::Master db{config}; MakeDb(db, 100); MakeSnapshot(db); EXPECT_EQ(DirFiles(snapshot_dir_).size(), 1); @@ -768,12 +770,13 @@ TEST_F(Durability, WorkerIdRecovery) { auto config = DbConfig(); config.worker_id = 5; config.db_recover_on_startup = true; - database::SingleNode recovered{config}; + config.recovering_cluster_size = 1; + database::Master recovered{config}; EXPECT_EQ(recovered.WorkerId(), config.worker_id); CompareDbs(db, recovered); - database::GraphDbAccessor dba(recovered); - EXPECT_NE(dba.VerticesCount(), 0); - EXPECT_NE(dba.EdgesCount(), 0); + auto dba = recovered.Access(); + EXPECT_NE(dba->VerticesCount(), 0); + EXPECT_NE(dba->EdgesCount(), 0); } // WorkerIds are not equal and recovery should fail @@ -781,11 +784,12 @@ TEST_F(Durability, WorkerIdRecovery) { auto config = DbConfig(); config.worker_id = 10; config.db_recover_on_startup = true; - database::SingleNode recovered{config}; + config.recovering_cluster_size = 1; + database::Master recovered{config}; EXPECT_NE(recovered.WorkerId(), db.WorkerId()); - database::GraphDbAccessor dba(recovered); - EXPECT_EQ(dba.VerticesCount(), 0); - EXPECT_EQ(dba.EdgesCount(), 0); + auto dba = recovered.Access(); + EXPECT_EQ(dba->VerticesCount(), 0); + EXPECT_EQ(dba->EdgesCount(), 0); } } @@ -800,9 +804,9 @@ TEST_F(Durability, SequentialRecovery) { }; auto init_db = [](database::GraphDb &db) { - database::GraphDbAccessor dba{db}; - for (int i = 0; i < kNumVertices; ++i) dba.InsertVertex(i); - dba.Commit(); + auto dba = db.Access(); + for (int i = 0; i < kNumVertices; ++i) dba->InsertVertex(i); + dba->Commit(); }; auto run_updates = [&random_int](database::GraphDb &db, @@ -811,15 +815,15 @@ TEST_F(Durability, SequentialRecovery) { for (int i = 0; i < kNumWorkers; ++i) { threads.emplace_back([&random_int, &db, &keep_running]() { while (keep_running) { - database::GraphDbAccessor dba{db}; - auto v = dba.FindVertex(random_int(kNumVertices), false); + auto dba = db.Access(); + auto v = dba->FindVertex(random_int(kNumVertices), false); try { - v.PropsSet(dba.Property("prop"), random_int(100)); + v.PropsSet(dba->Property("prop"), random_int(100)); } catch (utils::LockTimeoutException &) { } catch (mvcc::SerializationError &) { } - dba.InsertVertex(); - dba.Commit(); + dba->InsertVertex(); + dba->Commit(); } }); } diff --git a/tests/unit/graph_db.cpp b/tests/unit/graph_db.cpp index 085b072e1..25dbadd23 100644 --- a/tests/unit/graph_db.cpp +++ b/tests/unit/graph_db.cpp @@ -11,12 +11,11 @@ TEST(GraphDbTest, GarbageCollectIndices) { database::Config config; config.gc_cycle_sec = -1; database::SingleNode graph_db{config}; - std::unique_ptr<database::GraphDbAccessor> dba = - std::make_unique<database::GraphDbAccessor>(graph_db); + std::unique_ptr<database::GraphDbAccessor> dba = graph_db.Access(); auto commit = [&] { dba->Commit(); - dba = std::make_unique<database::GraphDbAccessor>(graph_db); + dba = graph_db.Access(); }; auto label = dba->Label("label"); auto property = dba->Property("property"); diff --git a/tests/unit/graph_db_accessor.cpp b/tests/unit/graph_db_accessor.cpp index 151ae6ef2..f11dcce45 100644 --- a/tests/unit/graph_db_accessor.cpp +++ b/tests/unit/graph_db_accessor.cpp @@ -17,22 +17,22 @@ auto Count(TIterable iterable) { TEST(GraphDbAccessorTest, InsertVertex) { SingleNode db; - GraphDbAccessor accessor(db); + auto accessor = db.Access(); gid::Generator generator(0); - EXPECT_EQ(Count(accessor.Vertices(false)), 0); + EXPECT_EQ(Count(accessor->Vertices(false)), 0); - EXPECT_EQ(accessor.InsertVertex().gid(), generator.Next()); - EXPECT_EQ(Count(accessor.Vertices(false)), 0); - EXPECT_EQ(Count(accessor.Vertices(true)), 1); - accessor.AdvanceCommand(); - EXPECT_EQ(Count(accessor.Vertices(false)), 1); + EXPECT_EQ(accessor->InsertVertex().gid(), generator.Next()); + EXPECT_EQ(Count(accessor->Vertices(false)), 0); + EXPECT_EQ(Count(accessor->Vertices(true)), 1); + accessor->AdvanceCommand(); + EXPECT_EQ(Count(accessor->Vertices(false)), 1); - EXPECT_EQ(accessor.InsertVertex().gid(), generator.Next()); - EXPECT_EQ(Count(accessor.Vertices(false)), 1); - EXPECT_EQ(Count(accessor.Vertices(true)), 2); - accessor.AdvanceCommand(); - EXPECT_EQ(Count(accessor.Vertices(false)), 2); + EXPECT_EQ(accessor->InsertVertex().gid(), generator.Next()); + EXPECT_EQ(Count(accessor->Vertices(false)), 1); + EXPECT_EQ(Count(accessor->Vertices(true)), 2); + accessor->AdvanceCommand(); + EXPECT_EQ(Count(accessor->Vertices(false)), 2); } TEST(GraphDbAccessorTest, UniqueVertexId) { @@ -42,9 +42,9 @@ TEST(GraphDbAccessorTest, UniqueVertexId) { std::vector<std::thread> threads; for (int i = 0; i < 50; i++) { threads.emplace_back([&db, &ids]() { - GraphDbAccessor dba(db); + auto dba = db.Access(); auto access = ids.access(); - for (int i = 0; i < 200; i++) access.insert(dba.InsertVertex().gid()); + for (int i = 0; i < 200; i++) access.insert(dba->InsertVertex().gid()); }); } @@ -54,66 +54,66 @@ TEST(GraphDbAccessorTest, UniqueVertexId) { TEST(GraphDbAccessorTest, RemoveVertexSameTransaction) { SingleNode db; - GraphDbAccessor accessor(db); + auto accessor = db.Access(); - EXPECT_EQ(Count(accessor.Vertices(false)), 0); + EXPECT_EQ(Count(accessor->Vertices(false)), 0); - auto va1 = accessor.InsertVertex(); - accessor.AdvanceCommand(); - EXPECT_EQ(Count(accessor.Vertices(false)), 1); + auto va1 = accessor->InsertVertex(); + accessor->AdvanceCommand(); + EXPECT_EQ(Count(accessor->Vertices(false)), 1); - EXPECT_TRUE(accessor.RemoveVertex(va1)); - EXPECT_EQ(Count(accessor.Vertices(false)), 1); - EXPECT_EQ(Count(accessor.Vertices(true)), 0); - accessor.AdvanceCommand(); - EXPECT_EQ(Count(accessor.Vertices(false)), 0); - EXPECT_EQ(Count(accessor.Vertices(true)), 0); + EXPECT_TRUE(accessor->RemoveVertex(va1)); + EXPECT_EQ(Count(accessor->Vertices(false)), 1); + EXPECT_EQ(Count(accessor->Vertices(true)), 0); + accessor->AdvanceCommand(); + EXPECT_EQ(Count(accessor->Vertices(false)), 0); + EXPECT_EQ(Count(accessor->Vertices(true)), 0); } TEST(GraphDbAccessorTest, RemoveVertexDifferentTransaction) { SingleNode db; // first transaction creates a vertex { - GraphDbAccessor accessor(db); - accessor.InsertVertex(); - accessor.Commit(); + auto accessor = db.Access(); + accessor->InsertVertex(); + accessor->Commit(); } // second transaction checks that it sees it, and deletes it { - GraphDbAccessor accessor(db); - EXPECT_EQ(Count(accessor.Vertices(false)), 1); - EXPECT_EQ(Count(accessor.Vertices(true)), 1); - for (auto vertex_accessor : accessor.Vertices(false)) - accessor.RemoveVertex(vertex_accessor); - accessor.Commit(); + auto accessor = db.Access(); + EXPECT_EQ(Count(accessor->Vertices(false)), 1); + EXPECT_EQ(Count(accessor->Vertices(true)), 1); + for (auto vertex_accessor : accessor->Vertices(false)) + accessor->RemoveVertex(vertex_accessor); + accessor->Commit(); } // third transaction checks that it does not see the vertex { - GraphDbAccessor accessor(db); - EXPECT_EQ(Count(accessor.Vertices(false)), 0); - EXPECT_EQ(Count(accessor.Vertices(true)), 0); + auto accessor = db.Access(); + EXPECT_EQ(Count(accessor->Vertices(false)), 0); + EXPECT_EQ(Count(accessor->Vertices(true)), 0); } } TEST(GraphDbAccessorTest, InsertEdge) { SingleNode db; - GraphDbAccessor dba(db); + auto dba = db.Access(); - auto va1 = dba.InsertVertex(); - auto va2 = dba.InsertVertex(); - dba.AdvanceCommand(); + auto va1 = dba->InsertVertex(); + auto va2 = dba->InsertVertex(); + dba->AdvanceCommand(); EXPECT_EQ(va1.in_degree(), 0); EXPECT_EQ(va1.out_degree(), 0); EXPECT_EQ(va2.in_degree(), 0); EXPECT_EQ(va2.out_degree(), 0); // setup (v1) - [:likes] -> (v2) - dba.InsertEdge(va1, va2, dba.EdgeType("likes")); - EXPECT_EQ(Count(dba.Edges(false)), 0); - EXPECT_EQ(Count(dba.Edges(true)), 1); - dba.AdvanceCommand(); - EXPECT_EQ(Count(dba.Edges(false)), 1); - EXPECT_EQ(Count(dba.Edges(true)), 1); + dba->InsertEdge(va1, va2, dba->EdgeType("likes")); + EXPECT_EQ(Count(dba->Edges(false)), 0); + EXPECT_EQ(Count(dba->Edges(true)), 1); + dba->AdvanceCommand(); + EXPECT_EQ(Count(dba->Edges(false)), 1); + EXPECT_EQ(Count(dba->Edges(true)), 1); EXPECT_EQ(va1.out().begin()->to(), va2); EXPECT_EQ(va2.in().begin()->from(), va1); EXPECT_EQ(va1.in_degree(), 0); @@ -122,12 +122,12 @@ TEST(GraphDbAccessorTest, InsertEdge) { EXPECT_EQ(va2.out_degree(), 0); // setup (v1) - [:likes] -> (v2) <- [:hates] - (v3) - auto va3 = dba.InsertVertex(); - dba.InsertEdge(va3, va2, dba.EdgeType("hates")); - EXPECT_EQ(Count(dba.Edges(false)), 1); - EXPECT_EQ(Count(dba.Edges(true)), 2); - dba.AdvanceCommand(); - EXPECT_EQ(Count(dba.Edges(false)), 2); + auto va3 = dba->InsertVertex(); + dba->InsertEdge(va3, va2, dba->EdgeType("hates")); + EXPECT_EQ(Count(dba->Edges(false)), 1); + EXPECT_EQ(Count(dba->Edges(true)), 2); + dba->AdvanceCommand(); + EXPECT_EQ(Count(dba->Edges(false)), 2); EXPECT_EQ(va3.out().begin()->to(), va2); EXPECT_EQ(va1.in_degree(), 0); EXPECT_EQ(va1.out_degree(), 1); @@ -144,13 +144,13 @@ TEST(GraphDbAccessorTest, UniqueEdgeId) { std::vector<std::thread> threads; for (int i = 0; i < 50; i++) { threads.emplace_back([&db, &ids]() { - GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto edge_type = dba.EdgeType("edge_type"); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto edge_type = dba->EdgeType("edge_type"); auto access = ids.access(); for (int i = 0; i < 200; i++) - access.insert(dba.InsertEdge(v1, v2, edge_type).gid()); + access.insert(dba->InsertEdge(v1, v2, edge_type).gid()); }); } @@ -160,37 +160,37 @@ TEST(GraphDbAccessorTest, UniqueEdgeId) { TEST(GraphDbAccessorTest, RemoveEdge) { SingleNode db; - GraphDbAccessor dba(db); + auto dba = db.Access(); // setup (v1) - [:likes] -> (v2) <- [:hates] - (v3) - auto va1 = dba.InsertVertex(); - auto va2 = dba.InsertVertex(); - auto va3 = dba.InsertVertex(); - dba.InsertEdge(va1, va2, dba.EdgeType("likes")); - dba.InsertEdge(va3, va2, dba.EdgeType("hates")); - dba.AdvanceCommand(); - EXPECT_EQ(Count(dba.Edges(false)), 2); - EXPECT_EQ(Count(dba.Edges(true)), 2); + auto va1 = dba->InsertVertex(); + auto va2 = dba->InsertVertex(); + auto va3 = dba->InsertVertex(); + dba->InsertEdge(va1, va2, dba->EdgeType("likes")); + dba->InsertEdge(va3, va2, dba->EdgeType("hates")); + dba->AdvanceCommand(); + EXPECT_EQ(Count(dba->Edges(false)), 2); + EXPECT_EQ(Count(dba->Edges(true)), 2); // remove all [:hates] edges - for (auto edge : dba.Edges(false)) - if (edge.EdgeType() == dba.EdgeType("hates")) dba.RemoveEdge(edge); - EXPECT_EQ(Count(dba.Edges(false)), 2); - EXPECT_EQ(Count(dba.Edges(true)), 1); + for (auto edge : dba->Edges(false)) + if (edge.EdgeType() == dba->EdgeType("hates")) dba->RemoveEdge(edge); + EXPECT_EQ(Count(dba->Edges(false)), 2); + EXPECT_EQ(Count(dba->Edges(true)), 1); // current state: (v1) - [:likes] -> (v2), (v3) - dba.AdvanceCommand(); - EXPECT_EQ(Count(dba.Edges(false)), 1); - EXPECT_EQ(Count(dba.Edges(true)), 1); - EXPECT_EQ(Count(dba.Vertices(false)), 3); - EXPECT_EQ(Count(dba.Vertices(true)), 3); - for (auto edge : dba.Edges(false)) { - EXPECT_EQ(edge.EdgeType(), dba.EdgeType("likes")); + dba->AdvanceCommand(); + EXPECT_EQ(Count(dba->Edges(false)), 1); + EXPECT_EQ(Count(dba->Edges(true)), 1); + EXPECT_EQ(Count(dba->Vertices(false)), 3); + EXPECT_EQ(Count(dba->Vertices(true)), 3); + for (auto edge : dba->Edges(false)) { + EXPECT_EQ(edge.EdgeType(), dba->EdgeType("likes")); auto v1 = edge.from(); auto v2 = edge.to(); // ensure correct connectivity for all the vertices - for (auto vertex : dba.Vertices(false)) { + for (auto vertex : dba->Vertices(false)) { if (vertex == v1) { EXPECT_EQ(vertex.in_degree(), 0); EXPECT_EQ(vertex.out_degree(), 1); @@ -207,73 +207,73 @@ TEST(GraphDbAccessorTest, RemoveEdge) { TEST(GraphDbAccessorTest, DetachRemoveVertex) { SingleNode db; - GraphDbAccessor dba(db); + auto dba = db.Access(); // setup (v0)- []->(v1)<-[]-(v2)<-[]-(v3) std::vector<VertexAccessor> vertices; - for (int i = 0; i < 4; ++i) vertices.emplace_back(dba.InsertVertex()); + for (int i = 0; i < 4; ++i) vertices.emplace_back(dba->InsertVertex()); - auto edge_type = dba.EdgeType("type"); - dba.InsertEdge(vertices[0], vertices[1], edge_type); - dba.InsertEdge(vertices[2], vertices[1], edge_type); - dba.InsertEdge(vertices[3], vertices[2], edge_type); + auto edge_type = dba->EdgeType("type"); + dba->InsertEdge(vertices[0], vertices[1], edge_type); + dba->InsertEdge(vertices[2], vertices[1], edge_type); + dba->InsertEdge(vertices[3], vertices[2], edge_type); - dba.AdvanceCommand(); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); // ensure that plain remove does NOT work - EXPECT_EQ(Count(dba.Vertices(false)), 4); - EXPECT_EQ(Count(dba.Edges(false)), 3); - EXPECT_FALSE(dba.RemoveVertex(vertices[0])); - EXPECT_FALSE(dba.RemoveVertex(vertices[1])); - EXPECT_FALSE(dba.RemoveVertex(vertices[2])); - EXPECT_EQ(Count(dba.Vertices(false)), 4); - EXPECT_EQ(Count(dba.Edges(false)), 3); + EXPECT_EQ(Count(dba->Vertices(false)), 4); + EXPECT_EQ(Count(dba->Edges(false)), 3); + EXPECT_FALSE(dba->RemoveVertex(vertices[0])); + EXPECT_FALSE(dba->RemoveVertex(vertices[1])); + EXPECT_FALSE(dba->RemoveVertex(vertices[2])); + EXPECT_EQ(Count(dba->Vertices(false)), 4); + EXPECT_EQ(Count(dba->Edges(false)), 3); - dba.DetachRemoveVertex(vertices[2]); - EXPECT_EQ(Count(dba.Vertices(false)), 4); - EXPECT_EQ(Count(dba.Vertices(true)), 3); - EXPECT_EQ(Count(dba.Edges(false)), 3); - EXPECT_EQ(Count(dba.Edges(true)), 1); - dba.AdvanceCommand(); + dba->DetachRemoveVertex(vertices[2]); + EXPECT_EQ(Count(dba->Vertices(false)), 4); + EXPECT_EQ(Count(dba->Vertices(true)), 3); + EXPECT_EQ(Count(dba->Edges(false)), 3); + EXPECT_EQ(Count(dba->Edges(true)), 1); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), 3); - EXPECT_EQ(Count(dba.Edges(false)), 1); - EXPECT_TRUE(dba.RemoveVertex(vertices[3])); - EXPECT_EQ(Count(dba.Vertices(true)), 2); - EXPECT_EQ(Count(dba.Vertices(false)), 3); - dba.AdvanceCommand(); + EXPECT_EQ(Count(dba->Vertices(false)), 3); + EXPECT_EQ(Count(dba->Edges(false)), 1); + EXPECT_TRUE(dba->RemoveVertex(vertices[3])); + EXPECT_EQ(Count(dba->Vertices(true)), 2); + EXPECT_EQ(Count(dba->Vertices(false)), 3); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), 2); - EXPECT_EQ(Count(dba.Edges(false)), 1); - for (auto va : dba.Vertices(false)) EXPECT_FALSE(dba.RemoveVertex(va)); - dba.AdvanceCommand(); + EXPECT_EQ(Count(dba->Vertices(false)), 2); + EXPECT_EQ(Count(dba->Edges(false)), 1); + for (auto va : dba->Vertices(false)) EXPECT_FALSE(dba->RemoveVertex(va)); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), 2); - EXPECT_EQ(Count(dba.Edges(false)), 1); - for (auto va : dba.Vertices(false)) { - EXPECT_FALSE(dba.RemoveVertex(va)); - dba.DetachRemoveVertex(va); + EXPECT_EQ(Count(dba->Vertices(false)), 2); + EXPECT_EQ(Count(dba->Edges(false)), 1); + for (auto va : dba->Vertices(false)) { + EXPECT_FALSE(dba->RemoveVertex(va)); + dba->DetachRemoveVertex(va); break; } - EXPECT_EQ(Count(dba.Vertices(true)), 1); - EXPECT_EQ(Count(dba.Vertices(false)), 2); - dba.AdvanceCommand(); + EXPECT_EQ(Count(dba->Vertices(true)), 1); + EXPECT_EQ(Count(dba->Vertices(false)), 2); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), 1); - EXPECT_EQ(Count(dba.Edges(false)), 0); + EXPECT_EQ(Count(dba->Vertices(false)), 1); + EXPECT_EQ(Count(dba->Edges(false)), 0); // remove the last vertex, it has no connections // so that should work - for (auto va : dba.Vertices(false)) EXPECT_TRUE(dba.RemoveVertex(va)); - dba.AdvanceCommand(); + for (auto va : dba->Vertices(false)) EXPECT_TRUE(dba->RemoveVertex(va)); + dba->AdvanceCommand(); - EXPECT_EQ(Count(dba.Vertices(false)), 0); - EXPECT_EQ(Count(dba.Edges(false)), 0); + EXPECT_EQ(Count(dba->Vertices(false)), 0); + EXPECT_EQ(Count(dba->Edges(false)), 0); } TEST(GraphDbAccessorTest, DetachRemoveVertexMultiple) { @@ -281,113 +281,113 @@ TEST(GraphDbAccessorTest, DetachRemoveVertexMultiple) { // same vertex / edge multiple times SingleNode db; - GraphDbAccessor dba(db); + auto dba = db.Access(); // setup: make a fully connected N graph // with cycles too! int N = 7; std::vector<VertexAccessor> vertices; - auto edge_type = dba.EdgeType("edge"); - for (int i = 0; i < N; ++i) vertices.emplace_back(dba.InsertVertex()); + auto edge_type = dba->EdgeType("edge"); + for (int i = 0; i < N; ++i) vertices.emplace_back(dba->InsertVertex()); for (int j = 0; j < N; ++j) for (int k = 0; k < N; ++k) - dba.InsertEdge(vertices[j], vertices[k], edge_type); + dba->InsertEdge(vertices[j], vertices[k], edge_type); - dba.AdvanceCommand(); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), N); - EXPECT_EQ(Count(dba.Edges(false)), N * N); + EXPECT_EQ(Count(dba->Vertices(false)), N); + EXPECT_EQ(Count(dba->Edges(false)), N * N); // detach delete one edge - dba.DetachRemoveVertex(vertices[0]); - dba.AdvanceCommand(); + dba->DetachRemoveVertex(vertices[0]); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), N - 1); - EXPECT_EQ(Count(dba.Edges(false)), (N - 1) * (N - 1)); + EXPECT_EQ(Count(dba->Vertices(false)), N - 1); + EXPECT_EQ(Count(dba->Edges(false)), (N - 1) * (N - 1)); // detach delete two neighboring edges - dba.DetachRemoveVertex(vertices[1]); - dba.DetachRemoveVertex(vertices[2]); - dba.AdvanceCommand(); + dba->DetachRemoveVertex(vertices[1]); + dba->DetachRemoveVertex(vertices[2]); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), N - 3); - EXPECT_EQ(Count(dba.Edges(false)), (N - 3) * (N - 3)); + EXPECT_EQ(Count(dba->Vertices(false)), N - 3); + EXPECT_EQ(Count(dba->Edges(false)), (N - 3) * (N - 3)); // detach delete everything, buwahahahaha - for (int l = 3; l < N; ++l) dba.DetachRemoveVertex(vertices[l]); - dba.AdvanceCommand(); + for (int l = 3; l < N; ++l) dba->DetachRemoveVertex(vertices[l]); + dba->AdvanceCommand(); for (auto &vertex : vertices) vertex.Reconstruct(); - EXPECT_EQ(Count(dba.Vertices(false)), 0); - EXPECT_EQ(Count(dba.Edges(false)), 0); + EXPECT_EQ(Count(dba->Vertices(false)), 0); + EXPECT_EQ(Count(dba->Edges(false)), 0); } TEST(GraphDbAccessorTest, Labels) { SingleNode db; - GraphDbAccessor dba(db); + auto dba = db.Access(); - Label label_friend = dba.Label("friend"); - EXPECT_EQ(label_friend, dba.Label("friend")); - EXPECT_NE(label_friend, dba.Label("friend2")); - EXPECT_EQ(dba.LabelName(label_friend), "friend"); + Label label_friend = dba->Label("friend"); + EXPECT_EQ(label_friend, dba->Label("friend")); + EXPECT_NE(label_friend, dba->Label("friend2")); + EXPECT_EQ(dba->LabelName(label_friend), "friend"); // test that getting labels through a different accessor works - EXPECT_EQ(label_friend, GraphDbAccessor(db).Label("friend")); - EXPECT_NE(label_friend, GraphDbAccessor(db).Label("friend2")); + EXPECT_EQ(label_friend, db.Access()->Label("friend")); + EXPECT_NE(label_friend, db.Access()->Label("friend2")); } TEST(GraphDbAccessorTest, EdgeTypes) { SingleNode db; - GraphDbAccessor dba(db); + auto dba = db.Access(); - EdgeType edge_type = dba.EdgeType("likes"); - EXPECT_EQ(edge_type, dba.EdgeType("likes")); - EXPECT_NE(edge_type, dba.EdgeType("hates")); - EXPECT_EQ(dba.EdgeTypeName(edge_type), "likes"); + EdgeType edge_type = dba->EdgeType("likes"); + EXPECT_EQ(edge_type, dba->EdgeType("likes")); + EXPECT_NE(edge_type, dba->EdgeType("hates")); + EXPECT_EQ(dba->EdgeTypeName(edge_type), "likes"); // test that getting labels through a different accessor works - EXPECT_EQ(edge_type, GraphDbAccessor(db).EdgeType("likes")); - EXPECT_NE(edge_type, GraphDbAccessor(db).EdgeType("hates")); + EXPECT_EQ(edge_type, db.Access()->EdgeType("likes")); + EXPECT_NE(edge_type, db.Access()->EdgeType("hates")); } TEST(GraphDbAccessorTest, Properties) { SingleNode db; - GraphDbAccessor dba(db); + auto dba = db.Access(); - Property prop = dba.Property("name"); - EXPECT_EQ(prop, dba.Property("name")); - EXPECT_NE(prop, dba.Property("surname")); - EXPECT_EQ(dba.PropertyName(prop), "name"); + Property prop = dba->Property("name"); + EXPECT_EQ(prop, dba->Property("name")); + EXPECT_NE(prop, dba->Property("surname")); + EXPECT_EQ(dba->PropertyName(prop), "name"); // test that getting labels through a different accessor works - EXPECT_EQ(prop, GraphDbAccessor(db).Property("name")); - EXPECT_NE(prop, GraphDbAccessor(db).Property("surname")); + EXPECT_EQ(prop, db.Access()->Property("name")); + EXPECT_NE(prop, db.Access()->Property("surname")); } TEST(GraphDbAccessorTest, Transfer) { SingleNode db; - GraphDbAccessor dba1(db); - auto prop = dba1.Property("property"); - VertexAccessor v1 = dba1.InsertVertex(); + auto dba1 = db.Access(); + auto prop = dba1->Property("property"); + VertexAccessor v1 = dba1->InsertVertex(); v1.PropsSet(prop, 1); - VertexAccessor v2 = dba1.InsertVertex(); + VertexAccessor v2 = dba1->InsertVertex(); v2.PropsSet(prop, 2); - EdgeAccessor e12 = dba1.InsertEdge(v1, v2, dba1.EdgeType("et")); + EdgeAccessor e12 = dba1->InsertEdge(v1, v2, dba1->EdgeType("et")); e12.PropsSet(prop, 12); // make dba2 that has dba1 in it's snapshot, so data isn't visible - GraphDbAccessor dba2(db); - EXPECT_EQ(dba2.Transfer(v1), std::experimental::nullopt); - EXPECT_EQ(dba2.Transfer(e12), std::experimental::nullopt); + auto dba2 = db.Access(); + EXPECT_EQ(dba2->Transfer(v1), std::experimental::nullopt); + EXPECT_EQ(dba2->Transfer(e12), std::experimental::nullopt); // make dba3 that does not have dba1 in it's snapshot - dba1.Commit(); - GraphDbAccessor dba3(db); + dba1->Commit(); + auto dba3 = db.Access(); // we can transfer accessors even though the GraphDbAccessor they // belong to is not alive anymore - EXPECT_EQ(dba3.Transfer(v1)->PropsAt(prop).Value<int64_t>(), 1); - EXPECT_EQ(dba3.Transfer(e12)->PropsAt(prop).Value<int64_t>(), 12); + EXPECT_EQ(dba3->Transfer(v1)->PropsAt(prop).Value<int64_t>(), 1); + EXPECT_EQ(dba3->Transfer(e12)->PropsAt(prop).Value<int64_t>(), 12); } int main(int argc, char **argv) { diff --git a/tests/unit/graph_db_accessor_index_api.cpp b/tests/unit/graph_db_accessor_index_api.cpp index bb910865b..8619f57c3 100644 --- a/tests/unit/graph_db_accessor_index_api.cpp +++ b/tests/unit/graph_db_accessor_index_api.cpp @@ -23,7 +23,7 @@ auto Count(TIterable iterable) { class GraphDbAccessorIndex : public testing::Test { protected: database::SingleNode db; - std::experimental::optional<database::GraphDbAccessor> dba{db}; + std::unique_ptr<database::GraphDbAccessor> dba{db.Access()}; storage::Property property = dba->Property("property"); storage::Label label = dba->Label("label"); storage::EdgeType edge_type = dba->EdgeType("edge_type"); @@ -44,7 +44,7 @@ class GraphDbAccessorIndex : public testing::Test { // commits the current dba, and replaces it with a new one void Commit() { dba->Commit(); - dba.emplace(db); + dba = db.Access(); } }; @@ -142,9 +142,9 @@ TEST(GraphDbAccessorIndexApi, LabelPropertyBuildIndexConcurrent) { std::vector<std::thread> threads; for (int index = 0; index < THREAD_COUNT; ++index) { threads.emplace_back([&db, index]() { - database::GraphDbAccessor dba(db); - dba.BuildIndex(dba.Label("l" + std::to_string(index)), - dba.Property("p" + std::to_string(index))); + auto dba = db.Access(); + dba->BuildIndex(dba->Label("l" + std::to_string(index)), + dba->Property("p" + std::to_string(index))); }); } diff --git a/tests/unit/interpreter.cpp b/tests/unit/interpreter.cpp index 678ababcd..b342aa2ec 100644 --- a/tests/unit/interpreter.cpp +++ b/tests/unit/interpreter.cpp @@ -19,9 +19,9 @@ class InterpreterTest : public ::testing::Test { auto Interpret(const std::string &query, const std::map<std::string, query::TypedValue> ¶ms = {}) { - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); ResultStreamFaker<query::TypedValue> stream; - auto results = interpreter_(query, dba, params, false); + auto results = interpreter_(query, *dba, params, false); stream.Header(results.header()); results.PullAll(stream); stream.Summary(results.summary()); @@ -146,19 +146,19 @@ TEST_F(InterpreterTest, Bfs) { // Set up. { - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); auto add_node = [&](int level, bool reachable) { - auto node = dba.InsertVertex(); - node.PropsSet(dba.Property(kId), id++); - node.PropsSet(dba.Property(kReachable), reachable); + auto node = dba->InsertVertex(); + node.PropsSet(dba->Property(kId), id++); + node.PropsSet(dba->Property(kReachable), reachable); levels[level].push_back(node); return node; }; auto add_edge = [&](VertexAccessor &v1, VertexAccessor &v2, bool reachable) { - auto edge = dba.InsertEdge(v1, v2, dba.EdgeType("edge")); - edge.PropsSet(dba.Property(kReachable), reachable); + auto edge = dba->InsertEdge(v1, v2, dba->EdgeType("edge")); + edge.PropsSet(dba->Property(kReachable), reachable); }; // Add source node. @@ -196,15 +196,15 @@ TEST_F(InterpreterTest, Bfs) { add_edge(node1, node2, false); } - dba.Commit(); + dba->Commit(); } - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); ResultStreamFaker<query::TypedValue> stream; auto results = interpreter_( "MATCH (n {id: 0})-[r *bfs..5 (e, n | n.reachable and " "e.reachable)]->(m) RETURN r", - dba, {}, false); + *dba, {}, false); stream.Header(results.header()); results.PullAll(stream); stream.Summary(results.summary()); @@ -225,14 +225,14 @@ TEST_F(InterpreterTest, Bfs) { EXPECT_EQ(edges.size(), expected_level); // Check that starting node is correct. EXPECT_EQ( - edges[0].from().PropsAt(dba.Property(kId)).template Value<int64_t>(), + edges[0].from().PropsAt(dba->Property(kId)).template Value<int64_t>(), 0); for (int i = 1; i < static_cast<int>(edges.size()); ++i) { // Check that edges form a connected path. EXPECT_EQ(edges[i - 1].to(), edges[i].from()); } auto matched_id = - edges.back().to().PropsAt(dba.Property(kId)).Value<int64_t>(); + edges.back().to().PropsAt(dba->Property(kId)).Value<int64_t>(); // Check that we didn't match that node already. EXPECT_TRUE(matched_ids.insert(matched_id).second); // Check that shortest path was found. @@ -247,9 +247,9 @@ TEST_F(InterpreterTest, Bfs) { TEST_F(InterpreterTest, CreateIndexInMulticommandTransaction) { ResultStreamFaker<query::TypedValue> stream; - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); ASSERT_THROW( - interpreter_("CREATE INDEX ON :X(y)", dba, {}, true).PullAll(stream), + interpreter_("CREATE INDEX ON :X(y)", *dba, {}, true).PullAll(stream), query::IndexInMulticommandTxException); } @@ -257,20 +257,21 @@ TEST_F(InterpreterTest, CreateIndexInMulticommandTransaction) { TEST_F(InterpreterTest, ShortestPath) { { ResultStreamFaker<query::TypedValue> stream; - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); interpreter_( "CREATE (n:A {x: 1}), (m:B {x: 2}), (l:C {x: 1}), (n)-[:r1 {w: 1 " "}]->(m)-[:r2 {w: 2}]->(l), (n)-[:r3 {w: 4}]->(l)", - dba, {}, true) + *dba, {}, true) .PullAll(stream); - dba.Commit(); + dba->Commit(); } ResultStreamFaker<query::TypedValue> stream; - database::GraphDbAccessor dba(db_); - auto results = interpreter_( - "MATCH (n)-[e *wshortest 5 (e, n | e.w) ]->(m) return e", dba, {}, false); + auto dba = db_.Access(); + auto results = + interpreter_("MATCH (n)-[e *wshortest 5 (e, n | e.w) ]->(m) return e", + *dba, {}, false); stream.Header(results.header()); results.PullAll(stream); stream.Summary(results.summary()); @@ -288,7 +289,7 @@ TEST_F(InterpreterTest, ShortestPath) { std::vector<std::string> datum; for (const auto &edge : edges) { - datum.push_back(dba.EdgeTypeName(edge.EdgeType())); + datum.push_back(dba->EdgeTypeName(edge.EdgeType())); } bool any_match = false; diff --git a/tests/unit/query_common.hpp b/tests/unit/query_common.hpp index 497774a37..277dc77f5 100644 --- a/tests/unit/query_common.hpp +++ b/tests/unit/query_common.hpp @@ -5,9 +5,10 @@ /// /// AstStorage storage; // Macros rely on storage being in scope. /// // PROPERTY_LOOKUP and PROPERTY_PAIR macros -/// // rely on a DbAccessor named dba. +/// // rely on a DbAccessor *reference* named dba. /// database::SingleNode db; -/// database::GraphDbAccessor dba(db); +/// auto dba_ptr = db.Access(); +/// auto &dba = *dba_ptr; /// /// QUERY(MATCH(PATTERN(NODE("n"), EDGE("e"), NODE("m"))), /// WHERE(LESS(PROPERTY_LOOKUP("e", edge_prop), LITERAL(3))), diff --git a/tests/unit/query_cost_estimator.cpp b/tests/unit/query_cost_estimator.cpp index 82b7d896f..910315b30 100644 --- a/tests/unit/query_cost_estimator.cpp +++ b/tests/unit/query_cost_estimator.cpp @@ -24,7 +24,7 @@ using MiscParam = CostEstimator<database::GraphDbAccessor>::MiscParam; class QueryCostEstimator : public ::testing::Test { protected: database::SingleNode db; - std::experimental::optional<database::GraphDbAccessor> dba{db}; + std::unique_ptr<database::GraphDbAccessor> dba{db.Access()}; storage::Label label = dba->Label("label"); storage::Property property = dba->Property("property"); @@ -40,7 +40,7 @@ class QueryCostEstimator : public ::testing::Test { void SetUp() { // create the index in the current db accessor and then swap it to a new one dba->BuildIndex(label, property); - dba.emplace(db); + dba = db.Access(); } Symbol NextSymbol() { diff --git a/tests/unit/query_expression_evaluator.cpp b/tests/unit/query_expression_evaluator.cpp index 2999df82b..efc69b08f 100644 --- a/tests/unit/query_expression_evaluator.cpp +++ b/tests/unit/query_expression_evaluator.cpp @@ -31,8 +31,8 @@ struct NoContextExpressionEvaluator { NoContextExpressionEvaluator() {} Frame frame{128}; database::SingleNode db; - database::GraphDbAccessor dba{db}; - Context ctx{dba}; + std::unique_ptr<database::GraphDbAccessor> dba{db.Access()}; + Context ctx{*dba}; ExpressionEvaluator eval{frame, &ctx, GraphView::OLD}; }; @@ -54,8 +54,8 @@ TypedValue EvaluateFunction(const std::string &function_name, TypedValue EvaluateFunction(const std::string &function_name, const std::vector<TypedValue> &args) { database::SingleNode db; - database::GraphDbAccessor dba{db}; - Context ctx{dba}; + auto dba = db.Access(); + Context ctx{*dba}; return EvaluateFunction(function_name, args, &ctx); } @@ -415,7 +415,8 @@ TEST(ExpressionEvaluator, MapIndexing) { AstStorage storage; NoContextExpressionEvaluator eval; database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto *map_literal = storage.Create<MapLiteral>( std::unordered_map<std::pair<std::string, storage::Property>, Expression *>{ @@ -454,7 +455,7 @@ TEST(ExpressionEvaluator, MapIndexing) { TEST(ExpressionEvaluator, VertexAndEdgeIndexing) { AstStorage storage; NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto edge_type = dba.EdgeType("edge_type"); auto prop = dba.Property("prop"); @@ -684,7 +685,8 @@ class ExpressionEvaluatorPropertyLookup : public testing::Test { AstStorage storage; NoContextExpressionEvaluator eval; database::SingleNode db; - database::GraphDbAccessor dba{db}; + std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()}; + database::GraphDbAccessor &dba{*dba_ptr}; std::pair<std::string, storage::Property> prop_age = PROPERTY_PAIR("age"); std::pair<std::string, storage::Property> prop_height = PROPERTY_PAIR("height"); @@ -732,11 +734,11 @@ TEST(ExpressionEvaluator, LabelsTest) { AstStorage storage; NoContextExpressionEvaluator eval; database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - v1.add_label(dba.Label("ANIMAL")); - v1.add_label(dba.Label("DOG")); - v1.add_label(dba.Label("NICE_DOG")); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + v1.add_label(dba->Label("ANIMAL")); + v1.add_label(dba->Label("DOG")); + v1.add_label(dba->Label("NICE_DOG")); auto *identifier = storage.Create<Identifier>("n"); auto node_symbol = eval.ctx.symbol_table_.CreateSymbol("n", true); eval.ctx.symbol_table_[*identifier] = node_symbol; @@ -744,15 +746,15 @@ TEST(ExpressionEvaluator, LabelsTest) { { auto *op = storage.Create<LabelsTest>( identifier, - std::vector<storage::Label>{dba.Label("DOG"), dba.Label("ANIMAL")}); + std::vector<storage::Label>{dba->Label("DOG"), dba->Label("ANIMAL")}); auto value = op->Accept(eval.eval); EXPECT_EQ(value.Value<bool>(), true); } { auto *op = storage.Create<LabelsTest>( identifier, - std::vector<storage::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"), - dba.Label("ANIMAL")}); + std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"), + dba->Label("ANIMAL")}); auto value = op->Accept(eval.eval); EXPECT_EQ(value.Value<bool>(), false); } @@ -760,8 +762,8 @@ TEST(ExpressionEvaluator, LabelsTest) { eval.frame[node_symbol] = TypedValue::Null; auto *op = storage.Create<LabelsTest>( identifier, - std::vector<storage::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"), - dba.Label("ANIMAL")}); + std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"), + dba->Label("ANIMAL")}); auto value = op->Accept(eval.eval); EXPECT_TRUE(value.IsNull()); } @@ -772,8 +774,8 @@ TEST(ExpressionEvaluator, Aggregation) { auto aggr = storage.Create<Aggregation>(storage.Create<PrimitiveLiteral>(42), nullptr, Aggregation::Op::COUNT); database::SingleNode db; - database::GraphDbAccessor dba(db); - Context ctx(dba); + auto dba = db.Access(); + Context ctx(*dba); auto aggr_sym = ctx.symbol_table_.CreateSymbol("aggr", true); ctx.symbol_table_[*aggr] = aggr_sym; Frame frame{ctx.symbol_table_.max_position()}; @@ -815,15 +817,15 @@ TEST(ExpressionEvaluator, FunctionEndNode) { ASSERT_EQ(EvaluateFunction("ENDNODE", {TypedValue::Null}).type(), TypedValue::Type::Null); database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - v1.add_label(dba.Label("label1")); - auto v2 = dba.InsertVertex(); - v2.add_label(dba.Label("label2")); - auto e = dba.InsertEdge(v1, v2, dba.EdgeType("t")); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + v1.add_label(dba->Label("label1")); + auto v2 = dba->InsertVertex(); + v2.add_label(dba->Label("label2")); + auto e = dba->InsertEdge(v1, v2, dba->EdgeType("t")); ASSERT_TRUE(EvaluateFunction("ENDNODE", {e}) .Value<VertexAccessor>() - .has_label(dba.Label("label2"))); + .has_label(dba->Label("label2"))); ASSERT_THROW(EvaluateFunction("ENDNODE", {2}), QueryRuntimeException); } @@ -844,7 +846,7 @@ TEST(ExpressionEvaluator, FunctionProperties) { ASSERT_EQ(EvaluateFunction("PROPERTIES", {TypedValue::Null}).type(), TypedValue::Type::Null); NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto v1 = dba.InsertVertex(); v1.PropsSet(dba.Property("height"), 5); v1.PropsSet(dba.Property("age"), 10); @@ -900,12 +902,12 @@ TEST(ExpressionEvaluator, FunctionSize) { ASSERT_THROW(EvaluateFunction("SIZE", {5}), QueryRuntimeException); database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v0 = dba.InsertVertex(); + auto dba = db.Access(); + auto v0 = dba->InsertVertex(); query::Path path(v0); EXPECT_EQ(EvaluateFunction("SIZE", {path}).ValueInt(), 0); - auto v1 = dba.InsertVertex(); - path.Expand(dba.InsertEdge(v0, v1, dba.EdgeType("type"))); + auto v1 = dba->InsertVertex(); + path.Expand(dba->InsertEdge(v0, v1, dba->EdgeType("type"))); path.Expand(v1); EXPECT_EQ(EvaluateFunction("SIZE", {path}).ValueInt(), 1); } @@ -915,15 +917,15 @@ TEST(ExpressionEvaluator, FunctionStartNode) { ASSERT_EQ(EvaluateFunction("STARTNODE", {TypedValue::Null}).type(), TypedValue::Type::Null); database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - v1.add_label(dba.Label("label1")); - auto v2 = dba.InsertVertex(); - v2.add_label(dba.Label("label2")); - auto e = dba.InsertEdge(v1, v2, dba.EdgeType("t")); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + v1.add_label(dba->Label("label1")); + auto v2 = dba->InsertVertex(); + v2.add_label(dba->Label("label2")); + auto e = dba->InsertEdge(v1, v2, dba->EdgeType("t")); ASSERT_TRUE(EvaluateFunction("STARTNODE", {e}) .Value<VertexAccessor>() - .has_label(dba.Label("label1"))); + .has_label(dba->Label("label1"))); ASSERT_THROW(EvaluateFunction("STARTNODE", {2}), QueryRuntimeException); } @@ -932,12 +934,12 @@ TEST(ExpressionEvaluator, FunctionDegree) { ASSERT_EQ(EvaluateFunction("DEGREE", {TypedValue::Null}).type(), TypedValue::Type::Null); database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto v3 = dba.InsertVertex(); - auto e12 = dba.InsertEdge(v1, v2, dba.EdgeType("t")); - dba.InsertEdge(v3, v2, dba.EdgeType("t")); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto v3 = dba->InsertVertex(); + auto e12 = dba->InsertEdge(v1, v2, dba->EdgeType("t")); + dba->InsertEdge(v3, v2, dba->EdgeType("t")); ASSERT_EQ(EvaluateFunction("DEGREE", {v1}).Value<int64_t>(), 1); ASSERT_EQ(EvaluateFunction("DEGREE", {v2}).Value<int64_t>(), 2); ASSERT_EQ(EvaluateFunction("DEGREE", {v3}).Value<int64_t>(), 1); @@ -992,7 +994,7 @@ TEST(ExpressionEvaluator, FunctionType) { ASSERT_EQ(EvaluateFunction("TYPE", {TypedValue::Null}).type(), TypedValue::Type::Null); NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto v1 = dba.InsertVertex(); v1.add_label(dba.Label("label1")); auto v2 = dba.InsertVertex(); @@ -1008,7 +1010,7 @@ TEST(ExpressionEvaluator, FunctionLabels) { ASSERT_EQ(EvaluateFunction("LABELS", {TypedValue::Null}).type(), TypedValue::Type::Null); NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto v = dba.InsertVertex(); v.add_label(dba.Label("label1")); v.add_label(dba.Label("label2")); @@ -1031,7 +1033,7 @@ TEST(ExpressionEvaluator, FunctionNodesRelationships) { { NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto v1 = dba.InsertVertex(); auto v2 = dba.InsertVertex(); auto v3 = dba.InsertVertex(); @@ -1089,7 +1091,7 @@ TEST(ExpressionEvaluator, FunctionKeys) { ASSERT_EQ(EvaluateFunction("KEYS", {TypedValue::Null}).type(), TypedValue::Type::Null); NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto v1 = dba.InsertVertex(); v1.PropsSet(dba.Property("height"), 5); v1.PropsSet(dba.Property("age"), 10); @@ -1456,7 +1458,7 @@ TEST(ExpressionEvaluator, FunctionIndexInfo) { EXPECT_THROW(EvaluateFunction("INDEXINFO", {1}, &eval.ctx), QueryRuntimeException); EXPECT_EQ(EvaluateFunction("INDEXINFO", {}, &eval.ctx).ValueList().size(), 0); - auto &dba = eval.dba; + auto &dba = *eval.dba; dba.InsertVertex().add_label(dba.Label("l1")); { auto info = @@ -1475,7 +1477,7 @@ TEST(ExpressionEvaluator, FunctionIndexInfo) { TEST(ExpressionEvaluator, FunctionId) { NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto va = dba.InsertVertex(); auto ea = dba.InsertEdge(va, va, dba.EdgeType("edge")); auto vb = dba.InsertVertex(); @@ -1491,7 +1493,7 @@ TEST(ExpressionEvaluator, FunctionId) { TEST(ExpressionEvaluator, FunctionWorkerIdException) { database::SingleNode db; NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto va = dba.InsertVertex(); EXPECT_THROW(EvaluateFunction("WORKERID", {}, &eval.ctx), QueryRuntimeException); @@ -1501,10 +1503,9 @@ TEST(ExpressionEvaluator, FunctionWorkerIdException) { TEST(ExpressionEvaluator, FunctionWorkerIdSingleNode) { NoContextExpressionEvaluator eval; - auto &dba = eval.dba; + auto &dba = *eval.dba; auto va = dba.InsertVertex(); - EXPECT_EQ(EvaluateFunction("WORKERID", {va}, &eval.ctx).Value<int64_t>(), - eval.db.WorkerId()); + EXPECT_EQ(EvaluateFunction("WORKERID", {va}, &eval.ctx).Value<int64_t>(), 0); } TEST(ExpressionEvaluator, FunctionToStringNull) { diff --git a/tests/unit/query_plan_accumulate_aggregate.cpp b/tests/unit/query_plan_accumulate_aggregate.cpp index 36caaae7a..f3158ab12 100644 --- a/tests/unit/query_plan_accumulate_aggregate.cpp +++ b/tests/unit/query_plan_accumulate_aggregate.cpp @@ -28,7 +28,8 @@ TEST(QueryPlan, Accumulate) { auto check = [&](bool accumulate) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto prop = dba.Property("x"); auto v1 = dba.InsertVertex(); @@ -88,7 +89,7 @@ TEST(QueryPlan, AccumulateAdvance) { auto check = [&](bool advance) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -99,7 +100,7 @@ TEST(QueryPlan, AccumulateAdvance) { auto accumulate = std::make_shared<Accumulate>( create, std::vector<Symbol>{sym_n}, advance); auto match = MakeScanAll(storage, symbol_table, "m", accumulate); - EXPECT_EQ(advance ? 1 : 0, PullAll(match.op_, dba, symbol_table)); + EXPECT_EQ(advance ? 1 : 0, PullAll(match.op_, *dba, symbol_table)); }; check(false); check(true); @@ -150,7 +151,8 @@ std::shared_ptr<Produce> MakeAggregationProduce( class QueryPlanAggregateOps : public ::testing::Test { protected: database::SingleNode db; - database::GraphDbAccessor dba{db}; + std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()}; + database::GraphDbAccessor &dba{*dba_ptr}; storage::Property prop = dba.Property("prop"); AstStorage storage; @@ -289,7 +291,8 @@ TEST(QueryPlan, AggregateGroupByValues) { // Also test the "remember" part of the Aggregation API as final results are // obtained via a property lookup of a remembered node. database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // a vector of TypedValue to be set as property values on vertices // most of them should result in a distinct group (commented where not) @@ -348,7 +351,8 @@ TEST(QueryPlan, AggregateMultipleGroupBy) { // for different records and assert that we get the correct combination // of values in our groups database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto prop1 = dba.Property("prop1"); auto prop2 = dba.Property("prop2"); @@ -383,7 +387,7 @@ TEST(QueryPlan, AggregateMultipleGroupBy) { TEST(QueryPlan, AggregateNoInput) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -393,7 +397,7 @@ TEST(QueryPlan, AggregateNoInput) { auto produce = MakeAggregationProduce(nullptr, symbol_table, storage, {two}, {Aggregation::Op::COUNT}, {}, {}); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(1, results.size()); EXPECT_EQ(1, results[0].size()); EXPECT_EQ(TypedValue::Type::Int, results[0][0].type()); @@ -410,7 +414,8 @@ TEST(QueryPlan, AggregateCountEdgeCases) { // - 2 vertices in database, property set on both database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto prop = dba.Property("prop"); AstStorage storage; @@ -462,7 +467,8 @@ TEST(QueryPlan, AggregateFirstValueTypes) { // type check database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto v1 = dba.InsertVertex(); auto prop_string = dba.Property("string"); @@ -519,7 +525,8 @@ TEST(QueryPlan, AggregateTypes) { // (that logic is defined and tested by TypedValue) database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto p1 = dba.Property("p1"); // has only string props dba.InsertVertex().PropsSet(p1, "string"); @@ -575,7 +582,7 @@ TEST(QueryPlan, AggregateTypes) { TEST(QueryPlan, Unwind) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -598,7 +605,7 @@ TEST(QueryPlan, Unwind) { symbol_table[*y_ne] = symbol_table.CreateSymbol("y_ne", true); auto produce = MakeProduce(unwind_1, x_ne, y_ne); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); ASSERT_EQ(4, results.size()); const std::vector<int> expected_x_card{3, 3, 3, 1}; auto expected_x_card_it = expected_x_card.begin(); diff --git a/tests/unit/query_plan_bag_semantics.cpp b/tests/unit/query_plan_bag_semantics.cpp index bf0082e67..78e354179 100644 --- a/tests/unit/query_plan_bag_semantics.cpp +++ b/tests/unit/query_plan_bag_semantics.cpp @@ -23,7 +23,7 @@ using namespace query::plan; TEST(QueryPlan, Skip) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -31,28 +31,28 @@ TEST(QueryPlan, Skip) { auto n = MakeScanAll(storage, symbol_table, "n1"); auto skip = std::make_shared<plan::Skip>(n.op_, LITERAL(2)); - EXPECT_EQ(0, PullAll(skip, dba, symbol_table)); + EXPECT_EQ(0, PullAll(skip, *dba, symbol_table)); - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(0, PullAll(skip, dba, symbol_table)); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(0, PullAll(skip, *dba, symbol_table)); - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(0, PullAll(skip, dba, symbol_table)); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(0, PullAll(skip, *dba, symbol_table)); - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(1, PullAll(skip, dba, symbol_table)); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(1, PullAll(skip, *dba, symbol_table)); - for (int i = 0; i < 10; ++i) dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(11, PullAll(skip, dba, symbol_table)); + for (int i = 0; i < 10; ++i) dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(11, PullAll(skip, *dba, symbol_table)); } TEST(QueryPlan, Limit) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -60,23 +60,23 @@ TEST(QueryPlan, Limit) { auto n = MakeScanAll(storage, symbol_table, "n1"); auto skip = std::make_shared<plan::Limit>(n.op_, LITERAL(2)); - EXPECT_EQ(0, PullAll(skip, dba, symbol_table)); + EXPECT_EQ(0, PullAll(skip, *dba, symbol_table)); - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(1, PullAll(skip, dba, symbol_table)); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(1, PullAll(skip, *dba, symbol_table)); - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(2, PullAll(skip, dba, symbol_table)); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(2, PullAll(skip, *dba, symbol_table)); - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(2, PullAll(skip, dba, symbol_table)); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(2, PullAll(skip, *dba, symbol_table)); - for (int i = 0; i < 10; ++i) dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(2, PullAll(skip, dba, symbol_table)); + for (int i = 0; i < 10; ++i) dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(2, PullAll(skip, *dba, symbol_table)); } TEST(QueryPlan, CreateLimit) { @@ -84,10 +84,10 @@ TEST(QueryPlan, CreateLimit) { // MATCH (n) CREATE (m) LIMIT 1 // in the end we need to have 3 vertices in the db database::SingleNode db; - database::GraphDbAccessor dba(db); - dba.InsertVertex(); - dba.InsertVertex(); - dba.AdvanceCommand(); + auto dba = db.Access(); + dba->InsertVertex(); + dba->InsertVertex(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -98,14 +98,15 @@ TEST(QueryPlan, CreateLimit) { auto c = std::make_shared<CreateNode>(n.op_, m, false); auto skip = std::make_shared<plan::Limit>(c, LITERAL(1)); - EXPECT_EQ(1, PullAll(skip, dba, symbol_table)); - dba.AdvanceCommand(); - EXPECT_EQ(3, CountIterable(dba.Vertices(false))); + EXPECT_EQ(1, PullAll(skip, *dba, symbol_table)); + dba->AdvanceCommand(); + EXPECT_EQ(3, CountIterable(dba->Vertices(false))); } TEST(QueryPlan, OrderBy) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; AstStorage storage; SymbolTable symbol_table; auto prop = dba.Property("prop"); @@ -166,7 +167,8 @@ TEST(QueryPlan, OrderBy) { TEST(QueryPlan, OrderByMultiple) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; AstStorage storage; SymbolTable symbol_table; @@ -202,7 +204,8 @@ TEST(QueryPlan, OrderByMultiple) { auto order_by = std::make_shared<plan::OrderBy>( n.op_, std::vector<std::pair<Ordering, Expression *>>{ - {Ordering::ASC, n_p1}, {Ordering::DESC, n_p2}, + {Ordering::ASC, n_p1}, + {Ordering::DESC, n_p2}, }, std::vector<Symbol>{n.sym_}); auto n_p1_ne = NEXPR("n.p1", n_p1); @@ -222,7 +225,8 @@ TEST(QueryPlan, OrderByMultiple) { TEST(QueryPlan, OrderByExceptions) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; AstStorage storage; SymbolTable symbol_table; auto prop = dba.Property("prop"); diff --git a/tests/unit/query_plan_create_set_remove_delete.cpp b/tests/unit/query_plan_create_set_remove_delete.cpp index 619503258..206753df1 100644 --- a/tests/unit/query_plan_create_set_remove_delete.cpp +++ b/tests/unit/query_plan_create_set_remove_delete.cpp @@ -18,7 +18,8 @@ using namespace query::plan; TEST(QueryPlan, CreateNodeWithAttributes) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; storage::Label label = dba.Label("Person"); auto property = PROPERTY_PAIR("prop"); @@ -52,7 +53,8 @@ TEST(QueryPlan, CreateNodeWithAttributes) { TEST(QueryPlan, CreateReturn) { // test CREATE (n:Person {age: 42}) RETURN n, n.age database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; storage::Label label = dba.Label("Person"); auto property = PROPERTY_PAIR("property"); @@ -93,7 +95,8 @@ TEST(QueryPlan, CreateReturn) { TEST(QueryPlan, CreateExpand) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; storage::Label label_node_1 = dba.Label("Node1"); storage::Label label_node_2 = dba.Label("Node2"); @@ -167,13 +170,13 @@ TEST(QueryPlan, CreateExpand) { TEST(QueryPlan, MatchCreateNode) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // add three nodes we'll match and expand-create from - dba.InsertVertex(); - dba.InsertVertex(); - dba.InsertVertex(); - dba.AdvanceCommand(); + dba->InsertVertex(); + dba->InsertVertex(); + dba->InsertVertex(); + dba->AdvanceCommand(); SymbolTable symbol_table; AstStorage storage; @@ -186,34 +189,34 @@ TEST(QueryPlan, MatchCreateNode) { // creation op auto create_node = std::make_shared<CreateNode>(n_scan_all.op_, m, false); - EXPECT_EQ(CountIterable(dba.Vertices(false)), 3); - PullAll(create_node, dba, symbol_table); - dba.AdvanceCommand(); - EXPECT_EQ(CountIterable(dba.Vertices(false)), 6); + EXPECT_EQ(CountIterable(dba->Vertices(false)), 3); + PullAll(create_node, *dba, symbol_table); + dba->AdvanceCommand(); + EXPECT_EQ(CountIterable(dba->Vertices(false)), 6); } TEST(QueryPlan, MatchCreateExpand) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // add three nodes we'll match and expand-create from - dba.InsertVertex(); - dba.InsertVertex(); - dba.InsertVertex(); - dba.AdvanceCommand(); + dba->InsertVertex(); + dba->InsertVertex(); + dba->InsertVertex(); + dba->AdvanceCommand(); - // storage::Label label_node_1 = dba.Label("Node1"); - // storage::Label label_node_2 = dba.Label("Node2"); - // storage::Property property = dba.Label("prop"); - storage::EdgeType edge_type = dba.EdgeType("edge_type"); + // storage::Label label_node_1 = dba->Label("Node1"); + // storage::Label label_node_2 = dba->Label("Node2"); + // storage::Property property = dba->Label("prop"); + storage::EdgeType edge_type = dba->EdgeType("edge_type"); SymbolTable symbol_table; AstStorage storage; auto test_create_path = [&](bool cycle, int expected_nodes_created, int expected_edges_created) { - int before_v = CountIterable(dba.Vertices(false)); - int before_e = CountIterable(dba.Edges(false)); + int before_v = CountIterable(dba->Vertices(false)); + int before_e = CountIterable(dba->Edges(false)); // data for the first node auto n_scan_all = MakeScanAll(storage, symbol_table, "n"); @@ -231,12 +234,12 @@ TEST(QueryPlan, MatchCreateExpand) { auto create_expand = std::make_shared<CreateExpand>(m, r, n_scan_all.op_, n_scan_all.sym_, cycle); - PullAll(create_expand, dba, symbol_table); - dba.AdvanceCommand(); + PullAll(create_expand, *dba, symbol_table); + dba->AdvanceCommand(); - EXPECT_EQ(CountIterable(dba.Vertices(false)) - before_v, + EXPECT_EQ(CountIterable(dba->Vertices(false)) - before_v, expected_nodes_created); - EXPECT_EQ(CountIterable(dba.Edges(false)) - before_e, + EXPECT_EQ(CountIterable(dba->Edges(false)) - before_e, expected_edges_created); }; @@ -246,19 +249,19 @@ TEST(QueryPlan, MatchCreateExpand) { TEST(QueryPlan, Delete) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // make a fully-connected (one-direction, no cycles) with 4 nodes std::vector<VertexAccessor> vertices; - for (int i = 0; i < 4; ++i) vertices.push_back(dba.InsertVertex()); - auto type = dba.EdgeType("type"); + for (int i = 0; i < 4; ++i) vertices.push_back(dba->InsertVertex()); + auto type = dba->EdgeType("type"); for (int j = 0; j < 4; ++j) for (int k = j + 1; k < 4; ++k) - dba.InsertEdge(vertices[j], vertices[k], type); + dba->InsertEdge(vertices[j], vertices[k], type); - dba.AdvanceCommand(); - EXPECT_EQ(4, CountIterable(dba.Vertices(false))); - EXPECT_EQ(6, CountIterable(dba.Edges(false))); + dba->AdvanceCommand(); + EXPECT_EQ(4, CountIterable(dba->Vertices(false))); + EXPECT_EQ(6, CountIterable(dba->Edges(false))); AstStorage storage; SymbolTable symbol_table; @@ -270,10 +273,10 @@ TEST(QueryPlan, Delete) { symbol_table[*n_get] = n.sym_; auto delete_op = std::make_shared<plan::Delete>( n.op_, std::vector<Expression *>{n_get}, false); - EXPECT_THROW(PullAll(delete_op, dba, symbol_table), QueryRuntimeException); - dba.AdvanceCommand(); - EXPECT_EQ(4, CountIterable(dba.Vertices(false))); - EXPECT_EQ(6, CountIterable(dba.Edges(false))); + EXPECT_THROW(PullAll(delete_op, *dba, symbol_table), QueryRuntimeException); + dba->AdvanceCommand(); + EXPECT_EQ(4, CountIterable(dba->Vertices(false))); + EXPECT_EQ(6, CountIterable(dba->Edges(false))); } // detach delete a single vertex @@ -284,12 +287,12 @@ TEST(QueryPlan, Delete) { auto delete_op = std::make_shared<plan::Delete>( n.op_, std::vector<Expression *>{n_get}, true); Frame frame(symbol_table.max_position()); - Context context(dba); + Context context(*dba); context.symbol_table_ = symbol_table; - delete_op->MakeCursor(dba)->Pull(frame, context); - dba.AdvanceCommand(); - EXPECT_EQ(3, CountIterable(dba.Vertices(false))); - EXPECT_EQ(3, CountIterable(dba.Edges(false))); + delete_op->MakeCursor(*dba)->Pull(frame, context); + dba->AdvanceCommand(); + EXPECT_EQ(3, CountIterable(dba->Vertices(false))); + EXPECT_EQ(3, CountIterable(dba->Edges(false))); } // delete all remaining edges @@ -302,10 +305,10 @@ TEST(QueryPlan, Delete) { symbol_table[*r_get] = r_m.edge_sym_; auto delete_op = std::make_shared<plan::Delete>( r_m.op_, std::vector<Expression *>{r_get}, false); - PullAll(delete_op, dba, symbol_table); - dba.AdvanceCommand(); - EXPECT_EQ(3, CountIterable(dba.Vertices(false))); - EXPECT_EQ(0, CountIterable(dba.Edges(false))); + PullAll(delete_op, *dba, symbol_table); + dba->AdvanceCommand(); + EXPECT_EQ(3, CountIterable(dba->Vertices(false))); + EXPECT_EQ(0, CountIterable(dba->Edges(false))); } // delete all remaining vertices @@ -315,10 +318,10 @@ TEST(QueryPlan, Delete) { symbol_table[*n_get] = n.sym_; auto delete_op = std::make_shared<plan::Delete>( n.op_, std::vector<Expression *>{n_get}, false); - PullAll(delete_op, dba, symbol_table); - dba.AdvanceCommand(); - EXPECT_EQ(0, CountIterable(dba.Vertices(false))); - EXPECT_EQ(0, CountIterable(dba.Edges(false))); + PullAll(delete_op, *dba, symbol_table); + dba->AdvanceCommand(); + EXPECT_EQ(0, CountIterable(dba->Vertices(false))); + EXPECT_EQ(0, CountIterable(dba->Edges(false))); } } @@ -336,14 +339,14 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) { auto test_delete = [](bool detach) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - dba.InsertEdge(v1, v2, dba.EdgeType("T")); - dba.AdvanceCommand(); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); - EXPECT_EQ(1, CountIterable(dba.Edges(false))); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + dba->InsertEdge(v1, v2, dba->EdgeType("T")); + dba->AdvanceCommand(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); + EXPECT_EQ(1, CountIterable(dba->Edges(false))); AstStorage storage; SymbolTable symbol_table; @@ -363,10 +366,10 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) { auto delete_op = std::make_shared<plan::Delete>( r_m.op_, std::vector<Expression *>{n_get, r_get, m_get}, detach); - EXPECT_EQ(2, PullAll(delete_op, dba, symbol_table)); - dba.AdvanceCommand(); - EXPECT_EQ(0, CountIterable(dba.Vertices(false))); - EXPECT_EQ(0, CountIterable(dba.Edges(false))); + EXPECT_EQ(2, PullAll(delete_op, *dba, symbol_table)); + dba->AdvanceCommand(); + EXPECT_EQ(0, CountIterable(dba->Vertices(false))); + EXPECT_EQ(0, CountIterable(dba->Edges(false))); }; test_delete(true); @@ -375,7 +378,8 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) { TEST(QueryPlan, DeleteReturn) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // make a fully-connected (one-direction, no cycles) with 4 nodes auto prop = PROPERTY_PAIR("property"); @@ -414,14 +418,14 @@ TEST(QueryPlan, DeleteReturn) { TEST(QueryPlan, DeleteNull) { // test (simplified) WITH Null as x delete x database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; auto once = std::make_shared<Once>(); auto delete_op = std::make_shared<plan::Delete>( once, std::vector<Expression *>{LITERAL(TypedValue::Null)}, false); - EXPECT_EQ(1, PullAll(delete_op, dba, symbol_table)); + EXPECT_EQ(1, PullAll(delete_op, *dba, symbol_table)); } TEST(QueryPlan, DeleteAdvance) { @@ -435,9 +439,9 @@ TEST(QueryPlan, DeleteAdvance) { // record is not used in subsequent clauses, but // we are not yet compatible with that database::SingleNode db; - database::GraphDbAccessor dba(db); - dba.InsertVertex(); - dba.AdvanceCommand(); + auto dba = db.Access(); + dba->InsertVertex(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -449,12 +453,13 @@ TEST(QueryPlan, DeleteAdvance) { n.op_, std::vector<Expression *>{n_get}, false); auto advance = std::make_shared<Accumulate>( delete_op, std::vector<Symbol>{n.sym_}, true); - EXPECT_THROW(PullAll(advance, dba, symbol_table), ReconstructionException); + EXPECT_THROW(PullAll(advance, *dba, symbol_table), ReconstructionException); } TEST(QueryPlan, SetProperty) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // graph with 4 vertices in connected pairs // the origin vertex in each par and both edges @@ -506,19 +511,19 @@ TEST(QueryPlan, SetProperty) { TEST(QueryPlan, SetProperties) { auto test_set_properties = [](bool update) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // graph: ({a: 0})-[:R {b:1}]->({c:2}) - auto prop_a = dba.Property("a"); - auto prop_b = dba.Property("b"); - auto prop_c = dba.Property("c"); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto e = dba.InsertEdge(v1, v2, dba.EdgeType("R")); + auto prop_a = dba->Property("a"); + auto prop_b = dba->Property("b"); + auto prop_c = dba->Property("c"); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto e = dba->InsertEdge(v1, v2, dba->EdgeType("R")); v1.PropsSet(prop_a, 0); e.PropsSet(prop_b, 1); v2.PropsSet(prop_c, 2); - dba.AdvanceCommand(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -541,11 +546,11 @@ TEST(QueryPlan, SetProperties) { std::make_shared<plan::SetProperties>(r_m.op_, n.sym_, r_ident, op); auto set_m_to_r = std::make_shared<plan::SetProperties>( set_r_to_n, r_m.edge_sym_, m_ident, op); - EXPECT_EQ(1, PullAll(set_m_to_r, dba, symbol_table)); - dba.AdvanceCommand(); + EXPECT_EQ(1, PullAll(set_m_to_r, *dba, symbol_table)); + dba->AdvanceCommand(); - EXPECT_EQ(CountIterable(dba.Edges(false)), 1); - for (EdgeAccessor edge : dba.Edges(false)) { + EXPECT_EQ(CountIterable(dba->Edges(false)), 1); + for (EdgeAccessor edge : dba->Edges(false)) { VertexAccessor from = edge.from(); EXPECT_EQ(from.Properties().size(), update ? 2 : 1); if (update) { @@ -576,14 +581,14 @@ TEST(QueryPlan, SetProperties) { TEST(QueryPlan, SetLabels) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto label1 = dba.Label("label1"); - auto label2 = dba.Label("label2"); - auto label3 = dba.Label("label3"); - dba.InsertVertex().add_label(label1); - dba.InsertVertex().add_label(label1); - dba.AdvanceCommand(); + auto label1 = dba->Label("label1"); + auto label2 = dba->Label("label2"); + auto label3 = dba->Label("label3"); + dba->InsertVertex().add_label(label1); + dba->InsertVertex().add_label(label1); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -591,9 +596,9 @@ TEST(QueryPlan, SetLabels) { auto n = MakeScanAll(storage, symbol_table, "n"); auto label_set = std::make_shared<plan::SetLabels>( n.op_, n.sym_, std::vector<storage::Label>{label2, label3}); - EXPECT_EQ(2, PullAll(label_set, dba, symbol_table)); + EXPECT_EQ(2, PullAll(label_set, *dba, symbol_table)); - for (VertexAccessor vertex : dba.Vertices(false)) { + for (VertexAccessor vertex : dba->Vertices(false)) { vertex.SwitchNew(); EXPECT_EQ(3, vertex.labels().size()); EXPECT_TRUE(vertex.has_label(label2)); @@ -603,7 +608,8 @@ TEST(QueryPlan, SetLabels) { TEST(QueryPlan, RemoveProperty) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // graph with 4 vertices in connected pairs // the origin vertex in each par and both edges @@ -656,19 +662,19 @@ TEST(QueryPlan, RemoveProperty) { TEST(QueryPlan, RemoveLabels) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto label1 = dba.Label("label1"); - auto label2 = dba.Label("label2"); - auto label3 = dba.Label("label3"); - auto v1 = dba.InsertVertex(); + auto label1 = dba->Label("label1"); + auto label2 = dba->Label("label2"); + auto label3 = dba->Label("label3"); + auto v1 = dba->InsertVertex(); v1.add_label(label1); v1.add_label(label2); v1.add_label(label3); - auto v2 = dba.InsertVertex(); + auto v2 = dba->InsertVertex(); v2.add_label(label1); v2.add_label(label3); - dba.AdvanceCommand(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -676,9 +682,9 @@ TEST(QueryPlan, RemoveLabels) { auto n = MakeScanAll(storage, symbol_table, "n"); auto label_remove = std::make_shared<plan::RemoveLabels>( n.op_, n.sym_, std::vector<storage::Label>{label1, label2}); - EXPECT_EQ(2, PullAll(label_remove, dba, symbol_table)); + EXPECT_EQ(2, PullAll(label_remove, *dba, symbol_table)); - for (VertexAccessor vertex : dba.Vertices(false)) { + for (VertexAccessor vertex : dba->Vertices(false)) { vertex.SwitchNew(); EXPECT_EQ(1, vertex.labels().size()); EXPECT_FALSE(vertex.has_label(label1)); @@ -688,7 +694,8 @@ TEST(QueryPlan, RemoveLabels) { TEST(QueryPlan, NodeFilterSet) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // Create a graph such that (v1 {prop: 42}) is connected to v2 and v3. auto v1 = dba.InsertVertex(); auto prop = PROPERTY_PAIR("property"); @@ -729,7 +736,8 @@ TEST(QueryPlan, NodeFilterSet) { TEST(QueryPlan, FilterRemove) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // Create a graph such that (v1 {prop: 42}) is connected to v2 and v3. auto v1 = dba.InsertVertex(); auto prop = PROPERTY_PAIR("property"); @@ -766,11 +774,11 @@ TEST(QueryPlan, FilterRemove) { TEST(QueryPlan, SetRemove) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v = dba.InsertVertex(); - auto label1 = dba.Label("label1"); - auto label2 = dba.Label("label2"); - dba.AdvanceCommand(); + auto dba = db.Access(); + auto v = dba->InsertVertex(); + auto label1 = dba->Label("label1"); + auto label2 = dba->Label("label2"); + dba->AdvanceCommand(); // Create operations which match (v) and set and remove v :label. // The expected result is single (v) as it was at the start. AstStorage storage; @@ -781,8 +789,8 @@ TEST(QueryPlan, SetRemove) { scan_all.op_, scan_all.sym_, std::vector<storage::Label>{label1, label2}); auto rem = std::make_shared<plan::RemoveLabels>( set, scan_all.sym_, std::vector<storage::Label>{label1, label2}); - EXPECT_EQ(1, PullAll(rem, dba, symbol_table)); - dba.AdvanceCommand(); + EXPECT_EQ(1, PullAll(rem, *dba, symbol_table)); + dba->AdvanceCommand(); v.Reconstruct(); EXPECT_FALSE(v.has_label(label1)); EXPECT_FALSE(v.has_label(label2)); @@ -796,7 +804,8 @@ TEST(QueryPlan, Merge) { // and sets some property (for result validation) // - merge_create branch just sets some other property database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto v1 = dba.InsertVertex(); auto v2 = dba.InsertVertex(); dba.InsertEdge(v1, v2, dba.EdgeType("Type")); @@ -842,7 +851,7 @@ TEST(QueryPlan, MergeNoInput) { // merge with no input, creates a single node database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -852,16 +861,17 @@ TEST(QueryPlan, MergeNoInput) { auto create = std::make_shared<CreateNode>(nullptr, node, false); auto merge = std::make_shared<plan::Merge>(nullptr, create, create); - EXPECT_EQ(0, CountIterable(dba.Vertices(false))); - EXPECT_EQ(1, PullAll(merge, dba, symbol_table)); - dba.AdvanceCommand(); - EXPECT_EQ(1, CountIterable(dba.Vertices(false))); + EXPECT_EQ(0, CountIterable(dba->Vertices(false))); + EXPECT_EQ(1, PullAll(merge, *dba, symbol_table)); + dba->AdvanceCommand(); + EXPECT_EQ(1, CountIterable(dba->Vertices(false))); } TEST(QueryPlan, SetPropertyOnNull) { // SET (Null).prop = 42 database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; AstStorage storage; SymbolTable symbol_table; auto prop = PROPERTY_PAIR("property"); @@ -876,7 +886,7 @@ TEST(QueryPlan, SetPropertyOnNull) { TEST(QueryPlan, SetPropertiesOnNull) { // OPTIONAL MATCH (n) SET n = n database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; auto n = MakeScanAll(storage, symbol_table, "n"); @@ -886,15 +896,15 @@ TEST(QueryPlan, SetPropertiesOnNull) { std::vector<Symbol>{n.sym_}); auto set_op = std::make_shared<plan::SetProperties>( optional, n.sym_, n_ident, plan::SetProperties::Op::REPLACE); - EXPECT_EQ(0, CountIterable(dba.Vertices(false))); - EXPECT_EQ(1, PullAll(set_op, dba, symbol_table)); + EXPECT_EQ(0, CountIterable(dba->Vertices(false))); + EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table)); } TEST(QueryPlan, SetLabelsOnNull) { // OPTIONAL MATCH (n) SET n :label database::SingleNode db; - database::GraphDbAccessor dba(db); - auto label = dba.Label("label"); + auto dba = db.Access(); + auto label = dba->Label("label"); AstStorage storage; SymbolTable symbol_table; auto n = MakeScanAll(storage, symbol_table, "n"); @@ -904,14 +914,15 @@ TEST(QueryPlan, SetLabelsOnNull) { std::vector<Symbol>{n.sym_}); auto set_op = std::make_shared<plan::SetLabels>( optional, n.sym_, std::vector<storage::Label>{label}); - EXPECT_EQ(0, CountIterable(dba.Vertices(false))); - EXPECT_EQ(1, PullAll(set_op, dba, symbol_table)); + EXPECT_EQ(0, CountIterable(dba->Vertices(false))); + EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table)); } TEST(QueryPlan, RemovePropertyOnNull) { // REMOVE (Null).prop database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; AstStorage storage; SymbolTable symbol_table; auto prop = PROPERTY_PAIR("property"); @@ -925,8 +936,8 @@ TEST(QueryPlan, RemovePropertyOnNull) { TEST(QueryPlan, RemoveLabelsOnNull) { // OPTIONAL MATCH (n) REMOVE n :label database::SingleNode db; - database::GraphDbAccessor dba(db); - auto label = dba.Label("label"); + auto dba = db.Access(); + auto label = dba->Label("label"); AstStorage storage; SymbolTable symbol_table; auto n = MakeScanAll(storage, symbol_table, "n"); @@ -936,26 +947,27 @@ TEST(QueryPlan, RemoveLabelsOnNull) { std::vector<Symbol>{n.sym_}); auto remove_op = std::make_shared<plan::RemoveLabels>( optional, n.sym_, std::vector<storage::Label>{label}); - EXPECT_EQ(0, CountIterable(dba.Vertices(false))); - EXPECT_EQ(1, PullAll(remove_op, dba, symbol_table)); + EXPECT_EQ(0, CountIterable(dba->Vertices(false))); + EXPECT_EQ(1, PullAll(remove_op, *dba, symbol_table)); } TEST(QueryPlan, CreateIndex) { // CREATE INDEX ON :Label(property) database::SingleNode db; - database::GraphDbAccessor dba(db); - auto label = dba.Label("label"); - auto property = dba.Property("property"); - EXPECT_FALSE(dba.LabelPropertyIndexExists(label, property)); + auto dba = db.Access(); + auto label = dba->Label("label"); + auto property = dba->Property("property"); + EXPECT_FALSE(dba->LabelPropertyIndexExists(label, property)); auto create_index = std::make_shared<plan::CreateIndex>(label, property); SymbolTable symbol_table; - EXPECT_EQ(PullAll(create_index, dba, symbol_table), 1); - EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property)); + EXPECT_EQ(PullAll(create_index, *dba, symbol_table), 1); + EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property)); } TEST(QueryPlan, DeleteSetProperty) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // Add a single vertex. dba.InsertVertex(); dba.AdvanceCommand(); @@ -978,7 +990,8 @@ TEST(QueryPlan, DeleteSetProperty) { TEST(QueryPlan, DeleteSetPropertiesFromMap) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // Add a single vertex. dba.InsertVertex(); dba.AdvanceCommand(); @@ -1009,7 +1022,8 @@ TEST(QueryPlan, DeleteSetPropertiesFromMap) { TEST(QueryPlan, DeleteSetPropertiesFromVertex) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // Add a single vertex. { auto v = dba.InsertVertex(); @@ -1040,11 +1054,11 @@ TEST(QueryPlan, DeleteSetPropertiesFromVertex) { TEST(QueryPlan, DeleteRemoveLabels) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Add a single vertex. - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(1, CountIterable(dba.Vertices(false))); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(1, CountIterable(dba->Vertices(false))); AstStorage storage; SymbolTable symbol_table; // MATCH (n) DELETE n REMOVE n :label @@ -1053,14 +1067,15 @@ TEST(QueryPlan, DeleteRemoveLabels) { symbol_table[*n_get] = n.sym_; auto delete_op = std::make_shared<plan::Delete>( n.op_, std::vector<Expression *>{n_get}, false); - std::vector<storage::Label> labels{dba.Label("label")}; + std::vector<storage::Label> labels{dba->Label("label")}; auto rem_op = std::make_shared<plan::RemoveLabels>(delete_op, n.sym_, labels); - EXPECT_THROW(PullAll(rem_op, dba, symbol_table), QueryRuntimeException); + EXPECT_THROW(PullAll(rem_op, *dba, symbol_table), QueryRuntimeException); } TEST(QueryPlan, DeleteRemoveProperty) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // Add a single vertex. dba.InsertVertex(); dba.AdvanceCommand(); diff --git a/tests/unit/query_plan_edge_cases.cpp b/tests/unit/query_plan_edge_cases.cpp index b0b964a78..41c1aa9da 100644 --- a/tests/unit/query_plan_edge_cases.cpp +++ b/tests/unit/query_plan_edge_cases.cpp @@ -17,15 +17,15 @@ DECLARE_bool(query_cost_planner); class QueryExecution : public testing::Test { protected: std::experimental::optional<database::SingleNode> db_; - std::experimental::optional<database::GraphDbAccessor> dba_; + std::unique_ptr<database::GraphDbAccessor> dba_; void SetUp() { db_.emplace(); - dba_.emplace(*db_); + dba_ = db_->Access(); } void TearDown() { - dba_ = std::experimental::nullopt; + dba_ = nullptr; db_ = std::experimental::nullopt; } @@ -33,7 +33,7 @@ class QueryExecution : public testing::Test { * variable to hold a new accessor with a new transaction */ void Commit() { dba_->Commit(); - dba_.emplace(*db_); + dba_ = db_->Access(); } /** Executes the query and returns the results. diff --git a/tests/unit/query_plan_match_filter_return.cpp b/tests/unit/query_plan_match_filter_return.cpp index bac579336..5d3e80bd0 100644 --- a/tests/unit/query_plan_match_filter_return.cpp +++ b/tests/unit/query_plan_match_filter_return.cpp @@ -28,18 +28,18 @@ using namespace query::plan; class MatchReturnFixture : public testing::Test { protected: database::SingleNode db_; - database::GraphDbAccessor dba_{db_}; + std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()}; AstStorage storage; SymbolTable symbol_table; void AddVertices(int count) { - for (int i = 0; i < count; i++) dba_.InsertVertex(); + for (int i = 0; i < count; i++) dba_->InsertVertex(); } template <typename TResult> std::vector<TResult> Results(std::shared_ptr<Produce> &op) { std::vector<TResult> res; - for (const auto &row : CollectProduce(op.get(), symbol_table, dba_)) + for (const auto &row : CollectProduce(op.get(), symbol_table, *dba_)) res.emplace_back(row[0].Value<TResult>()); return res; } @@ -47,7 +47,7 @@ class MatchReturnFixture : public testing::Test { TEST_F(MatchReturnFixture, MatchReturn) { AddVertices(2); - dba_.AdvanceCommand(); + dba_->AdvanceCommand(); auto test_pull_count = [&](GraphView graph_view) { auto scan_all = @@ -57,21 +57,21 @@ TEST_F(MatchReturnFixture, MatchReturn) { symbol_table[*output->expression_] = scan_all.sym_; symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true); - return PullAll(produce, dba_, symbol_table); + return PullAll(produce, *dba_, symbol_table); }; EXPECT_EQ(2, test_pull_count(GraphView::NEW)); EXPECT_EQ(2, test_pull_count(GraphView::OLD)); - dba_.InsertVertex(); + dba_->InsertVertex(); EXPECT_EQ(3, test_pull_count(GraphView::NEW)); EXPECT_EQ(2, test_pull_count(GraphView::OLD)); - dba_.AdvanceCommand(); + dba_->AdvanceCommand(); EXPECT_EQ(3, test_pull_count(GraphView::OLD)); } TEST_F(MatchReturnFixture, MatchReturnPath) { AddVertices(2); - dba_.AdvanceCommand(); + dba_->AdvanceCommand(); auto scan_all = MakeScanAll(storage, symbol_table, "n", nullptr); Symbol path_sym = symbol_table.CreateSymbol("path", true); @@ -84,7 +84,7 @@ TEST_F(MatchReturnFixture, MatchReturnPath) { auto results = Results<query::Path>(produce); ASSERT_EQ(results.size(), 2); std::vector<query::Path> expected_paths; - for (const auto &v : dba_.Vertices(false)) expected_paths.emplace_back(v); + for (const auto &v : dba_->Vertices(false)) expected_paths.emplace_back(v); ASSERT_EQ(expected_paths.size(), 2); EXPECT_TRUE(std::is_permutation(expected_paths.begin(), expected_paths.end(), results.begin())); @@ -92,11 +92,11 @@ TEST_F(MatchReturnFixture, MatchReturnPath) { TEST(QueryPlan, MatchReturnCartesian) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - dba.InsertVertex().add_label(dba.Label("l1")); - dba.InsertVertex().add_label(dba.Label("l2")); - dba.AdvanceCommand(); + dba->InsertVertex().add_label(dba->Label("l1")); + dba->InsertVertex().add_label(dba->Label("l2")); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -113,7 +113,7 @@ TEST(QueryPlan, MatchReturnCartesian) { symbol_table.CreateSymbol("named_expression_2", true); auto produce = MakeProduce(m.op_, return_n, return_m); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 4); // ensure the result ordering is OK: // "n" from the results is the same for the first two rows, while "m" isn't @@ -125,12 +125,12 @@ TEST(QueryPlan, MatchReturnCartesian) { TEST(QueryPlan, StandaloneReturn) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // add a few nodes to the database - dba.InsertVertex(); - dba.InsertVertex(); - dba.AdvanceCommand(); + dba->InsertVertex(); + dba->InsertVertex(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -139,7 +139,7 @@ TEST(QueryPlan, StandaloneReturn) { auto produce = MakeProduce(std::shared_ptr<LogicalOperator>(nullptr), output); symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 1); EXPECT_EQ(results[0].size(), 1); EXPECT_EQ(results[0][0].Value<int64_t>(), 42); @@ -147,7 +147,8 @@ TEST(QueryPlan, StandaloneReturn) { TEST(QueryPlan, NodeFilterLabelsAndProperties) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // add a few nodes to the database storage::Label label = dba.Label("Label"); @@ -203,28 +204,28 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) { TEST(QueryPlan, NodeFilterMultipleLabels) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // add a few nodes to the database - storage::Label label1 = dba.Label("label1"); - storage::Label label2 = dba.Label("label2"); - storage::Label label3 = dba.Label("label3"); + storage::Label label1 = dba->Label("label1"); + storage::Label label2 = dba->Label("label2"); + storage::Label label3 = dba->Label("label3"); // the test will look for nodes that have label1 and label2 - dba.InsertVertex(); // NOT accepted - dba.InsertVertex().add_label(label1); // NOT accepted - dba.InsertVertex().add_label(label2); // NOT accepted - dba.InsertVertex().add_label(label3); // NOT accepted - auto v1 = dba.InsertVertex(); // YES accepted + dba->InsertVertex(); // NOT accepted + dba->InsertVertex().add_label(label1); // NOT accepted + dba->InsertVertex().add_label(label2); // NOT accepted + dba->InsertVertex().add_label(label3); // NOT accepted + auto v1 = dba->InsertVertex(); // YES accepted v1.add_label(label1); v1.add_label(label2); - auto v2 = dba.InsertVertex(); // NOT accepted + auto v2 = dba->InsertVertex(); // NOT accepted v2.add_label(label1); v2.add_label(label3); - auto v3 = dba.InsertVertex(); // YES accepted + auto v3 = dba->InsertVertex(); // YES accepted v3.add_label(label1); v3.add_label(label2); v3.add_label(label3); - dba.AdvanceCommand(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -247,23 +248,23 @@ TEST(QueryPlan, NodeFilterMultipleLabels) { symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true); symbol_table[*output->expression_] = n.sym_; - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 2); } TEST(QueryPlan, Cartesian) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); auto add_vertex = [&dba](std::string label) { - auto vertex = dba.InsertVertex(); - vertex.add_label(dba.Label(label)); + auto vertex = dba->InsertVertex(); + vertex.add_label(dba->Label(label)); return vertex; }; std::vector<VertexAccessor> vertices{add_vertex("v1"), add_vertex("v2"), add_vertex("v3")}; - dba.AdvanceCommand(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -286,7 +287,7 @@ TEST(QueryPlan, Cartesian) { auto produce = MakeProduce(cartesian_op, return_n, return_m); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 9); for (int i = 0; i < 3; ++i) { for (int j = 0; j < 3; ++j) { @@ -298,7 +299,7 @@ TEST(QueryPlan, Cartesian) { TEST(QueryPlan, CartesianEmptySet) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -321,22 +322,22 @@ TEST(QueryPlan, CartesianEmptySet) { auto produce = MakeProduce(cartesian_op, return_n, return_m); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 0); } TEST(QueryPlan, CartesianThreeWay) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); auto add_vertex = [&dba](std::string label) { - auto vertex = dba.InsertVertex(); - vertex.add_label(dba.Label(label)); + auto vertex = dba->InsertVertex(); + vertex.add_label(dba->Label(label)); return vertex; }; std::vector<VertexAccessor> vertices{add_vertex("v1"), add_vertex("v2"), add_vertex("v3")}; - dba.AdvanceCommand(); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -369,7 +370,7 @@ TEST(QueryPlan, CartesianThreeWay) { auto produce = MakeProduce(cartesian_op_2, return_n, return_m, return_l); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 27); int id = 0; for (int i = 0; i < 3; ++i) { @@ -387,23 +388,23 @@ TEST(QueryPlan, CartesianThreeWay) { class ExpandFixture : public testing::Test { protected: database::SingleNode db_; - database::GraphDbAccessor dba_{db_}; + std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()}; AstStorage storage; SymbolTable symbol_table; // make a V-graph (v3)<-[r2]-(v1)-[r1]->(v2) - VertexAccessor v1 = dba_.InsertVertex(); - VertexAccessor v2 = dba_.InsertVertex(); - VertexAccessor v3 = dba_.InsertVertex(); - storage::EdgeType edge_type = dba_.EdgeType("Edge"); - EdgeAccessor r1 = dba_.InsertEdge(v1, v2, edge_type); - EdgeAccessor r2 = dba_.InsertEdge(v1, v3, edge_type); + VertexAccessor v1 = dba_->InsertVertex(); + VertexAccessor v2 = dba_->InsertVertex(); + VertexAccessor v3 = dba_->InsertVertex(); + storage::EdgeType edge_type = dba_->EdgeType("Edge"); + EdgeAccessor r1 = dba_->InsertEdge(v1, v2, edge_type); + EdgeAccessor r2 = dba_->InsertEdge(v1, v3, edge_type); void SetUp() override { - v1.add_label(dba_.Label("l1")); - v2.add_label(dba_.Label("l2")); - v3.add_label(dba_.Label("l3")); - dba_.AdvanceCommand(); + v1.add_label(dba_->Label("l1")); + v2.add_label(dba_->Label("l2")); + v3.add_label(dba_->Label("l3")); + dba_->AdvanceCommand(); } }; @@ -420,22 +421,22 @@ TEST_F(ExpandFixture, Expand) { symbol_table.CreateSymbol("named_expression_1", true); auto produce = MakeProduce(r_m.op_, output); - return PullAll(produce, dba_, symbol_table); + return PullAll(produce, *dba_, symbol_table); }; // test that expand works well for both old and new graph state v1.Reconstruct(); v2.Reconstruct(); v3.Reconstruct(); - dba_.InsertEdge(v1, v2, edge_type); - dba_.InsertEdge(v1, v3, edge_type); + dba_->InsertEdge(v1, v2, edge_type); + dba_->InsertEdge(v1, v3, edge_type); EXPECT_EQ(2, test_expand(EdgeAtom::Direction::OUT, GraphView::OLD)); EXPECT_EQ(2, test_expand(EdgeAtom::Direction::IN, GraphView::OLD)); EXPECT_EQ(4, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD)); EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, GraphView::NEW)); EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, GraphView::NEW)); EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::NEW)); - dba_.AdvanceCommand(); + dba_->AdvanceCommand(); EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, GraphView::OLD)); EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, GraphView::OLD)); EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD)); @@ -456,7 +457,7 @@ TEST_F(ExpandFixture, ExpandPath) { auto produce = MakeProduce(path, output); std::vector<query::Path> expected_paths{{v1, r2, v3}, {v1, r1, v2}}; - auto results = CollectProduce(produce.get(), symbol_table, dba_); + auto results = CollectProduce(produce.get(), symbol_table, *dba_); ASSERT_EQ(results.size(), 2); std::vector<query::Path> results_paths; for (const auto &result : results) @@ -484,11 +485,11 @@ class QueryPlanExpandVariable : public testing::Test { using map_int = std::unordered_map<int, int>; database::SingleNode db_; - database::GraphDbAccessor dba_{db_}; + std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()}; // labels for layers in the double chain std::vector<storage::Label> labels; // for all the edges - storage::EdgeType edge_type = dba_.EdgeType("edge_type"); + storage::EdgeType edge_type = dba_->EdgeType("edge_type"); AstStorage storage; SymbolTable symbol_table; @@ -502,26 +503,26 @@ class QueryPlanExpandVariable : public testing::Test { std::vector<VertexAccessor> layer; for (int from_layer_ind = -1; from_layer_ind < chain_length - 1; from_layer_ind++) { - std::vector<VertexAccessor> new_layer{dba_.InsertVertex(), - dba_.InsertVertex()}; - auto label = dba_.Label(std::to_string(from_layer_ind + 1)); + std::vector<VertexAccessor> new_layer{dba_->InsertVertex(), + dba_->InsertVertex()}; + auto label = dba_->Label(std::to_string(from_layer_ind + 1)); labels.push_back(label); for (size_t v_to_ind = 0; v_to_ind < new_layer.size(); v_to_ind++) { auto &v_to = new_layer[v_to_ind]; v_to.add_label(label); for (size_t v_from_ind = 0; v_from_ind < layer.size(); v_from_ind++) { auto &v_from = layer[v_from_ind]; - auto edge = dba_.InsertEdge(v_from, v_to, edge_type); - edge.PropsSet(dba_.Property("p"), + auto edge = dba_->InsertEdge(v_from, v_to, edge_type); + edge.PropsSet(dba_->Property("p"), fmt::format("V{}{}->V{}{}", from_layer_ind, v_from_ind, from_layer_ind + 1, v_to_ind)); } } layer = new_layer; } - dba_.AdvanceCommand(); - ASSERT_EQ(CountIterable(dba_.Vertices(false)), 2 * chain_length); - ASSERT_EQ(CountIterable(dba_.Edges(false)), 4 * (chain_length - 1)); + dba_->AdvanceCommand(); + ASSERT_EQ(CountIterable(dba_->Vertices(false)), 2 * chain_length); + ASSERT_EQ(CountIterable(dba_->Edges(false)), 4 * (chain_length - 1)); } /** @@ -591,8 +592,8 @@ class QueryPlanExpandVariable : public testing::Test { template <typename TResult> auto GetResults(std::shared_ptr<LogicalOperator> input_op, Symbol symbol) { Frame frame(symbol_table.max_position()); - auto cursor = input_op->MakeCursor(dba_); - Context context(dba_); + auto cursor = input_op->MakeCursor(*dba_); + Context context(*dba_); context.symbol_table_ = symbol_table; std::vector<TResult> results; while (cursor->Pull(frame, context)) @@ -686,7 +687,6 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessSingleAndVariableExpansion) { std::experimental::optional<size_t> upper, bool single_expansion_before, bool add_uniqueness_check) { - std::shared_ptr<LogicalOperator> last_op{nullptr}; std::vector<Symbol> symbols; @@ -765,22 +765,22 @@ TEST_F(QueryPlanExpandVariable, GraphState) { e); }; - auto new_edge_type = dba_.EdgeType("some_type"); + auto new_edge_type = dba_->EdgeType("some_type"); // add two vertices branching out from the second layer - for (VertexAccessor &vertex : dba_.Vertices(true)) + for (VertexAccessor &vertex : dba_->Vertices(true)) if (vertex.has_label(labels[1])) { - auto new_vertex = dba_.InsertVertex(); - dba_.InsertEdge(vertex, new_vertex, new_edge_type); + auto new_vertex = dba_->InsertVertex(); + dba_->InsertEdge(vertex, new_vertex, new_edge_type); } - ASSERT_EQ(CountIterable(dba_.Vertices(false)), 6); - ASSERT_EQ(CountIterable(dba_.Vertices(true)), 8); + ASSERT_EQ(CountIterable(dba_->Vertices(false)), 6); + ASSERT_EQ(CountIterable(dba_->Vertices(true)), 8); EXPECT_EQ(test_expand(GraphView::OLD, {}), (map_int{{2, 8}})); EXPECT_EQ(test_expand(GraphView::OLD, {new_edge_type}), (map_int{})); EXPECT_EQ(test_expand(GraphView::NEW, {}), (map_int{{2, 12}})); EXPECT_EQ(test_expand(GraphView::NEW, {edge_type}), (map_int{{2, 8}})); EXPECT_EQ(test_expand(GraphView::NEW, {new_edge_type}), (map_int{})); - dba_.AdvanceCommand(); + dba_->AdvanceCommand(); for (const auto graph_view : {GraphView::OLD, GraphView::NEW}) { EXPECT_EQ(test_expand(graph_view, {}), (map_int{{2, 12}})); EXPECT_EQ(test_expand(graph_view, {edge_type}), (map_int{{2, 8}})); @@ -806,7 +806,7 @@ TEST_F(QueryPlanExpandVariable, NamedPath) { std::vector<Symbol>{find_symbol("n"), e, find_symbol("m")}); std::vector<query::Path> expected_paths; - for (const auto &v : dba_.Vertices(labels[0], false)) + for (const auto &v : dba_->Vertices(labels[0], false)) for (const auto &e1 : v.out()) for (const auto &e2 : e1.to().out()) expected_paths.emplace_back(v, e1, e1.to(), e2, e2.to()); @@ -851,7 +851,8 @@ class QueryPlanExpandBfs if (cluster_) return cluster_->master(); return single_node_.get(); }()), - dba(*db_) {} + dba_ptr(db_->Access()), + dba(*dba_ptr) {} // Worker IDs where vertices are located. const std::vector<int> vertices = {0, 1, 1, 0, 1, 2}; @@ -861,7 +862,8 @@ class QueryPlanExpandBfs // Style-guide non-conformant name due to PROPERTY_PAIR and PROPERTY_LOOKUP // macro requirements. - database::GraphDbAccessor dba; + std::unique_ptr<database::GraphDbAccessor> dba_ptr; + database::GraphDbAccessor &dba; std::vector<storage::VertexAddress> v; AstStorage storage; @@ -883,8 +885,8 @@ class QueryPlanExpandBfs vertex.PropsSet(prop.second, id); v.push_back(vertex.GlobalAddress()); } else { - auto vertex = - dba.InsertVertexIntoRemote(worker, {}, {{prop.second, id}}); + auto vertex = database::InsertVertexIntoRemote(&dba, worker, {}, + {{prop.second, id}}); v.push_back(vertex.GlobalAddress()); } } @@ -973,7 +975,7 @@ class QueryPlanExpandBfs if (GetParam().first == TestType::DISTRIBUTED) cluster_->AdvanceCommand(tx_id); else - database::GraphDbAccessor(*db_, tx_id).AdvanceCommand(); + db_->Access(tx_id)->AdvanceCommand(); } }; @@ -1240,7 +1242,8 @@ class QueryPlanExpandWeightedShortestPath : public testing::Test { // style-guide non-conformant name due to PROPERTY_PAIR and // PROPERTY_LOOKUP macro requirements database::SingleNode db; - database::GraphDbAccessor dba{db}; + std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()}; + database::GraphDbAccessor &dba{*dba_ptr}; std::pair<std::string, storage::Property> prop = PROPERTY_PAIR("property"); storage::EdgeType edge_type = dba.EdgeType("edge_type"); @@ -1592,23 +1595,23 @@ TEST_F(QueryPlanExpandWeightedShortestPath, Exceptions) { TEST(QueryPlan, ExpandOptional) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; // graph (v2 {p: 2})<-[:T]-(v1 {p: 1})-[:T]->(v3 {p: 2}) - auto prop = dba.Property("p"); - auto edge_type = dba.EdgeType("T"); - auto v1 = dba.InsertVertex(); + auto prop = dba->Property("p"); + auto edge_type = dba->EdgeType("T"); + auto v1 = dba->InsertVertex(); v1.PropsSet(prop, 1); - auto v2 = dba.InsertVertex(); + auto v2 = dba->InsertVertex(); v2.PropsSet(prop, 2); - dba.InsertEdge(v1, v2, edge_type); - auto v3 = dba.InsertVertex(); + dba->InsertEdge(v1, v2, edge_type); + auto v3 = dba->InsertVertex(); v3.PropsSet(prop, 2); - dba.InsertEdge(v1, v3, edge_type); - dba.AdvanceCommand(); + dba->InsertEdge(v1, v3, edge_type); + dba->AdvanceCommand(); // MATCH (n) OPTIONAL MATCH (n)-[r]->(m) auto n = MakeScanAll(storage, symbol_table, "n"); @@ -1630,7 +1633,7 @@ TEST(QueryPlan, ExpandOptional) { symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true); auto produce = MakeProduce(optional, n_ne, r_ne, m_ne); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); ASSERT_EQ(4, results.size()); int v1_is_n_count = 0; for (auto &row : results) { @@ -1652,7 +1655,7 @@ TEST(QueryPlan, ExpandOptional) { TEST(QueryPlan, OptionalMatchEmptyDB) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; @@ -1667,14 +1670,14 @@ TEST(QueryPlan, OptionalMatchEmptyDB) { std::vector<Symbol>{n.sym_}); auto produce = MakeProduce(optional, n_ne); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); ASSERT_EQ(1, results.size()); EXPECT_EQ(results[0][0].type(), TypedValue::Type::Null); } TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; // OPTIONAL MATCH (n) @@ -1696,26 +1699,26 @@ TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) { symbol_table[*m_ne->expression_] = r_m.node_sym_; symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true); auto produce = MakeProduce(r_m.op_, m_ne); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(0, results.size()); } TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Make a graph with 2 connected, unlabeled nodes. - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto edge_type = dba.EdgeType("edge_type"); - dba.InsertEdge(v1, v2, edge_type); - dba.AdvanceCommand(); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); - EXPECT_EQ(1, CountIterable(dba.Edges(false))); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto edge_type = dba->EdgeType("edge_type"); + dba->InsertEdge(v1, v2, edge_type); + dba->AdvanceCommand(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); + EXPECT_EQ(1, CountIterable(dba->Edges(false))); AstStorage storage; SymbolTable symbol_table; // OPTIONAL MATCH (n :missing) auto n = MakeScanAll(storage, symbol_table, "n"); - auto label_missing = dba.Label("missing"); + auto label_missing = dba->Label("missing"); n.node_->labels_.emplace_back(label_missing); auto *filter_expr = @@ -1745,22 +1748,22 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) { symbol_table[*m_ne->expression_] = m.sym_; symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true); auto produce = MakeProduce(expand, m_ne); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(0, results.size()); } TEST(QueryPlan, ExpandExistingNode) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // make a graph (v1)->(v2) that // has a recursive edge (v1)->(v1) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto edge_type = dba.EdgeType("Edge"); - dba.InsertEdge(v1, v1, edge_type); - dba.InsertEdge(v1, v2, edge_type); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto edge_type = dba->EdgeType("Edge"); + dba->InsertEdge(v1, v1, edge_type); + dba->InsertEdge(v1, v2, edge_type); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -1783,7 +1786,7 @@ TEST(QueryPlan, ExpandExistingNode) { symbol_table.CreateSymbol("named_expression_1", true); auto produce = MakeProduce(r_n.op_, output); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), expected_result_count); }; @@ -1795,11 +1798,11 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) { // we're testing that expanding on BOTH // does only one expansion for a cycle database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto v = dba.InsertVertex(); - dba.InsertEdge(v, v, dba.EdgeType("et")); - dba.AdvanceCommand(); + auto v = dba->InsertVertex(); + dba->InsertEdge(v, v, dba->EdgeType("et")); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -1808,12 +1811,13 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) { auto r_ = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r", EdgeAtom::Direction::BOTH, {}, "_", false, GraphView::OLD); - EXPECT_EQ(1, PullAll(r_.op_, dba, symbol_table)); + EXPECT_EQ(1, PullAll(r_.op_, *dba, symbol_table)); } TEST(QueryPlan, EdgeFilter) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // make an N-star expanding from (v1) // where only one edge will qualify @@ -1882,17 +1886,17 @@ TEST(QueryPlan, EdgeFilter) { TEST(QueryPlan, EdgeFilterMultipleTypes) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto type_1 = dba.EdgeType("type_1"); - auto type_2 = dba.EdgeType("type_2"); - auto type_3 = dba.EdgeType("type_3"); - dba.InsertEdge(v1, v2, type_1); - dba.InsertEdge(v1, v2, type_2); - dba.InsertEdge(v1, v2, type_3); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto type_1 = dba->EdgeType("type_1"); + auto type_2 = dba->EdgeType("type_2"); + auto type_3 = dba->EdgeType("type_3"); + dba->InsertEdge(v1, v2, type_1); + dba->InsertEdge(v1, v2, type_2); + dba->InsertEdge(v1, v2, type_3); + dba->AdvanceCommand(); AstStorage storage; SymbolTable symbol_table; @@ -1911,13 +1915,14 @@ TEST(QueryPlan, EdgeFilterMultipleTypes) { symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true); symbol_table[*output->expression_] = r_m.node_sym_; - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 2); } TEST(QueryPlan, Filter) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; // add a 6 nodes with property 'prop', 2 have true as value auto property = PROPERTY_PAIR("property"); @@ -1946,15 +1951,15 @@ TEST(QueryPlan, Filter) { TEST(QueryPlan, ExpandUniquenessFilter) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // make a graph that has (v1)->(v2) and a recursive edge (v1)->(v1) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto edge_type = dba.EdgeType("edge_type"); - dba.InsertEdge(v1, v2, edge_type); - dba.InsertEdge(v1, v1, edge_type); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto edge_type = dba->EdgeType("edge_type"); + dba->InsertEdge(v1, v2, edge_type); + dba->InsertEdge(v1, v1, edge_type); + dba->AdvanceCommand(); auto check_expand_results = [&](bool vertex_uniqueness, bool edge_uniqueness) { @@ -1981,7 +1986,7 @@ TEST(QueryPlan, ExpandUniquenessFilter) { last_op, r2_n3.node_sym_, std::vector<Symbol>{n1.sym_, r1_n2.node_sym_}); - return PullAll(last_op, dba, symbol_table); + return PullAll(last_op, *dba, symbol_table); }; EXPECT_EQ(2, check_expand_results(false, false)); @@ -1994,14 +1999,13 @@ TEST(QueryPlan, Distinct) { // UNWIND [1, 2, 3, 3] AS x RETURN DISTINCT x database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); AstStorage storage; SymbolTable symbol_table; auto check_distinct = [&](const std::vector<TypedValue> input, const std::vector<TypedValue> output, bool assume_int_value) { - auto input_expr = LITERAL(TypedValue(input)); auto x = symbol_table.CreateSymbol("x", true); @@ -2016,7 +2020,7 @@ TEST(QueryPlan, Distinct) { symbol_table[*x_ne] = symbol_table.CreateSymbol("x_ne", true); auto produce = MakeProduce(distinct, x_ne); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); ASSERT_EQ(output.size(), results.size()); auto output_it = output.begin(); for (const auto &row : results) { @@ -2037,14 +2041,14 @@ TEST(QueryPlan, Distinct) { TEST(QueryPlan, ScanAllByLabel) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Add a vertex with a label and one without. - auto label = dba.Label("label"); - auto labeled_vertex = dba.InsertVertex(); + auto label = dba->Label("label"); + auto labeled_vertex = dba->InsertVertex(); labeled_vertex.add_label(label); - dba.InsertVertex(); - dba.AdvanceCommand(); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); + dba->InsertVertex(); + dba->AdvanceCommand(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); // MATCH (n :label) AstStorage storage; SymbolTable symbol_table; @@ -2055,7 +2059,7 @@ TEST(QueryPlan, ScanAllByLabel) { auto produce = MakeProduce(scan_all_by_label.op_, output); symbol_table[*output->expression_] = scan_all_by_label.sym_; symbol_table[*output] = symbol_table.CreateSymbol("n", true); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); ASSERT_EQ(results.size(), 1); auto result_row = results[0]; ASSERT_EQ(result_row.size(), 1); @@ -2065,8 +2069,8 @@ TEST(QueryPlan, ScanAllByLabel) { TEST(QueryPlan, ScanAllByLabelProperty) { database::SingleNode db; // Add 5 vertices with same label, but with different property values. - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); // vertex property values that will be stored into the DB // clang-format off std::vector<TypedValue> values{ @@ -2075,17 +2079,17 @@ TEST(QueryPlan, ScanAllByLabelProperty) { std::vector<TypedValue>{2}}; // clang-format on { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); for (const auto &value : values) { - auto vertex = dba.InsertVertex(); + auto vertex = dba->InsertVertex(); vertex.add_label(label); vertex.PropsSet(prop, value); } - dba.Commit(); - database::GraphDbAccessor(db).BuildIndex(label, prop); + dba->Commit(); + db.Access()->BuildIndex(label, prop); } - database::GraphDbAccessor dba(db); - ASSERT_EQ(14, CountIterable(dba.Vertices(false))); + auto dba = db.Access(); + ASSERT_EQ(14, CountIterable(dba->Vertices(false))); auto check = [&dba, label, prop](TypedValue lower, Bound::Type lower_type, TypedValue upper, Bound::Type upper_type, @@ -2100,7 +2104,7 @@ TEST(QueryPlan, ScanAllByLabelProperty) { auto produce = MakeProduce(scan_all.op_, output); symbol_table[*output->expression_] = scan_all.sym_; symbol_table[*output] = symbol_table.CreateSymbol("n", true); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); ASSERT_EQ(results.size(), expected.size()); for (size_t i = 0; i < expected.size(); i++) { TypedValue equal = @@ -2136,21 +2140,21 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) { database::SingleNode db; // Add 2 vertices with same label, but with property values that cannot be // compared. On the other hand, equality works fine. - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); { - database::GraphDbAccessor dba(db); - auto number_vertex = dba.InsertVertex(); + auto dba = db.Access(); + auto number_vertex = dba->InsertVertex(); number_vertex.add_label(label); number_vertex.PropsSet(prop, 42); - auto string_vertex = dba.InsertVertex(); + auto string_vertex = dba->InsertVertex(); string_vertex.add_label(label); string_vertex.PropsSet(prop, "string"); - dba.Commit(); - database::GraphDbAccessor(db).BuildIndex(label, prop); + dba->Commit(); + db.Access()->BuildIndex(label, prop); } - database::GraphDbAccessor dba(db); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); + auto dba = db.Access(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); // MATCH (n :label {prop: 42}) AstStorage storage; SymbolTable symbol_table; @@ -2161,7 +2165,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) { auto produce = MakeProduce(scan_all.op_, output); symbol_table[*output->expression_] = scan_all.sym_; symbol_table[*output] = symbol_table.CreateSymbol("n", true); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); ASSERT_EQ(results.size(), 1); const auto &row = results[0]; ASSERT_EQ(row.size(), 1); @@ -2173,20 +2177,20 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) { TEST(QueryPlan, ScanAllByLabelPropertyValueError) { database::SingleNode db; - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); for (int i = 0; i < 2; ++i) { - auto vertex = dba.InsertVertex(); + auto vertex = dba->InsertVertex(); vertex.add_label(label); vertex.PropsSet(prop, i); } - dba.Commit(); + dba->Commit(); } - database::GraphDbAccessor(db).BuildIndex(label, prop); - database::GraphDbAccessor dba(db); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); + db.Access()->BuildIndex(label, prop); + auto dba = db.Access(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); // MATCH (m), (n :label {prop: m}) AstStorage storage; SymbolTable symbol_table; @@ -2195,26 +2199,26 @@ TEST(QueryPlan, ScanAllByLabelPropertyValueError) { symbol_table[*ident_m] = scan_all.sym_; auto scan_index = MakeScanAllByLabelPropertyValue( storage, symbol_table, "n", label, prop, ident_m, scan_all.op_); - EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table), + EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table), QueryRuntimeException); } TEST(QueryPlan, ScanAllByLabelPropertyRangeError) { database::SingleNode db; - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); for (int i = 0; i < 2; ++i) { - auto vertex = dba.InsertVertex(); + auto vertex = dba->InsertVertex(); vertex.add_label(label); vertex.PropsSet(prop, i); } - dba.Commit(); + dba->Commit(); } - database::GraphDbAccessor(db).BuildIndex(label, prop); - database::GraphDbAccessor dba(db); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); + db.Access()->BuildIndex(label, prop); + auto dba = db.Access(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); // MATCH (m), (n :label {prop: m}) AstStorage storage; SymbolTable symbol_table; @@ -2227,7 +2231,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) { storage, symbol_table, "n", label, prop, Bound{ident_m, Bound::Type::INCLUSIVE}, std::experimental::nullopt, scan_all.op_); - EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table), + EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table), QueryRuntimeException); } { @@ -2235,7 +2239,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) { auto scan_index = MakeScanAllByLabelPropertyRange( storage, symbol_table, "n", label, prop, std::experimental::nullopt, Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_); - EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table), + EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table), QueryRuntimeException); } { @@ -2244,7 +2248,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) { storage, symbol_table, "n", label, prop, Bound{ident_m, Bound::Type::INCLUSIVE}, Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_); - EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table), + EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table), QueryRuntimeException); } } @@ -2254,20 +2258,20 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) { // Add 2 vertices with the same label, but one has a property value while // the other does not. Checking if the value is equal to null, should // yield no results. - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); { - database::GraphDbAccessor dba(db); - auto vertex = dba.InsertVertex(); + auto dba = db.Access(); + auto vertex = dba->InsertVertex(); vertex.add_label(label); - auto vertex_with_prop = dba.InsertVertex(); + auto vertex_with_prop = dba->InsertVertex(); vertex_with_prop.add_label(label); vertex_with_prop.PropsSet(prop, 42); - dba.Commit(); - database::GraphDbAccessor(db).BuildIndex(label, prop); + dba->Commit(); + db.Access()->BuildIndex(label, prop); } - database::GraphDbAccessor dba(db); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); + auto dba = db.Access(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); // MATCH (n :label {prop: 42}) AstStorage storage; SymbolTable symbol_table; @@ -2278,7 +2282,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) { auto produce = MakeProduce(scan_all.op_, output); symbol_table[*output->expression_] = scan_all.sym_; symbol_table[*output] = symbol_table.CreateSymbol("n", true); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 0); } @@ -2287,20 +2291,20 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) { // Add 2 vertices with the same label, but one has a property value while // the other does not. Checking if the value is between nulls, should // yield no results. - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); { - database::GraphDbAccessor dba(db); - auto vertex = dba.InsertVertex(); + auto dba = db.Access(); + auto vertex = dba->InsertVertex(); vertex.add_label(label); - auto vertex_with_prop = dba.InsertVertex(); + auto vertex_with_prop = dba->InsertVertex(); vertex_with_prop.add_label(label); vertex_with_prop.PropsSet(prop, 42); - dba.Commit(); - database::GraphDbAccessor(db).BuildIndex(label, prop); + dba->Commit(); + db.Access()->BuildIndex(label, prop); } - database::GraphDbAccessor dba(db); - EXPECT_EQ(2, CountIterable(dba.Vertices(false))); + auto dba = db.Access(); + EXPECT_EQ(2, CountIterable(dba->Vertices(false))); // MATCH (n :label) WHERE null <= n.prop < null AstStorage storage; SymbolTable symbol_table; @@ -2313,24 +2317,24 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) { auto produce = MakeProduce(scan_all.op_, output); symbol_table[*output->expression_] = scan_all.sym_; symbol_table[*output] = symbol_table.CreateSymbol("n", true); - auto results = CollectProduce(produce.get(), symbol_table, dba); + auto results = CollectProduce(produce.get(), symbol_table, *dba); EXPECT_EQ(results.size(), 0); } TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) { database::SingleNode db; - auto label = database::GraphDbAccessor(db).Label("label"); - auto prop = database::GraphDbAccessor(db).Property("prop"); + auto label = db.Access()->Label("label"); + auto prop = db.Access()->Property("prop"); { - database::GraphDbAccessor dba(db); - auto v = dba.InsertVertex(); + auto dba = db.Access(); + auto v = dba->InsertVertex(); v.add_label(label); v.PropsSet(prop, 2); - dba.Commit(); - database::GraphDbAccessor(db).BuildIndex(label, prop); + dba->Commit(); + db.Access()->BuildIndex(label, prop); } - database::GraphDbAccessor dba(db); - EXPECT_EQ(1, CountIterable(dba.Vertices(false))); + auto dba = db.Access(); + EXPECT_EQ(1, CountIterable(dba->Vertices(false))); AstStorage storage; SymbolTable symbol_table; @@ -2346,7 +2350,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) { auto scan_all = MakeScanAllByLabelPropertyValue(storage, symbol_table, "n", label, prop, x_expr, unwind); - EXPECT_EQ(PullAll(scan_all.op_, dba, symbol_table), 1); + EXPECT_EQ(PullAll(scan_all.op_, *dba, symbol_table), 1); } int main(int argc, char **argv) { diff --git a/tests/unit/query_semantic.cpp b/tests/unit/query_semantic.cpp index 6469a52ba..46df5c252 100644 --- a/tests/unit/query_semantic.cpp +++ b/tests/unit/query_semantic.cpp @@ -18,7 +18,8 @@ using namespace query; class TestSymbolGenerator : public ::testing::Test { protected: database::SingleNode db; - database::GraphDbAccessor dba{db}; + std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()}; + database::GraphDbAccessor &dba{*dba_ptr}; SymbolTable symbol_table; SymbolGenerator symbol_generator{symbol_table}; AstStorage storage; diff --git a/tests/unit/query_variable_start_planner.cpp b/tests/unit/query_variable_start_planner.cpp index 949c51bf3..525a52ba1 100644 --- a/tests/unit/query_variable_start_planner.cpp +++ b/tests/unit/query_variable_start_planner.cpp @@ -84,19 +84,19 @@ void CheckPlansProduce( TEST(TestVariableStartPlanner, MatchReturn) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Make a graph (v1) -[:r]-> (v2) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - dba.InsertEdge(v1, v2, dba.EdgeType("r")); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + dba->InsertEdge(v1, v2, dba->EdgeType("r")); + dba->AdvanceCommand(); // Test MATCH (n) -[r]-> (m) RETURN n AstStorage storage; QUERY(SINGLE_QUERY( MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m"))), RETURN("n"))); // We have 2 nodes `n` and `m` from which we could start, so expect 2 plans. - CheckPlansProduce(2, storage, dba, [&](const auto &results) { + CheckPlansProduce(2, storage, *dba, [&](const auto &results) { // We expect to produce only a single (v1) node. AssertRows(results, {{v1}}); }); @@ -104,14 +104,14 @@ TEST(TestVariableStartPlanner, MatchReturn) { TEST(TestVariableStartPlanner, MatchTripletPatternReturn) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto v3 = dba.InsertVertex(); - dba.InsertEdge(v1, v2, dba.EdgeType("r")); - dba.InsertEdge(v2, v3, dba.EdgeType("r")); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto v3 = dba->InsertVertex(); + dba->InsertEdge(v1, v2, dba->EdgeType("r")); + dba->InsertEdge(v2, v3, dba->EdgeType("r")); + dba->AdvanceCommand(); { // Test `MATCH (n) -[r]-> (m) -[e]-> (l) RETURN n` AstStorage storage; @@ -120,7 +120,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) { EDGE("e", Direction::OUT), NODE("l"))), RETURN("n"))); // We have 3 nodes: `n`, `m` and `l` from which we could start. - CheckPlansProduce(3, storage, dba, [&](const auto &results) { + CheckPlansProduce(3, storage, *dba, [&](const auto &results) { // We expect to produce only a single (v1) node. AssertRows(results, {{v1}}); }); @@ -132,7 +132,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) { MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m")), PATTERN(NODE("m"), EDGE("e", Direction::OUT), NODE("l"))), RETURN("n"))); - CheckPlansProduce(3, storage, dba, [&](const auto &results) { + CheckPlansProduce(3, storage, *dba, [&](const auto &results) { AssertRows(results, {{v1}}); }); } @@ -140,14 +140,14 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) { TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto v3 = dba.InsertVertex(); - dba.InsertEdge(v1, v2, dba.EdgeType("r")); - dba.InsertEdge(v2, v3, dba.EdgeType("r")); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto v3 = dba->InsertVertex(); + dba->InsertEdge(v1, v2, dba->EdgeType("r")); + dba->InsertEdge(v2, v3, dba->EdgeType("r")); + dba->AdvanceCommand(); // Test MATCH (n) -[r]-> (m) OPTIONAL MATCH (m) -[e]-> (l) RETURN n, l AstStorage storage; QUERY(SINGLE_QUERY( @@ -156,7 +156,7 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) { RETURN("n", "l"))); // We have 2 nodes `n` and `m` from which we could start the MATCH, and 2 // nodes for OPTIONAL MATCH. This should produce 2 * 2 plans. - CheckPlansProduce(4, storage, dba, [&](const auto &results) { + CheckPlansProduce(4, storage, *dba, [&](const auto &results) { // We expect to produce 2 rows: // * (v1), (v3) // * (v2), null @@ -166,13 +166,13 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) { TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Graph (v1) -[:r]-> (v2) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto r_type = dba.EdgeType("r"); - dba.InsertEdge(v1, v2, r_type); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto r_type = dba->EdgeType("r"); + dba->InsertEdge(v1, v2, r_type); + dba->AdvanceCommand(); // Test MATCH (n) -[r]-> (m) OPTIONAL MATCH (m) -[e]-> (l) // MERGE (u) -[q:r]-> (v) RETURN n, m, l, u, v AstStorage storage; @@ -183,7 +183,7 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) { RETURN("n", "m", "l", "u", "v"))); // Since MATCH, OPTIONAL MATCH and MERGE each have 2 nodes from which we can // start, we generate 2 * 2 * 2 plans. - CheckPlansProduce(8, storage, dba, [&](const auto &results) { + CheckPlansProduce(8, storage, *dba, [&](const auto &results) { // We expect to produce a single row: (v1), (v2), null, (v1), (v2) AssertRows(results, {{v1, v2, TypedValue::Null, v1, v2}}); }); @@ -191,12 +191,12 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) { TEST(TestVariableStartPlanner, MatchWithMatchReturn) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Graph (v1) -[:r]-> (v2) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - dba.InsertEdge(v1, v2, dba.EdgeType("r")); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + dba->InsertEdge(v1, v2, dba->EdgeType("r")); + dba->AdvanceCommand(); // Test MATCH (n) -[r]-> (m) WITH n MATCH (m) -[r]-> (l) RETURN n, m, l AstStorage storage; QUERY(SINGLE_QUERY( @@ -206,7 +206,7 @@ TEST(TestVariableStartPlanner, MatchWithMatchReturn) { RETURN("n", "m", "l"))); // We can start from 2 nodes in each match. Since WITH separates query parts, // we expect to get 2 plans for each, which totals 2 * 2. - CheckPlansProduce(4, storage, dba, [&](const auto &results) { + CheckPlansProduce(4, storage, *dba, [&](const auto &results) { // We expect to produce a single row: (v1), (v1), (v2) AssertRows(results, {{v1, v1, v2}}); }); @@ -214,14 +214,14 @@ TEST(TestVariableStartPlanner, MatchWithMatchReturn) { TEST(TestVariableStartPlanner, MatchVariableExpand) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); // Graph (v1) -[:r1]-> (v2) -[:r2]-> (v3) - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto v3 = dba.InsertVertex(); - auto r1 = dba.InsertEdge(v1, v2, dba.EdgeType("r1")); - auto r2 = dba.InsertEdge(v2, v3, dba.EdgeType("r2")); - dba.AdvanceCommand(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto v3 = dba->InsertVertex(); + auto r1 = dba->InsertEdge(v1, v2, dba->EdgeType("r1")); + auto r2 = dba->InsertEdge(v2, v3, dba->EdgeType("r2")); + dba->AdvanceCommand(); // Test MATCH (n) -[r*]-> (m) RETURN r AstStorage storage; auto edge = EDGE_VARIABLE("r", Direction::OUT); @@ -230,14 +230,15 @@ TEST(TestVariableStartPlanner, MatchVariableExpand) { TypedValue r1_list(std::vector<TypedValue>{r1}); // [r1] TypedValue r2_list(std::vector<TypedValue>{r2}); // [r2] TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2}); // [r1, r2] - CheckPlansProduce(2, storage, dba, [&](const auto &results) { + CheckPlansProduce(2, storage, *dba, [&](const auto &results) { AssertRows(results, {{r1_list}, {r2_list}, {r1_r2_list}}); }); } TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto id = dba.Property("id"); // Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3}) auto v1 = dba.InsertVertex(); @@ -264,16 +265,16 @@ TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) { TEST(TestVariableStartPlanner, MatchVariableExpandBoth) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto id = dba.Property("id"); + auto dba = db.Access(); + auto id = dba->Property("id"); // Graph (v1 {id:1}) -[:r1]-> (v2) -[:r2]-> (v3) - auto v1 = dba.InsertVertex(); + auto v1 = dba->InsertVertex(); v1.PropsSet(id, 1); - auto v2 = dba.InsertVertex(); - auto v3 = dba.InsertVertex(); - auto r1 = dba.InsertEdge(v1, v2, dba.EdgeType("r1")); - auto r2 = dba.InsertEdge(v2, v3, dba.EdgeType("r2")); - dba.AdvanceCommand(); + auto v2 = dba->InsertVertex(); + auto v3 = dba->InsertVertex(); + auto r1 = dba->InsertEdge(v1, v2, dba->EdgeType("r1")); + auto r2 = dba->InsertEdge(v2, v3, dba->EdgeType("r2")); + dba->AdvanceCommand(); // Test MATCH (n {id:1}) -[r*]- (m) RETURN r AstStorage storage; auto edge = EDGE_VARIABLE("r", Direction::BOTH); @@ -283,14 +284,15 @@ TEST(TestVariableStartPlanner, MatchVariableExpandBoth) { // We expect to get a single column with the following rows: TypedValue r1_list(std::vector<TypedValue>{r1}); // [r1] TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2}); // [r1, r2] - CheckPlansProduce(2, storage, dba, [&](const auto &results) { + CheckPlansProduce(2, storage, *dba, [&](const auto &results) { AssertRows(results, {{r1_list}, {r1_r2_list}}); }); } TEST(TestVariableStartPlanner, MatchBfs) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba_ptr = db.Access(); + auto &dba = *dba_ptr; auto id = dba.Property("id"); // Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3}) auto v1 = dba.InsertVertex(); diff --git a/tests/unit/record_edge_vertex_accessor.cpp b/tests/unit/record_edge_vertex_accessor.cpp index 6956a995c..b1f27b742 100644 --- a/tests/unit/record_edge_vertex_accessor.cpp +++ b/tests/unit/record_edge_vertex_accessor.cpp @@ -14,13 +14,13 @@ TEST(RecordAccessor, Properties) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto vertex = dba.InsertVertex(); + auto vertex = dba->InsertVertex(); auto &properties = vertex.Properties(); - auto property = dba.Property("PropName"); - auto property_other = dba.Property("Other"); + auto property = dba->Property("PropName"); + auto property_other = dba->Property("Other"); EXPECT_EQ(vertex.PropsAt(property).type(), PropertyValue::Type::Null); vertex.PropsSet(property, 42); @@ -36,39 +36,38 @@ TEST(RecordAccessor, Properties) { TEST(RecordAccessor, DbAccessor) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto vertex = dba.InsertVertex(); + auto vertex = dba->InsertVertex(); const auto &const_vertex_dba = vertex.db_accessor(); - EXPECT_EQ(&dba, &const_vertex_dba); + EXPECT_EQ(dba.get(), &const_vertex_dba); auto &vertex_dba = vertex.db_accessor(); - EXPECT_EQ(&dba, &vertex_dba); + EXPECT_EQ(dba.get(), &vertex_dba); } TEST(RecordAccessor, RecordEquality) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); EXPECT_EQ(v1, v1); EXPECT_NE(v1, v2); - auto e1 = dba.InsertEdge(v1, v2, dba.EdgeType("type")); - auto e2 = dba.InsertEdge(v1, v2, dba.EdgeType("type")); + auto e1 = dba->InsertEdge(v1, v2, dba->EdgeType("type")); + auto e2 = dba->InsertEdge(v1, v2, dba->EdgeType("type")); EXPECT_EQ(e1, e1); EXPECT_NE(e1, e2); } TEST(RecordAccessor, GlobalToLocalAddressConversion) { database::SingleNode db; - database::GraphDbAccessor dba(db); + auto dba = db.Access(); - auto v1 = dba.InsertVertex(); - storage::Address<mvcc::VersionList<Vertex>> global_address{v1.gid(), - db.WorkerId()}; + auto v1 = dba->InsertVertex(); + storage::Address<mvcc::VersionList<Vertex>> global_address{v1.gid(), 0}; EXPECT_FALSE(global_address.is_local()); - auto v1_from_global = VertexAccessor(global_address, dba); + auto v1_from_global = VertexAccessor(global_address, *dba); EXPECT_TRUE(v1_from_global.address().is_local()); EXPECT_EQ(v1_from_global.address(), v1.address()); } @@ -78,26 +77,26 @@ TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) { // test both Switches work on new record { - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); v1.SwitchOld(); v1.SwitchNew(); - dba.Commit(); + dba->Commit(); } // test both Switches work on existing record { - database::GraphDbAccessor dba(db); - auto v1 = *dba.Vertices(false).begin(); + auto dba = db.Access(); + auto v1 = *dba->Vertices(false).begin(); v1.SwitchOld(); v1.SwitchNew(); } // ensure switch exposes the right data { - database::GraphDbAccessor dba(db); - auto label = dba.Label("label"); - auto v1 = *dba.Vertices(false).begin(); + auto dba = db.Access(); + auto label = dba->Label("label"); + auto v1 = *dba->Vertices(false).begin(); EXPECT_FALSE(v1.has_label(label)); // old record v1.add_label(label); // modifying data does not switch to new @@ -111,26 +110,26 @@ TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) { TEST(RecordAccessor, Reconstruct) { database::SingleNode db; - auto label = database::GraphDbAccessor(db).Label("label"); + auto label = db.Access()->Label("label"); { // we must operate on an old vertex // because otherwise we only have new // so create a vertex and commit it - database::GraphDbAccessor dba(db); - dba.InsertVertex(); - dba.Commit(); + auto dba = db.Access(); + dba->InsertVertex(); + dba->Commit(); } // ensure we don't have label set - database::GraphDbAccessor dba(db); - auto v1 = *dba.Vertices(false).begin(); + auto dba = db.Access(); + auto v1 = *dba->Vertices(false).begin(); v1.SwitchNew(); EXPECT_FALSE(v1.has_label(label)); { // update the record through a different accessor - auto v1_other_accessor = *dba.Vertices(false).begin(); + auto v1_other_accessor = *dba->Vertices(false).begin(); v1_other_accessor.add_label(label); EXPECT_FALSE(v1.has_label(label)); v1_other_accessor.SwitchNew(); @@ -145,14 +144,14 @@ TEST(RecordAccessor, Reconstruct) { TEST(RecordAccessor, VertexLabels) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); auto &labels = v1.labels(); EXPECT_EQ(v1.labels().size(), 0); - storage::Label l1 = dba.Label("label1"); - storage::Label l2 = dba.Label("label2"); + storage::Label l1 = dba->Label("label1"); + storage::Label l2 = dba->Label("label2"); // adding labels EXPECT_FALSE(v1.has_label(l1)); @@ -172,7 +171,7 @@ TEST(RecordAccessor, VertexLabels) { EXPECT_EQ(labels.size(), 2); // removing labels - storage::Label l3 = dba.Label("label3"); + storage::Label l3 = dba->Label("label3"); v1.remove_label(l3); EXPECT_EQ(labels.size(), 2); @@ -186,38 +185,38 @@ TEST(RecordAccessor, VertexLabels) { TEST(RecordAccessor, EdgeType) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); - storage::EdgeType likes = dba.EdgeType("likes"); - storage::EdgeType hates = dba.EdgeType("hates"); + storage::EdgeType likes = dba->EdgeType("likes"); + storage::EdgeType hates = dba->EdgeType("hates"); - auto edge = dba.InsertEdge(v1, v2, likes); + auto edge = dba->InsertEdge(v1, v2, likes); EXPECT_EQ(edge.EdgeType(), likes); EXPECT_NE(edge.EdgeType(), hates); } TEST(RecordAccessor, EdgeIsCycle) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto likes = dba.EdgeType("edge_type"); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto likes = dba->EdgeType("edge_type"); - EXPECT_TRUE(dba.InsertEdge(v1, v1, likes).is_cycle()); - EXPECT_TRUE(dba.InsertEdge(v2, v2, likes).is_cycle()); - EXPECT_FALSE(dba.InsertEdge(v1, v2, likes).is_cycle()); - EXPECT_FALSE(dba.InsertEdge(v2, v1, likes).is_cycle()); + EXPECT_TRUE(dba->InsertEdge(v1, v1, likes).is_cycle()); + EXPECT_TRUE(dba->InsertEdge(v2, v2, likes).is_cycle()); + EXPECT_FALSE(dba->InsertEdge(v1, v2, likes).is_cycle()); + EXPECT_FALSE(dba->InsertEdge(v2, v1, likes).is_cycle()); } TEST(RecordAccessor, VertexEdgeConnections) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto edge = dba.InsertEdge(v1, v2, dba.EdgeType("likes")); - dba.AdvanceCommand(); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto edge = dba->InsertEdge(v1, v2, dba->EdgeType("likes")); + dba->AdvanceCommand(); EXPECT_EQ(edge.from(), v1); EXPECT_NE(edge.from(), v2); @@ -245,17 +244,17 @@ TEST(RecordAccessor, VertexEdgeConnections) { TEST(RecordAccessor, VertexEdgeConnectionsWithExistingVertex) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto v3 = dba.InsertVertex(); - auto edge_type = dba.EdgeType("edge type"); - auto e12 = dba.InsertEdge(v1, v2, edge_type); - auto e22 = dba.InsertEdge(v2, v2, edge_type); - auto e23a = dba.InsertEdge(v2, v3, edge_type); - auto e23b = dba.InsertEdge(v2, v3, edge_type); - auto e32 = dba.InsertEdge(v3, v2, edge_type); - dba.AdvanceCommand(); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto v3 = dba->InsertVertex(); + auto edge_type = dba->EdgeType("edge type"); + auto e12 = dba->InsertEdge(v1, v2, edge_type); + auto e22 = dba->InsertEdge(v2, v2, edge_type); + auto e23a = dba->InsertEdge(v2, v3, edge_type); + auto e23b = dba->InsertEdge(v2, v3, edge_type); + auto e32 = dba->InsertEdge(v3, v2, edge_type); + dba->AdvanceCommand(); TEST_EDGE_ITERABLE(v1.out(v1)); TEST_EDGE_ITERABLE(v1.out(v2), {e12}); @@ -280,17 +279,17 @@ TEST(RecordAccessor, VertexEdgeConnectionsWithExistingVertex) { TEST(RecordAccessor, VertexEdgeConnectionsWithEdgeType) { database::SingleNode db; - database::GraphDbAccessor dba(db); - auto v1 = dba.InsertVertex(); - auto v2 = dba.InsertVertex(); - auto a = dba.EdgeType("a"); - auto b = dba.EdgeType("b"); - auto c = dba.EdgeType("c"); - auto ea = dba.InsertEdge(v1, v2, a); - auto eb_1 = dba.InsertEdge(v2, v1, b); - auto eb_2 = dba.InsertEdge(v2, v1, b); - auto ec = dba.InsertEdge(v1, v2, c); - dba.AdvanceCommand(); + auto dba = db.Access(); + auto v1 = dba->InsertVertex(); + auto v2 = dba->InsertVertex(); + auto a = dba->EdgeType("a"); + auto b = dba->EdgeType("b"); + auto c = dba->EdgeType("c"); + auto ea = dba->InsertEdge(v1, v2, a); + auto eb_1 = dba->InsertEdge(v2, v1, b); + auto eb_2 = dba->InsertEdge(v2, v1, b); + auto ec = dba->InsertEdge(v1, v2, c); + dba->AdvanceCommand(); TEST_EDGE_ITERABLE(v1.in(), {eb_1, eb_2}); TEST_EDGE_ITERABLE(v2.in(), {ea, ec}); diff --git a/tests/unit/state_delta.cpp b/tests/unit/state_delta.cpp index f7e86e6e5..7d64189b7 100644 --- a/tests/unit/state_delta.cpp +++ b/tests/unit/state_delta.cpp @@ -9,15 +9,15 @@ TEST(StateDelta, CreateVertex) { gid::Generator generator(0); auto gid0 = generator.Next(); { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); auto delta = - database::StateDelta::CreateVertex(dba.transaction_id(), gid0, 0); - delta.Apply(dba); - dba.Commit(); + database::StateDelta::CreateVertex(dba->transaction_id(), gid0, 0); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto vertex = dba.FindVertexOptional(gid0, false); + auto dba = db.Access(); + auto vertex = dba->FindVertexOptional(gid0, false); EXPECT_TRUE(vertex); EXPECT_EQ(vertex->cypher_id(), 0); } @@ -28,20 +28,20 @@ TEST(StateDelta, RemoveVertex) { gid::Generator generator(0); auto gid0 = generator.Next(); { - database::GraphDbAccessor dba(db); - dba.InsertVertex(gid0); - dba.Commit(); + auto dba = db.Access(); + dba->InsertVertex(gid0); + dba->Commit(); } { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); auto delta = - database::StateDelta::RemoveVertex(dba.transaction_id(), gid0, true); - delta.Apply(dba); - dba.Commit(); + database::StateDelta::RemoveVertex(dba->transaction_id(), gid0, true); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto vertex = dba.FindVertexOptional(gid0, false); + auto dba = db.Access(); + auto vertex = dba->FindVertexOptional(gid0, false); EXPECT_FALSE(vertex); } } @@ -53,22 +53,22 @@ TEST(StateDelta, CreateEdge) { auto gid1 = generator.Next(); auto gid2 = generator.Next(); { - database::GraphDbAccessor dba(db); - dba.InsertVertex(gid0); - dba.InsertVertex(gid1); - dba.Commit(); + auto dba = db.Access(); + dba->InsertVertex(gid0); + dba->InsertVertex(gid1); + dba->Commit(); } { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); auto delta = - database::StateDelta::CreateEdge(dba.transaction_id(), gid2, 0, gid0, - gid1, dba.EdgeType("edge"), "edge"); - delta.Apply(dba); - dba.Commit(); + database::StateDelta::CreateEdge(dba->transaction_id(), gid2, 0, gid0, + gid1, dba->EdgeType("edge"), "edge"); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto edge = dba.FindEdgeOptional(gid2, false); + auto dba = db.Access(); + auto edge = dba->FindEdgeOptional(gid2, false); EXPECT_TRUE(edge); } } @@ -80,21 +80,21 @@ TEST(StateDelta, RemoveEdge) { auto gid1 = generator.Next(); auto gid2 = generator.Next(); { - database::GraphDbAccessor dba(db); - auto v0 = dba.InsertVertex(gid0); - auto v1 = dba.InsertVertex(gid1); - dba.InsertEdge(v0, v1, dba.EdgeType("edge"), gid2); - dba.Commit(); + auto dba = db.Access(); + auto v0 = dba->InsertVertex(gid0); + auto v1 = dba->InsertVertex(gid1); + dba->InsertEdge(v0, v1, dba->EdgeType("edge"), gid2); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto delta = database::StateDelta::RemoveEdge(dba.transaction_id(), gid2); - delta.Apply(dba); - dba.Commit(); + auto dba = db.Access(); + auto delta = database::StateDelta::RemoveEdge(dba->transaction_id(), gid2); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto edge = dba.FindEdgeOptional(gid2, false); + auto dba = db.Access(); + auto edge = dba->FindEdgeOptional(gid2, false); EXPECT_FALSE(edge); } } @@ -104,24 +104,24 @@ TEST(StateDelta, AddLabel) { gid::Generator generator(0); auto gid0 = generator.Next(); { - database::GraphDbAccessor dba(db); - dba.InsertVertex(gid0); - dba.Commit(); + auto dba = db.Access(); + dba->InsertVertex(gid0); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto delta = database::StateDelta::AddLabel(dba.transaction_id(), gid0, - dba.Label("label"), "label"); - delta.Apply(dba); - dba.Commit(); + auto dba = db.Access(); + auto delta = database::StateDelta::AddLabel(dba->transaction_id(), gid0, + dba->Label("label"), "label"); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto vertex = dba.FindVertexOptional(gid0, false); + auto dba = db.Access(); + auto vertex = dba->FindVertexOptional(gid0, false); EXPECT_TRUE(vertex); auto labels = vertex->labels(); EXPECT_EQ(labels.size(), 1); - EXPECT_EQ(labels[0], dba.Label("label")); + EXPECT_EQ(labels[0], dba->Label("label")); } } @@ -130,21 +130,21 @@ TEST(StateDelta, RemoveLabel) { gid::Generator generator(0); auto gid0 = generator.Next(); { - database::GraphDbAccessor dba(db); - auto vertex = dba.InsertVertex(gid0); - vertex.add_label(dba.Label("label")); - dba.Commit(); + auto dba = db.Access(); + auto vertex = dba->InsertVertex(gid0); + vertex.add_label(dba->Label("label")); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto delta = database::StateDelta::RemoveLabel(dba.transaction_id(), gid0, - dba.Label("label"), "label"); - delta.Apply(dba); - dba.Commit(); + auto dba = db.Access(); + auto delta = database::StateDelta::RemoveLabel(dba->transaction_id(), gid0, + dba->Label("label"), "label"); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto vertex = dba.FindVertexOptional(gid0, false); + auto dba = db.Access(); + auto vertex = dba->FindVertexOptional(gid0, false); EXPECT_TRUE(vertex); auto labels = vertex->labels(); EXPECT_EQ(labels.size(), 0); @@ -156,23 +156,23 @@ TEST(StateDelta, SetPropertyVertex) { gid::Generator generator(0); auto gid0 = generator.Next(); { - database::GraphDbAccessor dba(db); - dba.InsertVertex(gid0); - dba.Commit(); + auto dba = db.Access(); + dba->InsertVertex(gid0); + dba->Commit(); } { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); auto delta = database::StateDelta::PropsSetVertex( - dba.transaction_id(), gid0, dba.Property("property"), "property", + dba->transaction_id(), gid0, dba->Property("property"), "property", PropertyValue(2212)); - delta.Apply(dba); - dba.Commit(); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto vertex = dba.FindVertexOptional(gid0, false); + auto dba = db.Access(); + auto vertex = dba->FindVertexOptional(gid0, false); EXPECT_TRUE(vertex); - auto prop = vertex->PropsAt(dba.Property("property")); + auto prop = vertex->PropsAt(dba->Property("property")); EXPECT_EQ(prop.Value<int64_t>(), 2212); } } @@ -184,25 +184,25 @@ TEST(StateDelta, SetPropertyEdge) { auto gid1 = generator.Next(); auto gid2 = generator.Next(); { - database::GraphDbAccessor dba(db); - auto v0 = dba.InsertVertex(gid0); - auto v1 = dba.InsertVertex(gid1); - dba.InsertEdge(v0, v1, dba.EdgeType("edge"), gid2); - dba.Commit(); + auto dba = db.Access(); + auto v0 = dba->InsertVertex(gid0); + auto v1 = dba->InsertVertex(gid1); + dba->InsertEdge(v0, v1, dba->EdgeType("edge"), gid2); + dba->Commit(); } { - database::GraphDbAccessor dba(db); + auto dba = db.Access(); auto delta = database::StateDelta::PropsSetEdge( - dba.transaction_id(), gid2, dba.Property("property"), "property", + dba->transaction_id(), gid2, dba->Property("property"), "property", PropertyValue(2212)); - delta.Apply(dba); - dba.Commit(); + delta.Apply(*dba); + dba->Commit(); } { - database::GraphDbAccessor dba(db); - auto edge = dba.FindEdgeOptional(gid2, false); + auto dba = db.Access(); + auto edge = dba->FindEdgeOptional(gid2, false); EXPECT_TRUE(edge); - auto prop = edge->PropsAt(dba.Property("property")); + auto prop = edge->PropsAt(dba->Property("property")); EXPECT_EQ(prop.Value<int64_t>(), 2212); } } diff --git a/tests/unit/typed_value.cpp b/tests/unit/typed_value.cpp index 7d8780443..49297b4b7 100644 --- a/tests/unit/typed_value.cpp +++ b/tests/unit/typed_value.cpp @@ -19,7 +19,7 @@ class AllTypesFixture : public testing::Test { protected: std::vector<TypedValue> values_; database::SingleNode db_; - database::GraphDbAccessor dba_{db_}; + std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()}; void SetUp() override { values_.emplace_back(TypedValue::Null); @@ -35,10 +35,11 @@ class AllTypesFixture : public testing::Test { {"c", 42}, {"d", 0.5}, {"e", TypedValue::Null}}); - auto vertex = dba_.InsertVertex(); + auto vertex = dba_->InsertVertex(); values_.emplace_back(vertex); - values_.emplace_back(dba_.InsertEdge(vertex, vertex, dba_.EdgeType("et"))); - values_.emplace_back(query::Path(dba_.InsertVertex())); + values_.emplace_back( + dba_->InsertEdge(vertex, vertex, dba_->EdgeType("et"))); + values_.emplace_back(query::Path(dba_->InsertVertex())); } }; diff --git a/tools/tests/mg_recovery_check.cpp b/tools/tests/mg_recovery_check.cpp index 1ae6959e1..34326202e 100644 --- a/tools/tests/mg_recovery_check.cpp +++ b/tools/tests/mg_recovery_check.cpp @@ -31,21 +31,21 @@ class RecoveryTest : public ::testing::Test { }; TEST_F(RecoveryTest, TestVerticesRecovered) { - database::GraphDbAccessor dba(db_); - EXPECT_EQ(dba.VerticesCount(), 10); - EXPECT_EQ(dba.VerticesCount(dba.Label("Comment")), 5); - for (const auto &vertex : dba.Vertices(dba.Label("Comment"), false)) { - EXPECT_TRUE(vertex.has_label(dba.Label("Message"))); + auto dba = db_.Access(); + EXPECT_EQ(dba->VerticesCount(), 10); + EXPECT_EQ(dba->VerticesCount(dba->Label("Comment")), 5); + for (const auto &vertex : dba->Vertices(dba->Label("Comment"), false)) { + EXPECT_TRUE(vertex.has_label(dba->Label("Message"))); } - EXPECT_EQ(dba.VerticesCount(dba.Label("Forum")), 5); + EXPECT_EQ(dba->VerticesCount(dba->Label("Forum")), 5); } TEST_F(RecoveryTest, TestPropertyNull) { - database::GraphDbAccessor dba(db_); + auto dba = db_.Access(); bool found = false; - for (const auto &vertex : dba.Vertices(dba.Label("Comment"), false)) { - auto id_prop = query::TypedValue(vertex.PropsAt(dba.Property("id"))); - auto browser = query::TypedValue(vertex.PropsAt(dba.Property("browser"))); + for (const auto &vertex : dba->Vertices(dba->Label("Comment"), false)) { + auto id_prop = query::TypedValue(vertex.PropsAt(dba->Property("id"))); + auto browser = query::TypedValue(vertex.PropsAt(dba->Property("browser"))); if (id_prop.IsString() && id_prop.Value<std::string>() == "2") { EXPECT_FALSE(found); found = true; @@ -58,10 +58,10 @@ TEST_F(RecoveryTest, TestPropertyNull) { } TEST_F(RecoveryTest, TestEdgesRecovered) { - database::GraphDbAccessor dba(db_); - EXPECT_EQ(dba.EdgesCount(), 5); - for (const auto &edge : dba.Edges(false)) { - EXPECT_TRUE(edge.EdgeType() == dba.EdgeType("POSTED_ON")); + auto dba = db_.Access(); + EXPECT_EQ(dba->EdgesCount(), 5); + for (const auto &edge : dba->Edges(false)) { + EXPECT_TRUE(edge.EdgeType() == dba->EdgeType("POSTED_ON")); } }