Separate distributed implementation of GraphDbAccessor

Summary:
GraphDbAccessor is now constructed only through GraphDb. This allows the
concrete GraphDb to instantiate a concrete GraphDbAccessor. This allows
us to use virtual calls, so that the implementation may be kept
separate. The major downside of doing things this way is heap allocation
of GraphDbAccessor. In case it turns out to be a real performance
issues, another solution with pointer to static implementation may be
used.

InsertVertexIntoRemote is now a non-member function, which reduces
coupling. It made no sense for it to be member function because it used
only the public parts of GraphDbAccessor.

Reviewers: msantl, mtomic, mferencevic

Reviewed By: msantl

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D1504
This commit is contained in:
Teon Banek 2018-07-26 09:08:21 +02:00
parent e28fd2025d
commit 9f460914ed
74 changed files with 2279 additions and 2037 deletions
src
tests
tools/tests

View File

@ -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_; }

View File

@ -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

View File

@ -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),

View File

@ -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;

View File

@ -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) {

View File

@ -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

View File

@ -6,7 +6,7 @@
namespace distributed {
BfsRpcClients::BfsRpcClients(database::GraphDb *db,
BfsRpcClients::BfsRpcClients(database::DistributedGraphDb *db,
BfsSubcursorStorage *subcursor_storage,
RpcWorkerClients *clients,
DataManager *data_manager)

View File

@ -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};

View File

@ -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<#)))

View File

@ -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_;

View File

@ -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];
}
}

View File

@ -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_;

View File

@ -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);
});

View File

@ -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

View File

@ -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);
});

View File

@ -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);
});
}

View File

@ -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) {

View File

@ -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.

View File

@ -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.

View File

@ -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 &params,
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";

View File

@ -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

View File

@ -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

View File

@ -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_;

View File

@ -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

View File

@ -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);

View File

@ -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();
}
}
};

View File

@ -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()) {

View File

@ -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) {

View File

@ -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 {

View File

@ -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();
}
}

View File

@ -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();

View File

@ -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>

View File

@ -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 (...) {
}

View File

@ -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);
}
}

View File

@ -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();

View File

@ -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();
};

View File

@ -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;
}

View File

@ -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());

View File

@ -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);

View File

@ -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);

View File

@ -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");

View File

@ -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")));
}
}

View File

@ -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";

View File

@ -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");
}
}

View File

@ -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}});

View File

@ -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());
};

View File

@ -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);
}
}

View File

@ -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);

View File

@ -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);
}

View File

@ -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();

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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, &params, &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, &params, &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;
};

View File

@ -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));

View File

@ -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);
}

View File

@ -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();
}
});
}

View File

@ -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");

View File

@ -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) {

View File

@ -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)));
});
}

View File

@ -19,9 +19,9 @@ class InterpreterTest : public ::testing::Test {
auto Interpret(const std::string &query,
const std::map<std::string, query::TypedValue> &params = {}) {
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;

View File

@ -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))),

View File

@ -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() {

View File

@ -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) {

View File

@ -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();

View File

@ -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");

View File

@ -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();

View File

@ -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.

File diff suppressed because it is too large Load Diff

View File

@ -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;

View File

@ -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();

View File

@ -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});

View File

@ -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);
}
}

View File

@ -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()));
}
};

View File

@ -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"));
}
}