Refactor GraphDb

Summary:
GraphDb is refactored to become an API exposing different parts
necessary for the database to function. These different parts can have
different implementations in SingleNode or distributed Master/Server
GraphDb implementations.

Interally GraphDb is implemented using two class heirarchies. One
contains all the members and correct wiring for each situation. The
other takes care of initialization and shutdown. This architecture is
practical because it can guarantee that the initialization of the
object structure is complete, before initializing state.

Reviewers: buda, mislav.bradac, dgleich, teon.banek

Reviewed By: teon.banek

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D1093
This commit is contained in:
florijan 2018-01-12 15:17:04 +01:00
parent 10d17b4a35
commit 6fc6a27288
97 changed files with 2113 additions and 2193 deletions

View File

@ -25,9 +25,9 @@ DECLARE_int32(gc_cycle_sec);
static const std::string kLabel{"kLabel"};
static const std::string kProperty{"kProperty"};
void GenerateGraph(GraphDb &db) {
void GenerateGraph(database::GraphDb &db) {
{
GraphDbAccessor dba{db};
database::GraphDbAccessor dba{db};
dba.BuildIndex(dba.Label(kLabel), dba.Property(kProperty));
dba.Commit();
}
@ -56,7 +56,7 @@ void GenerateGraph(GraphDb &db) {
SpinLock vertices_lock;
for (int i = 0; i < FLAGS_thread_count; ++i) {
threads.emplace_back([&db, &vertex_ids, &vertices, &vertices_lock, i]() {
GraphDbAccessor dba{db};
database::GraphDbAccessor dba{db};
auto label = dba.Label(kLabel);
auto property = dba.Property(kProperty);
auto batch_size = FLAGS_vertex_count / FLAGS_thread_count;
@ -76,7 +76,7 @@ void GenerateGraph(GraphDb &db) {
<< timer.Elapsed().count() << " seconds.";
}
{
GraphDbAccessor dba{db};
database::GraphDbAccessor dba{db};
for (int i = 0; i < FLAGS_vertex_count; ++i)
vertices[i] = *dba.Transfer(vertices[i]);
@ -96,32 +96,32 @@ void GenerateGraph(GraphDb &db) {
}
}
auto EdgeIteration(GraphDb &db) {
GraphDbAccessor dba{db};
auto EdgeIteration(database::GraphDb &db) {
database::GraphDbAccessor dba{db};
int64_t sum{0};
for (auto edge : dba.Edges(false)) sum += edge.from().gid() + edge.to().gid();
return sum;
}
auto VertexIteration(GraphDb &db) {
GraphDbAccessor dba{db};
auto VertexIteration(database::GraphDb &db) {
database::GraphDbAccessor dba{db};
int64_t sum{0};
for (auto v : dba.Vertices(false))
for (auto e : v.out()) sum += e.gid() + e.to().gid();
return sum;
}
auto ConnectedComponentsEdges(GraphDb &db) {
auto ConnectedComponentsEdges(database::GraphDb &db) {
UnionFind<int64_t> connectivity{FLAGS_vertex_count};
GraphDbAccessor dba{db};
database::GraphDbAccessor dba{db};
for (auto edge : dba.Edges(false))
connectivity.Connect(edge.from().gid(), edge.to().gid());
return connectivity.Size();
}
auto ConnectedComponentsVertices(GraphDb &db) {
auto ConnectedComponentsVertices(database::GraphDb &db) {
UnionFind<int64_t> connectivity{FLAGS_vertex_count};
GraphDbAccessor dba{db};
database::GraphDbAccessor dba{db};
for (auto from : dba.Vertices(false)) {
for (auto out_edge : from.out())
connectivity.Connect(from.gid(), out_edge.to().gid());
@ -129,7 +129,7 @@ auto ConnectedComponentsVertices(GraphDb &db) {
return connectivity.Size();
}
auto ConnectedComponentsVerticesParallel(GraphDb &db) {
auto ConnectedComponentsVerticesParallel(database::GraphDb &db) {
UnionFind<int64_t> connectivity{FLAGS_vertex_count};
SpinLock connectivity_lock;
@ -143,7 +143,7 @@ auto ConnectedComponentsVerticesParallel(GraphDb &db) {
for (int i = 0; i < FLAGS_thread_count; ++i) {
threads.emplace_back(
[&connectivity, &connectivity_lock, &bounds, &db, i]() {
GraphDbAccessor dba{db};
database::GraphDbAccessor dba{db};
for (auto from :
dba.Vertices(dba.Label(kLabel), dba.Property(kProperty),
utils::MakeBoundInclusive(bounds[i]),
@ -159,11 +159,11 @@ auto ConnectedComponentsVerticesParallel(GraphDb &db) {
return connectivity.Size();
}
auto Expansion(GraphDb &db) {
auto Expansion(database::GraphDb &db) {
std::vector<int> component_ids(FLAGS_vertex_count, -1);
int next_component_id{0};
std::stack<VertexAccessor> expansion_stack;
GraphDbAccessor dba{db};
database::GraphDbAccessor dba{db};
for (auto v : dba.Vertices(false)) {
if (component_ids[v.gid()] != -1) continue;
auto component_id = next_component_id++;
@ -186,7 +186,7 @@ int main(int argc, char **argv) {
google::InitGoogleLogging(argv[0]);
FLAGS_gc_cycle_sec = -1;
GraphDb db;
database::SingleNode db;
GenerateGraph(db);
auto timed_call = [&db](auto callable, const std::string &descr) {
LOG(INFO) << "Running " << descr << "...";

View File

@ -9,10 +9,10 @@ set(memgraph_src_files
communication/messaging/protocol.cpp
communication/rpc/rpc.cpp
data_structures/concurrent/skiplist_gc.cpp
database/config.cpp
database/counters.cpp
database/graph_db.cpp
database/graph_db_accessor.cpp
database/graph_db_config.cpp
database/state_delta.cpp
distributed/coordination_master.cpp
distributed/coordination_worker.cpp

View File

@ -26,12 +26,7 @@ namespace communication::bolt {
/** Encapsulates Dbms and Interpreter that are passed through the network server
* and worker to the session. */
struct SessionData {
/** Constructs a SessionData object.
* @param args - Arguments forwarded to the GraphDb constructor. */
template <typename... TArgs>
SessionData(TArgs &&... args) : db(std::forward<TArgs>(args)...) {}
GraphDb db;
database::MasterBase &db;
query::Interpreter interpreter;
};
@ -202,7 +197,7 @@ class Session {
// TODO: Rethink if there is a way to hide some members. At the momement all
// of them are public.
TSocket socket_;
GraphDb &db_;
database::MasterBase &db_;
query::Interpreter &interpreter_;
TimeoutSocket timeout_socket_{*this};
@ -218,7 +213,7 @@ class Session {
State state_{State::Handshake};
// GraphDbAccessor of active transaction in the session, can be null if
// there is no associated transaction.
std::unique_ptr<GraphDbAccessor> db_accessor_;
std::unique_ptr<database::GraphDbAccessor> db_accessor_;
// Time of the last event.
std::chrono::time_point<std::chrono::steady_clock> last_event_time_ =
std::chrono::steady_clock::now();

View File

@ -75,13 +75,14 @@ State HandleRun(TSession &session, State state, Marker marker) {
// TODO: Possible (but very unlikely) race condition, where we have alive
// session during shutdown, but is_accepting_transactions_ isn't yet false.
// We should probably create transactions under some locking mechanism.
if (!session.db_.is_accepting_transactions_) {
if (!session.db_.is_accepting_transactions()) {
// Db is shutting down and doesn't accept new transactions so we should
// close this session.
return State::Close;
}
// Create new transaction.
session.db_accessor_ = std::make_unique<GraphDbAccessor>(session.db_);
session.db_accessor_ =
std::make_unique<database::GraphDbAccessor>(session.db_);
}
// If there was not explicitly started transaction before maybe we are

View File

@ -58,7 +58,7 @@ class System {
friend class Writer;
System(const std::string &address, uint16_t port);
System(const Endpoint &endpoint);
explicit System(const Endpoint &endpoint);
System(const System &) = delete;
System(System &&) = delete;
System &operator=(const System &) = delete;

View File

@ -1,19 +1,8 @@
#include <experimental/filesystem>
#include <iostream>
#include <limits>
#include <gflags/gflags.h>
#include "database/graph_db.hpp"
#include "utils/flag_validation.hpp"
namespace fs = std::experimental::filesystem;
// TODO review: tech docs say the default here is 'true', which it is in the
// community config. Should we set the default here to true? On some other
// points the tech docs are consistent with community config, and not with these
// defaults.
// Durability flags.
DEFINE_bool(durability_enabled, false,
"If durability (database persistence) should be enabled");
@ -29,15 +18,37 @@ DEFINE_int32(snapshot_max_retained, -1,
"Number of retained snapshots, -1 means without limit.");
DEFINE_bool(snapshot_on_exit, false, "Snapshot on exiting the database.");
// Misc flags.
DEFINE_int32(gc_cycle_sec, 30,
"Amount of time between starts of two cleaning cycles in seconds. "
"-1 to turn off.");
// Misc flags
DEFINE_int32(query_execution_time_sec, 180,
"Maximum allowed query execution time. Queries exceeding this "
"limit will be aborted. Value of -1 means no limit.");
DEFINE_int32(gc_cycle_sec, 30,
"Amount of time between starts of two cleaning cycles in seconds. "
"-1 to turn off.");
GraphDb::Config::Config()
// Distributed master/worker flags.
DEFINE_HIDDEN_int32(worker_id, 0,
"ID of a worker in a distributed system. Igored in "
"single-node and distributed-master.");
DEFINE_HIDDEN_string(master_host, "0.0.0.0",
"For master node indicates the host served on. For worker "
"node indicates the master location.");
DEFINE_VALIDATED_HIDDEN_int32(
master_port, 0,
"For master node the port on which to serve. For "
"worker node indicates the master's port.",
FLAG_IN_RANGE(0, std::numeric_limits<uint16_t>::max()));
DEFINE_HIDDEN_string(worker_host, "0.0.0.0",
"For worker node indicates the host served on. For master "
"node this flag is not used.");
DEFINE_VALIDATED_HIDDEN_int32(
worker_port, 0,
"For master node it's unused. For worker node "
"indicates the port on which to serve. If zero (default value), a port is "
"chosen at random. Sent to the master when registring worker node.",
FLAG_IN_RANGE(0, std::numeric_limits<uint16_t>::max()));
database::Config::Config()
// Durability flags.
: durability_enabled{FLAGS_durability_enabled},
durability_directory{FLAGS_durability_directory},
@ -47,4 +58,10 @@ GraphDb::Config::Config()
snapshot_on_exit{FLAGS_snapshot_on_exit},
// Misc flags.
gc_cycle_sec{FLAGS_gc_cycle_sec},
query_execution_time_sec{FLAGS_query_execution_time_sec} {}
query_execution_time_sec{FLAGS_query_execution_time_sec},
// Distributed flags.
worker_id{FLAGS_worker_id},
master_endpoint{FLAGS_master_host,
static_cast<uint16_t>(FLAGS_master_port)},
worker_endpoint{FLAGS_worker_host,
static_cast<uint16_t>(FLAGS_worker_port)} {}

View File

@ -1,95 +1,191 @@
#include <experimental/filesystem>
#include <functional>
#include <glog/logging.h>
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "communication/messaging/distributed.hpp"
#include "distributed/coordination_master.hpp"
#include "distributed/coordination_worker.hpp"
#include "durability/paths.hpp"
#include "durability/recovery.hpp"
#include "durability/snapshooter.hpp"
#include "storage/concurrent_id_mapper_master.hpp"
#include "storage/concurrent_id_mapper_single_node.hpp"
#include "storage/concurrent_id_mapper_worker.hpp"
#include "transactions/engine_master.hpp"
#include "transactions/engine_single_node.hpp"
#include "transactions/engine_worker.hpp"
#include "utils/timer.hpp"
#include "utils/flag_validation.hpp"
namespace fs = std::experimental::filesystem;
namespace database {
namespace impl {
#define INIT_MAPPERS(type, ...) \
labels_ = std::make_unique<type<GraphDbTypes::Label>>(__VA_ARGS__); \
edge_types_ = std::make_unique<type<GraphDbTypes::EdgeType>>(__VA_ARGS__); \
properties_ = std::make_unique<type<GraphDbTypes::Property>>(__VA_ARGS__);
class Base {
public:
explicit Base(const Config &config) : config_(config) {}
virtual ~Base() {}
GraphDb::GraphDb(Config config) : GraphDb(config, 0) {
tx_engine_ = std::make_unique<tx::SingleNodeEngine>(&wal_);
counters_ = std::make_unique<database::SingleNodeCounters>();
INIT_MAPPERS(storage::SingleNodeConcurrentIdMapper);
Start();
}
const Config config_;
GraphDb::GraphDb(communication::messaging::System &system,
distributed::MasterCoordination &master, Config config)
: GraphDb(config, 0) {
tx_engine_ = std::make_unique<tx::MasterEngine>(system, &wal_);
auto counters = std::make_unique<database::MasterCounters>(system);
counters_ = std::move(counters);
INIT_MAPPERS(storage::MasterConcurrentIdMapper, system);
get_endpoint_ = [&master](int worker_id) {
return master.GetEndpoint(worker_id);
};
Start();
}
virtual Storage &storage() = 0;
virtual StorageGc &storage_gc() = 0;
virtual durability::WriteAheadLog &wal() = 0;
virtual tx::Engine &tx_engine() = 0;
virtual storage::ConcurrentIdMapper<Label> &label_mapper() = 0;
virtual storage::ConcurrentIdMapper<EdgeType> &edge_type_mapper() = 0;
virtual storage::ConcurrentIdMapper<Property> &property_mapper() = 0;
virtual database::Counters &counters() = 0;
GraphDb::GraphDb(communication::messaging::System &system, int worker_id,
distributed::WorkerCoordination &worker,
Endpoint master_endpoint, Config config)
: GraphDb(config, worker_id) {
tx_engine_ = std::make_unique<tx::WorkerEngine>(system, master_endpoint);
counters_ =
std::make_unique<database::WorkerCounters>(system, master_endpoint);
INIT_MAPPERS(storage::WorkerConcurrentIdMapper, system, master_endpoint);
get_endpoint_ = [&worker](int worker_id) {
return worker.GetEndpoint(worker_id);
};
Start();
}
Base(const Base &) = delete;
Base(Base &&) = delete;
Base &operator=(const Base &) = delete;
Base &operator=(Base &&) = delete;
};
#undef INIT_MAPPERS
template <template <typename TId> class TMapper>
struct TypemapPack {
template <typename... TMapperArgs>
explicit TypemapPack(TMapperArgs &... args)
: label(args...), edge_type(args...), property(args...) {}
// TODO this should also be garbage collected
TMapper<Label> label;
TMapper<EdgeType> edge_type;
TMapper<Property> property;
};
GraphDb::GraphDb(Config config, int worker_id)
: config_(config),
worker_id_(worker_id),
gc_vertices_(vertices_, vertex_record_deleter_,
vertex_version_list_deleter_),
gc_edges_(edges_, edge_record_deleter_, edge_version_list_deleter_),
wal_{config.durability_directory, config.durability_enabled} {}
#define IMPL_GETTERS \
Storage &storage() override { return storage_; } \
StorageGc &storage_gc() override { return storage_gc_; } \
durability::WriteAheadLog &wal() override { return wal_; } \
tx::Engine &tx_engine() override { return tx_engine_; } \
storage::ConcurrentIdMapper<Label> &label_mapper() override { \
return typemap_pack_.label; \
} \
storage::ConcurrentIdMapper<EdgeType> &edge_type_mapper() override { \
return typemap_pack_.edge_type; \
} \
storage::ConcurrentIdMapper<Property> &property_mapper() override { \
return typemap_pack_.property; \
} \
database::Counters &counters() override { return counters_; }
void GraphDb::Start() {
// Pause of -1 means we shouldn't run the GC.
if (config_.gc_cycle_sec != -1) {
gc_scheduler_.Run(std::chrono::seconds(config_.gc_cycle_sec),
[this]() { CollectGarbage(); });
class SingleNode : public Base {
public:
explicit SingleNode(const Config &config) : Base(config) {}
IMPL_GETTERS
private:
Storage storage_{0};
durability::WriteAheadLog wal_{config_.durability_directory,
config_.durability_enabled};
tx::SingleNodeEngine tx_engine_{&wal_};
StorageGc storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec};
TypemapPack<storage::SingleNodeConcurrentIdMapper> typemap_pack_;
database::SingleNodeCounters counters_;
};
class Master : public Base {
public:
explicit Master(const Config &config) : Base(config) {}
IMPL_GETTERS
private:
communication::messaging::System system_{config_.master_endpoint};
Storage storage_{0};
durability::WriteAheadLog wal_{config_.durability_directory,
config_.durability_enabled};
tx::MasterEngine tx_engine_{system_, &wal_};
StorageGc storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec};
distributed::MasterCoordination coordination{system_};
TypemapPack<storage::MasterConcurrentIdMapper> typemap_pack_{system_};
database::MasterCounters counters_{system_};
};
class Worker : public Base {
public:
explicit Worker(const Config &config) : Base(config) {}
IMPL_GETTERS
void WaitForShutdown() { coordination_.WaitForShutdown(); }
private:
communication::messaging::System system_{config_.worker_endpoint};
distributed::WorkerCoordination coordination_{system_,
config_.master_endpoint};
tx::WorkerEngine tx_engine_{system_, config_.master_endpoint};
Storage storage_{config_.worker_id};
StorageGc storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec};
durability::WriteAheadLog wal_{config_.durability_directory,
config_.durability_enabled};
TypemapPack<storage::WorkerConcurrentIdMapper> typemap_pack_{
system_, config_.master_endpoint};
database::WorkerCounters counters_{system_, config_.master_endpoint};
};
#undef IMPL_GETTERS
} // namespace impl
GraphDb::GraphDb(std::unique_ptr<impl::Base> impl) : impl_(std::move(impl)) {
if (impl_->config_.durability_enabled)
durability::CheckDurabilityDir(impl_->config_.durability_directory);
if (impl_->config_.db_recover_on_startup)
durability::Recover(impl_->config_.durability_directory, *this);
if (impl_->config_.durability_enabled) {
wal().Enable();
snapshot_creator_ = std::make_unique<Scheduler>();
snapshot_creator_->Run(
std::chrono::seconds(impl_->config_.snapshot_cycle_sec),
[this] { MakeSnapshot(); });
}
}
// If snapshots are enabled we need the durability dir.
if (config_.durability_enabled)
durability::CheckDurabilityDir(config_.durability_directory);
GraphDb::~GraphDb() {
snapshot_creator_.release();
if (impl_->config_.snapshot_on_exit) MakeSnapshot();
}
if (config_.db_recover_on_startup)
durability::Recover(config_.durability_directory, *this);
if (config_.durability_enabled) wal_.Enable();
StartSnapshooting();
Storage &GraphDb::storage() { return impl_->storage(); }
if (config_.query_execution_time_sec != -1) {
durability::WriteAheadLog &GraphDb::wal() { return impl_->wal(); }
tx::Engine &GraphDb::tx_engine() { return impl_->tx_engine(); }
storage::ConcurrentIdMapper<Label> &GraphDb::label_mapper() {
return impl_->label_mapper();
}
storage::ConcurrentIdMapper<EdgeType> &GraphDb::edge_type_mapper() {
return impl_->edge_type_mapper();
}
storage::ConcurrentIdMapper<Property> &GraphDb::property_mapper() {
return impl_->property_mapper();
}
database::Counters &GraphDb::counters() { return impl_->counters(); }
void GraphDb::CollectGarbage() { impl_->storage_gc().CollectGarbage(); }
void GraphDb::MakeSnapshot() {
const bool status = durability::MakeSnapshot(
*this, fs::path(impl_->config_.durability_directory),
impl_->config_.snapshot_max_retained);
if (status) {
LOG(INFO) << "Snapshot created successfully." << std::endl;
} else {
LOG(ERROR) << "Snapshot creation failed!" << std::endl;
}
}
MasterBase::MasterBase(std::unique_ptr<impl::Base> impl)
: GraphDb(std::move(impl)) {
if (impl_->config_.query_execution_time_sec != -1) {
transaction_killer_.Run(
std::chrono::seconds(
std::max(1, std::min(5, config_.query_execution_time_sec / 4))),
std::chrono::seconds(std::max(
1, std::min(5, impl_->config_.query_execution_time_sec / 4))),
[this]() {
tx_engine_->LocalForEachActiveTransaction([this](tx::Transaction &t) {
tx_engine().LocalForEachActiveTransaction([this](tx::Transaction &t) {
if (t.creation_time() +
std::chrono::seconds(config_.query_execution_time_sec) <
std::chrono::seconds(
impl_->config_.query_execution_time_sec) <
std::chrono::steady_clock::now()) {
t.set_should_abort();
};
@ -98,109 +194,16 @@ void GraphDb::Start() {
}
}
void GraphDb::Shutdown() {
is_accepting_transactions_ = false;
tx_engine_->LocalForEachActiveTransaction(
[](auto &t) { t.set_should_abort(); });
}
void GraphDb::StartSnapshooting() {
if (config_.durability_enabled) {
auto create_snapshot = [this]() -> void {
if (!durability::MakeSnapshot(*this,
fs::path(config_.durability_directory),
config_.snapshot_max_retained)) {
LOG(WARNING) << "Durability: snapshot creation failed";
}
};
snapshot_creator_.Run(std::chrono::seconds(config_.snapshot_cycle_sec),
create_snapshot);
}
}
void GraphDb::CollectGarbage() {
// main garbage collection logic
// see wiki documentation for logic explanation
LOG(INFO) << "Garbage collector started";
const auto snapshot = tx_engine_->GlobalGcSnapshot();
{
// This can be run concurrently
utils::Timer x;
gc_vertices_.Run(snapshot, *tx_engine_);
gc_edges_.Run(snapshot, *tx_engine_);
VLOG(1) << "Garbage collector mvcc phase time: " << x.Elapsed().count();
}
// This has to be run sequentially after gc because gc modifies
// version_lists and changes the oldest visible record, on which Refresh
// depends.
{
// This can be run concurrently
utils::Timer x;
labels_index_.Refresh(snapshot, *tx_engine_);
label_property_index_.Refresh(snapshot, *tx_engine_);
VLOG(1) << "Garbage collector index phase time: " << x.Elapsed().count();
}
{
// We free expired objects with snapshot.back(), which is
// the ID of the oldest active transaction (or next active, if there
// are no currently active). That's legal because that was the
// last possible transaction that could have obtained pointers
// to those records. New snapshot can be used, different than one used for
// first two phases of gc.
utils::Timer x;
const auto snapshot = tx_engine_->GlobalGcSnapshot();
edge_record_deleter_.FreeExpiredObjects(snapshot.back());
vertex_record_deleter_.FreeExpiredObjects(snapshot.back());
edge_version_list_deleter_.FreeExpiredObjects(snapshot.back());
vertex_version_list_deleter_.FreeExpiredObjects(snapshot.back());
VLOG(1) << "Garbage collector deferred deletion phase time: "
<< x.Elapsed().count();
}
LOG(INFO) << "Garbage collector finished";
VLOG(2) << "gc snapshot: " << snapshot;
VLOG(2) << "edge_record_deleter_ size: " << edge_record_deleter_.Count();
VLOG(2) << "vertex record deleter_ size: " << vertex_record_deleter_.Count();
VLOG(2) << "edge_version_list_deleter_ size: "
<< edge_version_list_deleter_.Count();
VLOG(2) << "vertex_version_list_deleter_ size: "
<< vertex_version_list_deleter_.Count();
VLOG(2) << "vertices_ size: " << vertices_.access().size();
VLOG(2) << "edges_ size: " << edges_.access().size();
}
GraphDb::~GraphDb() {
// Stop the gc scheduler to not run into race conditions for deletions.
gc_scheduler_.Stop();
// Stop the snapshot creator to avoid snapshooting while database is being
// deleted.
snapshot_creator_.Stop();
// Stop transaction killer.
transaction_killer_.Stop();
// Create last database snapshot
if (config_.snapshot_on_exit == true) {
LOG(INFO) << "Creating snapshot on shutdown..." << std::endl;
const bool status =
durability::MakeSnapshot(*this, fs::path(config_.durability_directory),
config_.snapshot_max_retained);
if (status) {
std::cout << "Snapshot created successfully." << std::endl;
} else {
LOG(ERROR) << "Snapshot creation failed!" << std::endl;
}
}
// Delete vertices and edges which weren't collected before, also deletes
// records inside version list
for (auto &id_vlist : vertices_.access()) delete id_vlist.second;
for (auto &id_vlist : edges_.access()) delete id_vlist.second;
// Free expired records with the maximal possible id from all the deleters.
edge_record_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
vertex_record_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
edge_version_list_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
vertex_version_list_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
SingleNode::SingleNode(Config config)
: MasterBase(std::make_unique<impl::SingleNode>(config)) {}
Master::Master(Config config)
: MasterBase(std::make_unique<impl::Master>(config)) {}
Worker::Worker(Config config)
: GraphDb(std::make_unique<impl::Worker>(config)) {}
void Worker::WaitForShutdown() {
dynamic_cast<impl::Worker *>(impl_.get())->WaitForShutdown();
}
} // namespace database

View File

@ -1,44 +1,56 @@
#pragma once
#include <atomic>
#include <memory>
#include <mutex>
#include "cppitertools/filter.hpp"
#include "cppitertools/imap.hpp"
#include "gflags/gflags.h"
#include "glog/logging.h"
#include "data_structures/concurrent/concurrent_map.hpp"
#include "data_structures/concurrent/concurrent_set.hpp"
#include "database/counters.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/indexes/key_index.hpp"
#include "database/indexes/label_property_index.hpp"
#include "distributed/coordination_master.hpp"
#include "distributed/coordination_worker.hpp"
#include "database/storage.hpp"
#include "database/storage_gc.hpp"
#include "database/types.hpp"
#include "durability/wal.hpp"
#include "io/network/network_endpoint.hpp"
#include "mvcc/version_list.hpp"
#include "storage/concurrent_id_mapper.hpp"
#include "storage/concurrent_id_mapper_master.hpp"
#include "storage/concurrent_id_mapper_single_node.hpp"
#include "storage/concurrent_id_mapper_worker.hpp"
#include "storage/deferred_deleter.hpp"
#include "storage/edge.hpp"
#include "storage/garbage_collector.hpp"
#include "storage/vertex.hpp"
#include "transactions/engine.hpp"
#include "utils/scheduler.hpp"
namespace database {
/// Database configuration. Initialized from flags, but modifiable.
struct Config {
Config();
// Durability flags.
bool durability_enabled;
std::string durability_directory;
bool db_recover_on_startup;
int snapshot_cycle_sec;
int snapshot_max_retained;
int snapshot_on_exit;
// Misc flags.
int gc_cycle_sec;
int query_execution_time_sec;
// Distributed master/worker flags.
int worker_id;
io::network::NetworkEndpoint master_endpoint;
io::network::NetworkEndpoint worker_endpoint;
};
namespace impl {
class Base;
}
/**
* Main class which represents Database concept in code.
* This class is essentially a data structure. It exposes
* all the data publicly, and should therefore not be directly
* exposed to client functions. The GraphDbAccessor is used for that.
* 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 db.
* GraphDb db; // KeyIndex is created as a part of database::Storage
* int main() {
* GraphDbAccessor dba(db);
* auto v = dba.InsertVertex();
@ -55,127 +67,58 @@
* -> CRASH
*/
class GraphDb {
using Endpoint = io::network::NetworkEndpoint;
public:
/// GraphDb configuration. Initialized from flags, but modifiable.
struct Config {
Config();
// Durability flags.
bool durability_enabled;
std::string durability_directory;
bool db_recover_on_startup;
int snapshot_cycle_sec;
int snapshot_max_retained;
int snapshot_on_exit;
// Misc flags.
int gc_cycle_sec;
int query_execution_time_sec;
};
/** Single-node GraphDb ctor. */
explicit GraphDb(Config config = Config{});
/** Distributed master GraphDb ctor. */
GraphDb(communication::messaging::System &system,
distributed::MasterCoordination &master, Config config = Config());
/** Distributed worker GraphDb ctor. */
GraphDb(communication::messaging::System &system, int worker_id,
distributed::WorkerCoordination &worker, Endpoint master_endpoint,
Config config = Config());
private:
// Private ctor used by other ctors. */
GraphDb(Config config, int worker_id);
public:
/** Delete all vertices and edges and free all deferred deleters. */
~GraphDb();
GraphDb(const GraphDb &db) = delete;
GraphDb(GraphDb &&other) = delete;
GraphDb &operator=(const GraphDb &other) = delete;
GraphDb &operator=(GraphDb &&other) = delete;
/** Stop all transactions and set is_accepting_transactions_ to false. */
void Shutdown();
Storage &storage();
durability::WriteAheadLog &wal();
tx::Engine &tx_engine();
storage::ConcurrentIdMapper<Label> &label_mapper();
storage::ConcurrentIdMapper<EdgeType> &edge_type_mapper();
storage::ConcurrentIdMapper<Property> &property_mapper();
database::Counters &counters();
void CollectGarbage();
gid::Generator &VertexGenerator() { return vertex_generator_; }
gid::Generator &EdgeGenerator() { return edge_generator_; }
protected:
explicit GraphDb(std::unique_ptr<impl::Base> impl);
virtual ~GraphDb();
/** When this is false, no new transactions should be created. */
std::atomic<bool> is_accepting_transactions_{true};
std::unique_ptr<impl::Base> impl_;
private:
friend class GraphDbAccessor;
std::unique_ptr<Scheduler> snapshot_creator_;
Config config_;
/** Transaction engine related to this database. Master instance if this
* GraphDb is a single-node deployment, or the master in a distributed system.
* Otherwise a WorkerEngine instance. */
std::unique_ptr<tx::Engine> tx_engine_;
int worker_id_{0};
gid::Generator vertex_generator_{worker_id_};
gid::Generator edge_generator_{worker_id_};
// main storage for the graph
ConcurrentMap<gid::Gid, mvcc::VersionList<Vertex> *> vertices_;
ConcurrentMap<gid::Gid, mvcc::VersionList<Edge> *> edges_;
// Garbage collectors
GarbageCollector<ConcurrentMap<gid::Gid, mvcc::VersionList<Vertex> *>, Vertex>
gc_vertices_;
GarbageCollector<ConcurrentMap<gid::Gid, mvcc::VersionList<Edge> *>, Edge>
gc_edges_;
// Deleters for not relevant records
DeferredDeleter<Vertex> vertex_record_deleter_;
DeferredDeleter<Edge> edge_record_deleter_;
// Deleters for not relevant version_lists
DeferredDeleter<mvcc::VersionList<Vertex>> vertex_version_list_deleter_;
DeferredDeleter<mvcc::VersionList<Edge>> edge_version_list_deleter_;
// Id to value mappers.
// TODO this should be also garbage collected
std::unique_ptr<storage::ConcurrentIdMapper<GraphDbTypes::Label>> labels_;
std::unique_ptr<storage::ConcurrentIdMapper<GraphDbTypes::EdgeType>>
edge_types_;
std::unique_ptr<storage::ConcurrentIdMapper<GraphDbTypes::Property>>
properties_;
// indexes
KeyIndex<GraphDbTypes::Label, Vertex> labels_index_;
LabelPropertyIndex label_property_index_;
// Set of transactions ids which are building indexes currently
ConcurrentSet<tx::transaction_id_t> index_build_tx_in_progress_;
durability::WriteAheadLog wal_;
// Schedulers
Scheduler gc_scheduler_;
Scheduler snapshot_creator_;
// Periodically wakes up and hints to transactions that are running for a long
// time to stop their execution.
Scheduler transaction_killer_;
// DB level global counters, used in the "counter" function.
std::unique_ptr<database::Counters> counters_;
// Returns Endpoint info for worker ID. Different implementation in master vs.
// worker. Unused in single-node version.
std::function<io::network::NetworkEndpoint(int)> get_endpoint_;
// Starts DB operations once all members have been constructed.
void Start();
// Starts periodically generating database snapshots.
void StartSnapshooting();
void MakeSnapshot();
};
class MasterBase : public GraphDb {
public:
explicit MasterBase(std::unique_ptr<impl::Base> impl);
bool is_accepting_transactions() const { return is_accepting_transactions_; }
~MasterBase() {
is_accepting_transactions_ = false;
tx_engine().LocalForEachActiveTransaction(
[](auto &t) { t.set_should_abort(); });
}
private:
/** When this is false, no new transactions should be created. */
std::atomic<bool> is_accepting_transactions_{true};
Scheduler transaction_killer_;
};
class SingleNode : public MasterBase {
public:
explicit SingleNode(Config config = Config());
};
class Master : public MasterBase {
public:
explicit Master(Config config = Config());
};
class Worker : public GraphDb {
public:
explicit Worker(Config config = Config());
void WaitForShutdown();
};
} // namespace database

View File

@ -9,8 +9,10 @@
#include "utils/atomic.hpp"
#include "utils/on_scope_exit.hpp"
namespace database {
GraphDbAccessor::GraphDbAccessor(GraphDb &db)
: db_(db), transaction_(SingleNodeEngine().Begin()) {}
: db_(db), transaction_(*SingleNodeEngine().Begin()) {}
GraphDbAccessor::~GraphDbAccessor() {
if (!commited_ && !aborted_) {
@ -19,51 +21,52 @@ GraphDbAccessor::~GraphDbAccessor() {
}
tx::transaction_id_t GraphDbAccessor::transaction_id() const {
return transaction_->id_;
return transaction_.id_;
}
void GraphDbAccessor::AdvanceCommand() {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
SingleNodeEngine().Advance(transaction_->id_);
SingleNodeEngine().Advance(transaction_.id_);
}
void GraphDbAccessor::Commit() {
DCHECK(!commited_ && !aborted_) << "Already aborted or commited transaction.";
SingleNodeEngine().Commit(*transaction_);
SingleNodeEngine().Commit(transaction_);
commited_ = true;
}
void GraphDbAccessor::Abort() {
DCHECK(!commited_ && !aborted_) << "Already aborted or commited transaction.";
SingleNodeEngine().Abort(*transaction_);
SingleNodeEngine().Abort(transaction_);
aborted_ = true;
}
bool GraphDbAccessor::should_abort() const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return transaction_->should_abort();
return transaction_.should_abort();
}
durability::WriteAheadLog &GraphDbAccessor::wal() { return db_.wal_; }
durability::WriteAheadLog &GraphDbAccessor::wal() { return db_.wal(); }
VertexAccessor GraphDbAccessor::InsertVertex(
std::experimental::optional<gid::Gid> requested_gid) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
auto gid = db_.vertex_generator_.Next(requested_gid);
auto vertex_vlist = new mvcc::VersionList<Vertex>(*transaction_, gid);
auto gid = db_.storage().vertex_generator_.Next(requested_gid);
auto vertex_vlist = new mvcc::VersionList<Vertex>(transaction_, gid);
bool success = db_.vertices_.access().insert(gid, vertex_vlist).second;
bool success =
db_.storage().vertices_.access().insert(gid, vertex_vlist).second;
CHECK(success) << "Attempting to insert a vertex with an existing GID: "
<< gid;
db_.wal_.Emplace(database::StateDelta::CreateVertex(transaction_->id_,
vertex_vlist->gid_));
wal().Emplace(
database::StateDelta::CreateVertex(transaction_.id_, vertex_vlist->gid_));
return VertexAccessor(vertex_vlist, *this);
}
std::experimental::optional<VertexAccessor> GraphDbAccessor::FindVertex(
gid::Gid gid, bool current_state) {
auto collection_accessor = db_.vertices_.access();
auto collection_accessor = db_.storage().vertices_.access();
auto found = collection_accessor.find(gid);
if (found == collection_accessor.end()) return std::experimental::nullopt;
VertexAccessor record_accessor(found->second, *this);
@ -74,7 +77,7 @@ std::experimental::optional<VertexAccessor> GraphDbAccessor::FindVertex(
std::experimental::optional<EdgeAccessor> GraphDbAccessor::FindEdge(
gid::Gid gid, bool current_state) {
auto collection_accessor = db_.edges_.access();
auto collection_accessor = db_.storage().edges_.access();
auto found = collection_accessor.find(gid);
if (found == collection_accessor.end()) return std::experimental::nullopt;
EdgeAccessor record_accessor(found->second, *this);
@ -83,22 +86,22 @@ std::experimental::optional<EdgeAccessor> GraphDbAccessor::FindEdge(
return record_accessor;
}
void GraphDbAccessor::BuildIndex(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property) {
void GraphDbAccessor::BuildIndex(const class Label &label,
const class Property &property) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
db_.index_build_tx_in_progress_.access().insert(transaction_->id_);
db_.storage().index_build_tx_in_progress_.access().insert(transaction_.id_);
// on function exit remove the create index transaction from
// build_tx_in_progress
utils::OnScopeExit on_exit_1([this] {
auto removed =
db_.index_build_tx_in_progress_.access().remove(transaction_->id_);
auto removed = db_.storage().index_build_tx_in_progress_.access().remove(
transaction_.id_);
DCHECK(removed) << "Index creation transaction should be inside set";
});
const LabelPropertyIndex::Key key(label, property);
if (db_.label_property_index_.CreateIndex(key) == false) {
if (db_.storage().label_property_index_.CreateIndex(key) == false) {
throw IndexExistsException(
"Index is either being created by another transaction or already "
"exists.");
@ -109,12 +112,12 @@ void GraphDbAccessor::BuildIndex(const GraphDbTypes::Label &label,
// happened earlier. We have to first wait for every transaction that
// happend before, or a bit later than CreateIndex to end.
{
auto wait_transactions = db_.tx_engine_->GlobalActiveTransactions();
auto wait_transactions = transaction_.engine_.GlobalActiveTransactions();
auto active_index_creation_transactions =
db_.index_build_tx_in_progress_.access();
db_.storage().index_build_tx_in_progress_.access();
for (auto id : wait_transactions) {
if (active_index_creation_transactions.contains(id)) continue;
while (db_.tx_engine_->GlobalIsActive(id)) {
while (transaction_.engine_.GlobalIsActive(id)) {
// Active index creation set could only now start containing that id,
// since that thread could have not written to the set set and to avoid
// dead-lock we need to make sure we keep track of that
@ -132,17 +135,18 @@ void GraphDbAccessor::BuildIndex(const GraphDbTypes::Label &label,
// 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_;
db_.index_build_tx_in_progress_.access().insert(read_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
utils::OnScopeExit on_exit_2([read_transaction_id, this] {
auto removed =
db_.index_build_tx_in_progress_.access().remove(read_transaction_id);
auto removed = db_.storage().index_build_tx_in_progress_.access().remove(
read_transaction_id);
DCHECK(removed) << "Index building (read) transaction should be inside set";
});
for (auto vertex : dba.Vertices(label, false)) {
db_.label_property_index_.UpdateOnLabelProperty(vertex.address().local(),
vertex.current_);
db_.storage().label_property_index_.UpdateOnLabelProperty(
vertex.address().local(), vertex.current_);
}
// Commit transaction as we finished applying method on newest visible
// records. Write that transaction's ID to the WAL as the index has been
@ -150,70 +154,74 @@ void GraphDbAccessor::BuildIndex(const GraphDbTypes::Label &label,
// reason.
auto wal_build_index_tx_id = dba.transaction_id();
dba.Commit();
db_.wal_.Emplace(database::StateDelta::BuildIndex(
wal().Emplace(database::StateDelta::BuildIndex(
wal_build_index_tx_id, LabelName(label), PropertyName(property)));
// After these two operations we are certain that everything is contained in
// the index under the assumption that this transaction contained no
// vertex/edge insert/update before this method was invoked.
db_.label_property_index_.IndexFinishedBuilding(key);
db_.storage().label_property_index_.IndexFinishedBuilding(key);
}
void GraphDbAccessor::UpdateLabelIndices(const GraphDbTypes::Label &label,
void GraphDbAccessor::UpdateLabelIndices(const class Label &label,
const VertexAccessor &vertex_accessor,
const Vertex *const vertex) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(vertex_accessor.is_local()) << "Only local vertices belong in indexes";
auto *vlist_ptr = vertex_accessor.address().local();
db_.labels_index_.Update(label, vlist_ptr, vertex);
db_.label_property_index_.UpdateOnLabel(label, vlist_ptr, vertex);
db_.storage().labels_index_.Update(label, vlist_ptr, vertex);
db_.storage().label_property_index_.UpdateOnLabel(label, vlist_ptr, vertex);
}
void GraphDbAccessor::UpdatePropertyIndex(
const GraphDbTypes::Property &property,
const class Property &property,
const RecordAccessor<Vertex> &vertex_accessor, const Vertex *const vertex) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(vertex_accessor.is_local()) << "Only local vertices belong in indexes";
db_.label_property_index_.UpdateOnProperty(
db_.storage().label_property_index_.UpdateOnProperty(
property, vertex_accessor.address().local(), vertex);
}
int64_t GraphDbAccessor::VerticesCount() const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.vertices_.access().size();
return db_.storage().vertices_.access().size();
}
int64_t GraphDbAccessor::VerticesCount(const GraphDbTypes::Label &label) const {
int64_t GraphDbAccessor::VerticesCount(const class Label &label) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.labels_index_.Count(label);
return db_.storage().labels_index_.Count(label);
}
int64_t GraphDbAccessor::VerticesCount(
const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property) const {
int64_t GraphDbAccessor::VerticesCount(const class Label &label,
const class Property &property) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
const LabelPropertyIndex::Key key(label, property);
DCHECK(db_.label_property_index_.IndexExists(key)) << "Index doesn't exist.";
return db_.label_property_index_.Count(key);
DCHECK(db_.storage().label_property_index_.IndexExists(key))
<< "Index doesn't exist.";
return db_.storage().label_property_index_.Count(key);
}
int64_t GraphDbAccessor::VerticesCount(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property,
int64_t GraphDbAccessor::VerticesCount(const class Label &label,
const class Property &property,
const PropertyValue &value) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
const LabelPropertyIndex::Key key(label, property);
DCHECK(db_.label_property_index_.IndexExists(key)) << "Index doesn't exist.";
return db_.label_property_index_.PositionAndCount(key, value).second;
DCHECK(db_.storage().label_property_index_.IndexExists(key))
<< "Index doesn't exist.";
return db_.storage()
.label_property_index_.PositionAndCount(key, value)
.second;
}
int64_t GraphDbAccessor::VerticesCount(
const GraphDbTypes::Label &label, const GraphDbTypes::Property &property,
const class Label &label, const class Property &property,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper)
const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
const LabelPropertyIndex::Key key(label, property);
DCHECK(db_.label_property_index_.IndexExists(key)) << "Index doesn't exist.";
DCHECK(db_.storage().label_property_index_.IndexExists(key))
<< "Index doesn't exist.";
CHECK(lower || upper) << "At least one bound must be provided";
CHECK(!lower || lower.value().value().type() != PropertyValue::Type::Null)
<< "Null value is not a valid index bound";
@ -221,24 +229,24 @@ int64_t GraphDbAccessor::VerticesCount(
<< "Null value is not a valid index bound";
if (!upper) {
auto lower_pac =
db_.label_property_index_.PositionAndCount(key, lower.value().value());
int64_t size = db_.label_property_index_.Count(key);
auto lower_pac = db_.storage().label_property_index_.PositionAndCount(
key, lower.value().value());
int64_t size = db_.storage().label_property_index_.Count(key);
return std::max(0l,
size - lower_pac.first -
(lower.value().IsInclusive() ? 0l : lower_pac.second));
} else if (!lower) {
auto upper_pac =
db_.label_property_index_.PositionAndCount(key, upper.value().value());
auto upper_pac = db_.storage().label_property_index_.PositionAndCount(
key, upper.value().value());
return upper.value().IsInclusive() ? upper_pac.first + upper_pac.second
: upper_pac.first;
} else {
auto lower_pac =
db_.label_property_index_.PositionAndCount(key, lower.value().value());
auto upper_pac =
db_.label_property_index_.PositionAndCount(key, upper.value().value());
auto lower_pac = db_.storage().label_property_index_.PositionAndCount(
key, lower.value().value());
auto upper_pac = db_.storage().label_property_index_.PositionAndCount(
key, upper.value().value());
auto result = upper_pac.first - lower_pac.first;
if (lower.value().IsExclusive()) result -= lower_pac.second;
if (upper.value().IsInclusive()) result += upper_pac.second;
@ -262,14 +270,14 @@ bool GraphDbAccessor::RemoveVertex(VertexAccessor &vertex_accessor) {
// it's possible the vertex 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
if (vertex_accessor.current().is_expired_by(*transaction_)) return true;
if (vertex_accessor.current().is_expired_by(transaction_)) return true;
if (vertex_accessor.out_degree() > 0 || vertex_accessor.in_degree() > 0)
return false;
auto *vlist_ptr = vertex_accessor.address().local();
db_.wal_.Emplace(
database::StateDelta::RemoveVertex(transaction_->id_, vlist_ptr->gid_));
vlist_ptr->remove(vertex_accessor.current_, *transaction_);
wal().Emplace(
database::StateDelta::RemoveVertex(transaction_.id_, vlist_ptr->gid_));
vlist_ptr->remove(vertex_accessor.current_, transaction_);
return true;
}
@ -292,13 +300,13 @@ void GraphDbAccessor::DetachRemoveVertex(VertexAccessor &vertex_accessor) {
// it's possible the vertex 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
if (!vertex_accessor.current().is_expired_by(*transaction_))
if (!vertex_accessor.current().is_expired_by(transaction_))
vertex_accessor.address().local()->remove(vertex_accessor.current_,
*transaction_);
transaction_);
}
EdgeAccessor GraphDbAccessor::InsertEdge(
VertexAccessor &from, VertexAccessor &to, GraphDbTypes::EdgeType edge_type,
VertexAccessor &from, VertexAccessor &to, class EdgeType edge_type,
std::experimental::optional<gid::Gid> requested_gid) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
// An edge is created on the worker of it's "from" vertex.
@ -309,13 +317,13 @@ EdgeAccessor GraphDbAccessor::InsertEdge(
// EdgeAccessor and return it. The remote InsertEdge(...) will be calling
// remote Connect(...) if "to" is not local to it.
}
auto gid = db_.edge_generator_.Next(requested_gid);
auto gid = db_.storage().edge_generator_.Next(requested_gid);
auto edge_vlist = new mvcc::VersionList<Edge>(
*transaction_, gid, from.address(), to.address(), edge_type);
transaction_, gid, from.address(), to.address(), edge_type);
// We need to insert edge_vlist to edges_ before calling update since update
// can throw and edge_vlist will not be garbage collected if it is not in
// edges_ skiplist.
bool success = db_.edges_.access().insert(gid, edge_vlist).second;
bool success = db_.storage().edges_.access().insert(gid, edge_vlist).second;
CHECK(success) << "Attempting to insert an edge with an existing GID: "
<< gid;
@ -335,8 +343,8 @@ EdgeAccessor GraphDbAccessor::InsertEdge(
// TODO call remote Connect(from_gid, edge_gid, to_gid, edge_type). Possible
// outcomes are success or error (serialization, timeout).
}
db_.wal_.Emplace(database::StateDelta::CreateEdge(
transaction_->id_, edge_vlist->gid_, from.gid(), to.gid(),
wal().Emplace(database::StateDelta::CreateEdge(
transaction_.id_, edge_vlist->gid_, from.gid(), to.gid(),
EdgeTypeName(edge_type)));
return EdgeAccessor(edge_vlist, *this, from.address(), to.address(),
edge_type);
@ -344,7 +352,7 @@ EdgeAccessor GraphDbAccessor::InsertEdge(
int64_t GraphDbAccessor::EdgesCount() const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.edges_.access().size();
return db_.storage().edges_.access().size();
}
void GraphDbAccessor::RemoveEdge(EdgeAccessor &edge_accessor,
@ -360,66 +368,63 @@ void GraphDbAccessor::RemoveEdge(EdgeAccessor &edge_accessor,
// due to it getting matched multiple times by some patterns
// we can only delete it once, so check if it's already deleted
edge_accessor.SwitchNew();
if (edge_accessor.current().is_expired_by(*transaction_)) return;
if (edge_accessor.current().is_expired_by(transaction_)) return;
if (remove_from_from)
edge_accessor.from().update().out_.RemoveEdge(edge_accessor.address());
if (remove_from_to)
edge_accessor.to().update().in_.RemoveEdge(edge_accessor.address());
edge_accessor.address().local()->remove(edge_accessor.current_,
*transaction_);
db_.wal_.Emplace(
database::StateDelta::RemoveEdge(transaction_->id_, edge_accessor.gid()));
edge_accessor.address().local()->remove(edge_accessor.current_, transaction_);
wal().Emplace(
database::StateDelta::RemoveEdge(transaction_.id_, edge_accessor.gid()));
}
GraphDbTypes::Label GraphDbAccessor::Label(const std::string &label_name) {
Label GraphDbAccessor::Label(const std::string &label_name) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.labels_->value_to_id(label_name);
return db_.label_mapper().value_to_id(label_name);
}
const std::string &GraphDbAccessor::LabelName(
const GraphDbTypes::Label label) const {
const std::string &GraphDbAccessor::LabelName(const class Label label) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.labels_->id_to_value(label);
return db_.label_mapper().id_to_value(label);
}
GraphDbTypes::EdgeType GraphDbAccessor::EdgeType(
const std::string &edge_type_name) {
EdgeType GraphDbAccessor::EdgeType(const std::string &edge_type_name) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.edge_types_->value_to_id(edge_type_name);
return db_.edge_type_mapper().value_to_id(edge_type_name);
}
const std::string &GraphDbAccessor::EdgeTypeName(
const GraphDbTypes::EdgeType edge_type) const {
const class EdgeType edge_type) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.edge_types_->id_to_value(edge_type);
return db_.edge_type_mapper().id_to_value(edge_type);
}
GraphDbTypes::Property GraphDbAccessor::Property(
const std::string &property_name) {
Property GraphDbAccessor::Property(const std::string &property_name) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.properties_->value_to_id(property_name);
return db_.property_mapper().value_to_id(property_name);
}
const std::string &GraphDbAccessor::PropertyName(
const GraphDbTypes::Property property) const {
const class Property property) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.properties_->id_to_value(property);
return db_.property_mapper().id_to_value(property);
}
int64_t GraphDbAccessor::Counter(const std::string &name) {
return db_.counters_->Get(name);
return db_.counters().Get(name);
}
void GraphDbAccessor::CounterSet(const std::string &name, int64_t value) {
db_.counters_->Set(name, value);
db_.counters().Set(name, value);
}
std::vector<std::string> GraphDbAccessor::IndexInfo() const {
std::vector<std::string> info;
for (GraphDbTypes::Label label : db_.labels_index_.Keys()) {
for (class Label label : db_.storage().labels_index_.Keys()) {
info.emplace_back(":" + LabelName(label));
}
for (LabelPropertyIndex::Key key : db_.label_property_index_.Keys()) {
for (LabelPropertyIndex::Key key :
db_.storage().label_property_index_.Keys()) {
info.emplace_back(fmt::format(":{}({})", LabelName(key.label_),
PropertyName(key.property_)));
}
@ -437,3 +442,4 @@ template <>
GraphDbAccessor::RemoteCache<Edge> &GraphDbAccessor::remote_elements() {
return remote_edges();
}
} // namespace database

View File

@ -7,13 +7,16 @@
#include "cppitertools/imap.hpp"
#include "glog/logging.h"
#include "graph_db.hpp"
#include "database/graph_db.hpp"
#include "database/types.hpp"
#include "storage/edge_accessor.hpp"
#include "storage/vertex_accessor.hpp"
#include "transactions/engine_master.hpp"
#include "transactions/engine_single_node.hpp"
#include "transactions/transaction.hpp"
#include "utils/bound.hpp"
namespace database {
/** Thrown when creating an index which already exists. */
class IndexExistsException : public utils::BasicException {
using utils::BasicException::BasicException;
@ -29,10 +32,10 @@ class IndexExistsException : public utils::BasicException {
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;
friend class ::RecordAccessor<Vertex>;
friend class ::RecordAccessor<Edge>;
friend class ::VertexAccessor;
friend class ::EdgeAccessor;
/**
* Used for caching Vertices and Edges that are stored on another worker in a
@ -89,11 +92,6 @@ class GraphDbAccessor {
};
public:
/**
* Creates an accessor for the given database.
*
* @param db The database
*/
explicit GraphDbAccessor(GraphDb &db);
~GraphDbAccessor();
@ -171,7 +169,7 @@ class GraphDbAccessor {
[this](auto id_vlist) {
return VertexAccessor(id_vlist.second, *this);
},
db_.vertices_.access());
db_.storage().vertices_.access());
// filter out the accessors not visible to the current transaction
return iter::filter(
@ -191,11 +189,12 @@ class GraphDbAccessor {
* ignored).
* @return iterable collection
*/
auto Vertices(const GraphDbTypes::Label &label, bool current_state) {
auto Vertices(const Label &label, bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return iter::imap(
[this](auto vlist) { return VertexAccessor(vlist, *this); },
db_.labels_index_.GetVlists(label, *transaction_, current_state));
db_.storage().labels_index_.GetVlists(label, transaction_,
current_state));
}
/**
@ -210,16 +209,16 @@ class GraphDbAccessor {
* ignored).
* @return iterable collection
*/
auto Vertices(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property, bool current_state) {
auto Vertices(const Label &label, const Property &property,
bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(db_.label_property_index_.IndexExists(
DCHECK(db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property)))
<< "Label+property index doesn't exist.";
return iter::imap(
[this](auto vlist) { return VertexAccessor(vlist, *this); },
db_.label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), *transaction_,
db_.storage().label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), transaction_,
current_state));
}
@ -236,19 +235,18 @@ class GraphDbAccessor {
* ignored).
* @return iterable collection
*/
auto Vertices(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property,
auto Vertices(const Label &label, const Property &property,
const PropertyValue &value, bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(db_.label_property_index_.IndexExists(
DCHECK(db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property)))
<< "Label+property index doesn't exist.";
CHECK(value.type() != PropertyValue::Type::Null)
<< "Can't query index for propery value type null.";
return iter::imap(
[this](auto vlist) { return VertexAccessor(vlist, *this); },
db_.label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), value, *transaction_,
db_.storage().label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), value, transaction_,
current_state));
}
@ -280,19 +278,19 @@ class GraphDbAccessor {
* satisfy the bounds and are visible to the current transaction.
*/
auto Vertices(
const GraphDbTypes::Label &label, const GraphDbTypes::Property &property,
const Label &label, const Property &property,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper,
bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(db_.label_property_index_.IndexExists(
DCHECK(db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property)))
<< "Label+property index doesn't exist.";
return iter::imap(
[this](auto vlist) { return VertexAccessor(vlist, *this); },
db_.label_property_index_.GetVlists(
db_.storage().label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), lower, upper,
*transaction_, current_state));
transaction_, current_state));
}
/**
@ -314,7 +312,7 @@ class GraphDbAccessor {
* @return An accessor to the edge.
*/
EdgeAccessor InsertEdge(VertexAccessor &from, VertexAccessor &to,
GraphDbTypes::EdgeType type,
EdgeType type,
std::experimental::optional<gid::Gid> requested_gid =
std::experimental::nullopt);
@ -361,7 +359,7 @@ class GraphDbAccessor {
// wrap version lists into accessors, which will look for visible versions
auto accessors = iter::imap(
[this](auto id_vlist) { return EdgeAccessor(id_vlist.second, *this); },
db_.edges_.access());
db_.storage().edges_.access());
// filter out the accessors not visible to the current transaction
return iter::filter(
@ -418,17 +416,16 @@ class GraphDbAccessor {
* @param label - label to build for
* @param property - property to build for
*/
void BuildIndex(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property);
void BuildIndex(const Label &label, const Property &property);
/**
* @brief - Returns true if the given label+property index already exists and
* is ready for use.
*/
bool LabelPropertyIndexExists(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property) const {
bool LabelPropertyIndexExists(const Label &label,
const Property &property) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.label_property_index_.IndexExists(
return db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property));
}
@ -437,7 +434,7 @@ class GraphDbAccessor {
*/
std::vector<LabelPropertyIndex::Key> GetIndicesKeys() {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.label_property_index_.Keys();
return db_.storage().label_property_index_.Keys();
}
/**
@ -459,7 +456,7 @@ class GraphDbAccessor {
* @param label - label to check for
* @return number of vertices with the given label
*/
int64_t VerticesCount(const GraphDbTypes::Label &label) const;
int64_t VerticesCount(const Label &label) const;
/**
* Return approximate number of vertices under indexes with the given label
@ -471,8 +468,7 @@ class GraphDbAccessor {
* @return number of vertices with the given label, fails if no such
* label+property index exists.
*/
int64_t VerticesCount(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property) const;
int64_t VerticesCount(const Label &label, const Property &property) const;
/**
* Returns approximate number of vertices that have the given label
@ -480,8 +476,7 @@ class GraphDbAccessor {
*
* Assumes that an index for that (label, property) exists.
*/
int64_t VerticesCount(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property,
int64_t VerticesCount(const Label &label, const Property &property,
const PropertyValue &value) const;
/**
@ -494,7 +489,7 @@ class GraphDbAccessor {
* Assumes that an index for that (label, property) exists.
*/
int64_t VerticesCount(
const GraphDbTypes::Label &label, const GraphDbTypes::Property &property,
const Label &label, const Property &property,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper)
const;
@ -503,7 +498,7 @@ class GraphDbAccessor {
* Obtains the Label for the label's name.
* @return See above.
*/
GraphDbTypes::Label Label(const std::string &label_name);
Label Label(const std::string &label_name);
/**
* Obtains the label name (a string) for the given label.
@ -511,13 +506,13 @@ class GraphDbAccessor {
* @param label a Label.
* @return See above.
*/
const std::string &LabelName(const GraphDbTypes::Label label) const;
const std::string &LabelName(const class Label label) const;
/**
* Obtains the EdgeType for it's name.
* @return See above.
*/
GraphDbTypes::EdgeType EdgeType(const std::string &edge_type_name);
EdgeType EdgeType(const std::string &edge_type_name);
/**
* Obtains the edge type name (a string) for the given edge type.
@ -525,13 +520,13 @@ class GraphDbAccessor {
* @param edge_type an EdgeType.
* @return See above.
*/
const std::string &EdgeTypeName(const GraphDbTypes::EdgeType edge_type) const;
const std::string &EdgeTypeName(const class EdgeType edge_type) const;
/**
* Obtains the Property for it's name.
* @return See above.
*/
GraphDbTypes::Property Property(const std::string &property_name);
Property Property(const std::string &property_name);
/**
* Obtains the property name (a string) for the given property.
@ -539,7 +534,7 @@ class GraphDbAccessor {
* @param property a Property.
* @return See above.
*/
const std::string &PropertyName(const GraphDbTypes::Property property) const;
const std::string &PropertyName(const class Property property) const;
/** Returns the id of this accessor's transaction */
tx::transaction_id_t transaction_id() const;
@ -556,10 +551,7 @@ class GraphDbAccessor {
/** Return true if transaction is hinted to abort. */
bool should_abort() const;
/** Returns the transaction of this accessor */
const tx::Transaction &transaction() const { return *transaction_; }
/** Return's the database's write-ahead log */
const tx::Transaction &transaction() const { return transaction_; }
durability::WriteAheadLog &wal();
/**
@ -587,6 +579,26 @@ class GraphDbAccessor {
RemoteCache<TRecord> &remote_elements();
private:
GraphDb &db_;
tx::Transaction &transaction_;
bool commited_{false};
bool aborted_{false};
RemoteCache<Vertex> remote_vertices_;
RemoteCache<Edge> remote_edges_;
/** Casts the transaction engine to SingleNodeEngine and returns it. If the
* engine is a WorkerEngine (and not SingleNode nor Master), a call to this
* function will crash MG. */
tx::SingleNodeEngine &SingleNodeEngine() {
auto *single_node_engine =
dynamic_cast<tx::SingleNodeEngine *>(&db_.tx_engine());
DCHECK(single_node_engine)
<< "Asked for SingleNodeEngine on distributed worker";
return *single_node_engine;
}
/**
* Insert this vertex into corresponding label and label+property (if it
* exists) index.
@ -595,7 +607,7 @@ class GraphDbAccessor {
* @param vertex_accessor - vertex_accessor to insert
* @param vertex - vertex record to insert
*/
void UpdateLabelIndices(const GraphDbTypes::Label &label,
void UpdateLabelIndices(const class Label &label,
const VertexAccessor &vertex_accessor,
const Vertex *const vertex);
@ -606,29 +618,8 @@ class GraphDbAccessor {
* @param vertex_accessor - vertex accessor to insert
* @param vertex - vertex to insert
*/
void UpdatePropertyIndex(const GraphDbTypes::Property &property,
void UpdatePropertyIndex(const class Property &property,
const RecordAccessor<Vertex> &vertex_accessor,
const Vertex *const vertex);
/** Casts the DB's engine to SingleNodeEngine and returns it. If the DB's
* engine is RemoteEngine, this function will crash MG. It must be either
* SingleNodeEngine, or MasterEngine (which inherits it). */
tx::SingleNodeEngine &SingleNodeEngine() {
auto *single_node_engine =
dynamic_cast<tx::SingleNodeEngine *>(db_.tx_engine_.get());
DCHECK(single_node_engine)
<< "Asked for SingleNodeEngine on distributed worker";
return *single_node_engine;
}
GraphDb &db_;
/** The current transaction */
tx::Transaction *const transaction_;
bool commited_{false};
bool aborted_{false};
RemoteCache<Vertex> remote_vertices_;
RemoteCache<Edge> remote_edges_;
};
} // namespace database

View File

@ -10,7 +10,7 @@
#include "mvcc/version_list.hpp"
#include "transactions/transaction.hpp"
namespace IndexUtils {
namespace database::index {
/**
* @brief - Wrap beginning iterator to iterable object. This provides us with
* begin and end iterator, and allows us to iterate from the iterator given in
@ -181,4 +181,4 @@ static void Refresh(
}
}
}
}; // namespace IndexUtils
}; // namespace database::index

View File

@ -4,14 +4,16 @@
#include "data_structures/concurrent/concurrent_map.hpp"
#include "database/graph_db.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/indexes/index_common.hpp"
#include "database/types.hpp"
#include "mvcc/version_list.hpp"
#include "storage/edge.hpp"
#include "storage/vertex.hpp"
#include "transactions/transaction.hpp"
#include "utils/total_ordering.hpp"
namespace database {
/**
* @brief Implements index update and acquire.
* @Tparam TKey - underlying type by which to key objects
@ -60,8 +62,8 @@ class KeyIndex {
auto GetVlists(const TKey &key, tx::Transaction &t, bool current_state) {
auto access = GetKeyStorage(key)->access();
auto begin = access.begin();
return IndexUtils::GetVlists<typename SkipList<IndexEntry>::Iterator,
IndexEntry, TRecord>(
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
TRecord>(
std::move(access), begin, [](const IndexEntry &) { return true; }, t,
[key](const IndexEntry &, const TRecord *record) {
return KeyIndex::Exists(key, record);
@ -91,7 +93,7 @@ class KeyIndex {
* @param engine - transaction engine to see which records are commited
*/
void Refresh(const tx::Snapshot &snapshot, tx::Engine &engine) {
return IndexUtils::Refresh<TKey, IndexEntry, TRecord>(
return index::Refresh<TKey, IndexEntry, TRecord>(
indices_, snapshot, engine,
[](const TKey &key, const IndexEntry &entry) {
return KeyIndex::Exists(key, entry.record_);
@ -172,7 +174,7 @@ class KeyIndex {
* @param label - label to check for.
* @return true if it contains, false otherwise.
*/
static bool Exists(const GraphDbTypes::Label &label, const Vertex *const v) {
static bool Exists(const Label &label, const Vertex *const v) {
DCHECK(v != nullptr) << "Vertex is nullptr.";
// We have to check for existance of label because the transaction
// might not see the label, or the label was deleted and not yet
@ -186,8 +188,7 @@ class KeyIndex {
* @param edge_type - edge_type to check for.
* @return true if it has that edge_type, false otherwise.
*/
static bool Exists(const GraphDbTypes::EdgeType &edge_type,
const Edge *const e) {
static bool Exists(const EdgeType &edge_type, const Edge *const e) {
DCHECK(e != nullptr) << "Edge is nullptr.";
// We have to check for equality of edge types because the transaction
// might not see the edge type, or the edge type was deleted and not yet
@ -197,3 +198,4 @@ class KeyIndex {
ConcurrentMap<TKey, SkipList<IndexEntry> *> indices_;
};
} // namespace database

View File

@ -4,8 +4,8 @@
#include "data_structures/concurrent/concurrent_map.hpp"
#include "database/graph_db.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/indexes/index_common.hpp"
#include "database/types.hpp"
#include "mvcc/version_list.hpp"
#include "storage/edge.hpp"
#include "storage/vertex.hpp"
@ -13,6 +13,8 @@
#include "utils/bound.hpp"
#include "utils/total_ordering.hpp"
namespace database {
/**
* @brief Implements LabelPropertyIndex.
* Currently this provides implementation for:
@ -44,11 +46,10 @@ class LabelPropertyIndex {
*/
class Key : public TotalOrdering<Key> {
public:
const GraphDbTypes::Label label_;
const GraphDbTypes::Property property_;
const Label label_;
const Property property_;
Key(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property)
Key(const Label &label, const Property &property)
: label_(label), property_(property) {}
// Comparison operators - we need them to keep this sorted inside skiplist.
@ -120,8 +121,7 @@ class LabelPropertyIndex {
* @param vlist - pointer to vlist entry to add
* @param vertex - pointer to vertex record entry to add (contained in vlist)
*/
void UpdateOnLabel(const GraphDbTypes::Label &label,
mvcc::VersionList<Vertex> *const vlist,
void UpdateOnLabel(const Label &label, mvcc::VersionList<Vertex> *const vlist,
const Vertex *const vertex) {
for (auto index : indices_.access()) {
if (index.first.label_ != label) continue;
@ -141,7 +141,7 @@ class LabelPropertyIndex {
* @param vlist - pointer to vlist entry to add
* @param vertex - pointer to vertex record entry to add (contained in vlist)
*/
void UpdateOnProperty(const GraphDbTypes::Property &property,
void UpdateOnProperty(const Property &property,
mvcc::VersionList<Vertex> *const vlist,
const Vertex *const vertex) {
const auto &labels = vertex->labels_;
@ -171,8 +171,8 @@ class LabelPropertyIndex {
DCHECK(ready_for_use_.access().contains(key)) << "Index not yet ready.";
auto access = GetKeyStorage(key)->access();
auto begin = access.begin();
return IndexUtils::GetVlists<typename SkipList<IndexEntry>::Iterator,
IndexEntry, Vertex, SkipList<IndexEntry>>(
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
Vertex, SkipList<IndexEntry>>(
std::move(access), begin, [](const IndexEntry &) { return true; }, t,
[key](const IndexEntry &entry, const Vertex *const vertex) {
return LabelPropertyIndex::Exists(key, entry.value_, vertex);
@ -202,8 +202,8 @@ class LabelPropertyIndex {
auto start_iter = access.find_or_larger(IndexEntry(
value, reinterpret_cast<mvcc::VersionList<Vertex> *>(min_ptr),
reinterpret_cast<const Vertex *>(min_ptr)));
return IndexUtils::GetVlists<typename SkipList<IndexEntry>::Iterator,
IndexEntry, Vertex>(
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
Vertex>(
std::move(access), start_iter,
[value](const IndexEntry &entry) {
return !IndexEntry::Less(value, entry.value_) &&
@ -217,18 +217,16 @@ class LabelPropertyIndex {
}
/**
* @brief - Get an iterable over all mvcc::VersionLists that
* are contained in this index and satisfy the given bounds.
* Get an iterable over all mvcc::VersionLists that are contained in this
* index and satisfy the given bounds.
*
* The returned iterator will only contain
* vertices/edges whose property value is comparable with the
* given bounds (w.r.t. type). This has implications on Cypher
* query execuction semantics which have not been resolved yet.
* The returned iterator will only contain vertices/edges whose property value
* is comparable with the given bounds (w.r.t. type). This has implications on
* Cypher query execuction semantics which have not been resolved yet.
*
* At least one of the bounds must be specified. Bounds can't be
* @c PropertyValue::Null. If both bounds are
* specified, their PropertyValue elments must be of comparable
* types.
* At least one of the bounds must be specified. Bounds can't be @c
* PropertyValue::Null. If both bounds are specified, their PropertyValue
* elements must be of comparable types.
*
* @param key - Label+Property to query.
* @param lower - Lower bound of the interval.
@ -269,9 +267,10 @@ class LabelPropertyIndex {
auto access = GetKeyStorage(key)->access();
// create the iterator startpoint based on the lower bound
auto start_iter = lower ? access.find_or_larger(make_index_bound(
lower, lower.value().IsInclusive()))
: access.begin();
auto start_iter = lower
? access.find_or_larger(make_index_bound(
lower, lower.value().IsInclusive()))
: access.begin();
// a function that defines if an entry staisfies the filtering predicate.
// since we already handled the lower bound, we only need to deal with the
@ -297,8 +296,8 @@ class LabelPropertyIndex {
};
}
return IndexUtils::GetVlists<typename SkipList<IndexEntry>::Iterator,
IndexEntry, Vertex>(
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
Vertex>(
std::move(access), start_iter, predicate, transaction,
[key](const IndexEntry &entry, const Vertex *const vertex) {
return LabelPropertyIndex::Exists(key, entry.value_, vertex);
@ -369,7 +368,7 @@ class LabelPropertyIndex {
* transaction's snapshot, with that transaction's id appened as last.
*/
void Refresh(const tx::Snapshot &snapshot, tx::Engine &engine) {
return IndexUtils::Refresh<Key, IndexEntry, Vertex>(
return index::Refresh<Key, IndexEntry, Vertex>(
indices_, snapshot, engine,
[](const Key &key, const IndexEntry &entry) {
return LabelPropertyIndex::Exists(key, entry.value_, entry.record_);
@ -542,3 +541,4 @@ class LabelPropertyIndex {
ConcurrentMap<Key, SkipList<IndexEntry> *> indices_;
ConcurrentSet<Key> ready_for_use_;
};
} // namespace database

57
src/database/storage.hpp Normal file
View File

@ -0,0 +1,57 @@
#pragma once
#include "data_structures/concurrent/concurrent_map.hpp"
#include "data_structures/concurrent/concurrent_set.hpp"
#include "database/indexes/key_index.hpp"
#include "database/indexes/label_property_index.hpp"
#include "database/types.hpp"
#include "mvcc/version_list.hpp"
#include "storage/edge.hpp"
#include "storage/vertex.hpp"
namespace database {
/** A data structure containing the main data members of a graph database. */
class Storage {
public:
explicit Storage(int worker_id)
: vertex_generator_{worker_id}, edge_generator_{worker_id} {}
public:
~Storage() {
// Delete vertices and edges which weren't collected before, also deletes
// records inside version list
for (auto &id_vlist : vertices_.access()) delete id_vlist.second;
for (auto &id_vlist : edges_.access()) delete id_vlist.second;
}
Storage(const Storage &) = delete;
Storage(Storage &&) = delete;
Storage &operator=(const Storage &) = delete;
Storage &operator=(Storage &&) = delete;
gid::Generator &VertexGenerator() { return vertex_generator_; }
gid::Generator &EdgeGenerator() { return edge_generator_; }
private:
friend class GraphDbAccessor;
friend class StorageGc;
gid::Generator vertex_generator_;
gid::Generator edge_generator_;
// main storage for the graph
ConcurrentMap<gid::Gid, mvcc::VersionList<Vertex> *> vertices_;
ConcurrentMap<gid::Gid, mvcc::VersionList<Edge> *> edges_;
// indexes
KeyIndex<Label, Vertex> labels_index_;
LabelPropertyIndex label_property_index_;
// Set of transactions ids which are building indexes currently
ConcurrentSet<tx::transaction_id_t> index_build_tx_in_progress_;
// DB level global counters, used in the "counter" function.
ConcurrentMap<std::string, std::atomic<int64_t>> counters_;
};
} // namespace database

124
src/database/storage_gc.hpp Normal file
View File

@ -0,0 +1,124 @@
#pragma once
#include <chrono>
#include "data_structures/concurrent/concurrent_map.hpp"
#include "database/storage.hpp"
#include "mvcc/version_list.hpp"
#include "storage/deferred_deleter.hpp"
#include "storage/edge.hpp"
#include "storage/garbage_collector.hpp"
#include "storage/gid.hpp"
#include "storage/vertex.hpp"
#include "transactions/engine.hpp"
#include "utils/scheduler.hpp"
namespace database {
/** Garbage collection capabilities for database::Storage. Extracted into a
* separate class for better code organization, and because the GC requires a
* tx::Engine, while the Storage itself can exist without it. Even though, a
* database::Storage is always acompanied by a Gc.
*/
class StorageGc {
template <typename TRecord>
class MvccDeleter {
using VlistT = mvcc::VersionList<TRecord>;
public:
explicit MvccDeleter(ConcurrentMap<gid::Gid, VlistT *> &collection)
: gc_(collection, record_deleter_, version_list_deleter_) {}
DeferredDeleter<TRecord> record_deleter_;
DeferredDeleter<mvcc::VersionList<TRecord>> version_list_deleter_;
GarbageCollector<ConcurrentMap<gid::Gid, VlistT *>, TRecord> gc_;
};
public:
/** Creates a garbage collector for the given storage that uses the given
* tx::Engine. If `pause_sec` is greater then zero, then GC gets triggered
* periodically. */
StorageGc(Storage &storage, tx::Engine &tx_engine, int pause_sec)
: storage_(storage),
tx_engine_(tx_engine),
vertices_(storage.vertices_),
edges_(storage.edges_) {
if (pause_sec > 0)
scheduler_.Run(std::chrono::seconds(pause_sec),
[this] { CollectGarbage(); });
}
~StorageGc() {
scheduler_.Stop();
edges_.record_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
vertices_.record_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
edges_.version_list_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
vertices_.version_list_deleter_.FreeExpiredObjects(
tx::Transaction::MaxId());
}
StorageGc(const StorageGc &) = delete;
StorageGc(StorageGc &&) = delete;
StorageGc &operator=(const StorageGc &) = delete;
StorageGc &operator=(StorageGc &&) = delete;
void CollectGarbage() {
// main garbage collection logic
// see wiki documentation for logic explanation
LOG(INFO) << "Garbage collector started";
const auto snapshot = tx_engine_.GlobalGcSnapshot();
{
// This can be run concurrently
utils::Timer x;
vertices_.gc_.Run(snapshot, tx_engine_);
edges_.gc_.Run(snapshot, tx_engine_);
VLOG(1) << "Garbage collector mvcc phase time: " << x.Elapsed().count();
}
// This has to be run sequentially after gc because gc modifies
// version_lists and changes the oldest visible record, on which Refresh
// depends.
{
// This can be run concurrently
utils::Timer x;
storage_.labels_index_.Refresh(snapshot, tx_engine_);
storage_.label_property_index_.Refresh(snapshot, tx_engine_);
VLOG(1) << "Garbage collector index phase time: " << x.Elapsed().count();
}
{
// We free expired objects with snapshot.back(), which is
// the ID of the oldest active transaction (or next active, if there
// are no currently active). That's legal because that was the
// last possible transaction that could have obtained pointers
// to those records. New snapshot can be used, different than one used for
// first two phases of gc.
utils::Timer x;
const auto snapshot = tx_engine_.GlobalGcSnapshot();
edges_.record_deleter_.FreeExpiredObjects(snapshot.back());
vertices_.record_deleter_.FreeExpiredObjects(snapshot.back());
edges_.version_list_deleter_.FreeExpiredObjects(snapshot.back());
vertices_.version_list_deleter_.FreeExpiredObjects(snapshot.back());
VLOG(1) << "Garbage collector deferred deletion phase time: "
<< x.Elapsed().count();
}
LOG(INFO) << "Garbage collector finished";
VLOG(2) << "gc snapshot: " << snapshot;
VLOG(2) << "edge_record_deleter_ size: " << edges_.record_deleter_.Count();
VLOG(2) << "vertex record deleter_ size: "
<< vertices_.record_deleter_.Count();
VLOG(2) << "edge_version_list_deleter_ size: "
<< edges_.version_list_deleter_.Count();
VLOG(2) << "vertex_version_list_deleter_ size: "
<< vertices_.version_list_deleter_.Count();
VLOG(2) << "vertices_ size: " << storage_.vertices_.access().size();
VLOG(2) << "edges_ size: " << storage_.edges_.access().size();
}
private:
Storage &storage_;
tx::Engine &tx_engine_;
MvccDeleter<Vertex> vertices_;
MvccDeleter<Edge> edges_;
Scheduler scheduler_;
};
} // namespace database

View File

@ -1,12 +1,13 @@
#pragma once
#include <string>
#include <cstdint>
#include <functional>
#include "boost/serialization/base_object.hpp"
#include "utils/total_ordering.hpp"
namespace GraphDbTypes {
namespace database {
template <typename TSpecificType>
class Common : public TotalOrdering<TSpecificType> {
@ -73,19 +74,18 @@ class Property : public Common<Property> {
ar &boost::serialization::base_object<Common<Property>>(*this);
}
};
}; // namespace GraphDbTypes
}; // namespace database
namespace std {
template <>
struct hash<GraphDbTypes::Label>
: public GraphDbTypes::Common<GraphDbTypes::Label>::Hash {};
struct hash<database::Label> : public database::Common<database::Label>::Hash {
};
template <>
struct hash<GraphDbTypes::EdgeType>
: public GraphDbTypes::Common<GraphDbTypes::EdgeType>::Hash {};
struct hash<database::EdgeType>
: public database::Common<database::EdgeType>::Hash {};
template <>
struct hash<GraphDbTypes::Property>
: public GraphDbTypes::Common<GraphDbTypes::Property>::Hash {};
struct hash<database::Property>
: public database::Common<database::Property>::Hash {};
} // namespace std

View File

@ -26,7 +26,7 @@ class MasterCoordination {
int RegisterWorker(int desired_worker_id, Endpoint endpoint);
public:
MasterCoordination(communication::messaging::System &system);
explicit MasterCoordination(communication::messaging::System &system);
/** Shuts down all the workers and this master server. */
~MasterCoordination();

View File

@ -4,6 +4,7 @@
#include <memory>
#include <vector>
#include "database/types.hpp"
#include "storage/edge.hpp"
#include "storage/vertex.hpp"
#include "utils/serialization.hpp"
@ -95,11 +96,11 @@ void LoadProperties(TArchive &ar, PropertyValueStore &store) {
size_t count;
ar >> count;
for (size_t i = 0; i < count; ++i) {
GraphDbTypes::Property::StorageT prop;
database::Property::StorageT prop;
ar >> prop;
query::TypedValue value;
utils::LoadTypedValue(ar, value);
store.set(GraphDbTypes::Property(prop), static_cast<PropertyValue>(value));
store.set(database::Property(prop), static_cast<PropertyValue>(value));
}
}
@ -120,14 +121,14 @@ std::unique_ptr<Vertex> LoadVertex(TArchive &ar) {
ar >> count;
for (size_t i = 0; i < count; ++i) {
auto vertex_address = impl::LoadVertexAddress(ar);
GraphDbTypes::EdgeType::StorageT edge_type;
database::EdgeType::StorageT edge_type;
gid::Gid edge_id;
ar >> edge_id;
int edge_worker_id;
ar >> edge_worker_id;
ar >> edge_type;
edges.emplace(vertex_address, {edge_id, edge_worker_id},
GraphDbTypes::EdgeType(edge_type));
database::EdgeType(edge_type));
}
};
decode_edges(vertex->out_);
@ -136,7 +137,7 @@ std::unique_ptr<Vertex> LoadVertex(TArchive &ar) {
size_t count;
ar >> count;
for (size_t i = 0; i < count; ++i) {
GraphDbTypes::Label::StorageT label;
database::Label::StorageT label;
ar >> label;
vertex->labels_.emplace_back(label);
}
@ -155,10 +156,9 @@ template <typename TArchive>
std::unique_ptr<Edge> LoadEdge(TArchive &ar) {
auto from = impl::LoadVertexAddress(ar);
auto to = impl::LoadVertexAddress(ar);
GraphDbTypes::EdgeType::StorageT edge_type;
database::EdgeType::StorageT edge_type;
ar >> edge_type;
auto edge =
std::make_unique<Edge>(from, to, GraphDbTypes::EdgeType{edge_type});
auto edge = std::make_unique<Edge>(from, to, database::EdgeType{edge_type});
impl::LoadProperties(ar, edge->properties_);
return edge;

View File

@ -52,7 +52,7 @@ struct RecoveryData {
return false; \
}
bool RecoverSnapshot(const fs::path &snapshot_file, GraphDb &db,
bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb &db,
RecoveryData &recovery_data) {
HashedFileReader reader;
communication::bolt::Decoder<HashedFileReader> decoder(reader);
@ -79,12 +79,12 @@ bool RecoverSnapshot(const fs::path &snapshot_file, GraphDb &db,
// Vertex and edge generator ids
RETURN_IF_NOT(decoder.ReadValue(&dv, DecodedValue::Type::Int));
uint64_t vertex_generator_cnt = dv.ValueInt();
db.VertexGenerator().SetId(
std::max(db.VertexGenerator().LocalCount(), vertex_generator_cnt));
db.storage().VertexGenerator().SetId(std::max(
db.storage().VertexGenerator().LocalCount(), vertex_generator_cnt));
RETURN_IF_NOT(decoder.ReadValue(&dv, DecodedValue::Type::Int));
uint64_t edge_generator_cnt = dv.ValueInt();
db.EdgeGenerator().SetId(
std::max(db.EdgeGenerator().LocalCount(), edge_generator_cnt));
db.storage().EdgeGenerator().SetId(
std::max(db.storage().EdgeGenerator().LocalCount(), edge_generator_cnt));
RETURN_IF_NOT(decoder.ReadValue(&dv, DecodedValue::Type::Int));
recovery_data.snapshooter_tx_id = dv.ValueInt();
@ -107,7 +107,7 @@ bool RecoverSnapshot(const fs::path &snapshot_file, GraphDb &db,
property.ValueString());
}
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
for (int64_t i = 0; i < vertex_count; ++i) {
DecodedValue vertex_dv;
RETURN_IF_NOT(decoder.ReadValue(&vertex_dv, DecodedValue::Type::Vertex));
@ -152,7 +152,7 @@ bool RecoverSnapshot(const fs::path &snapshot_file, GraphDb &db,
#undef RETURN_IF_NOT
// TODO - finer-grained recovery feedback could be useful here.
bool RecoverWal(const fs::path &wal_dir, GraphDb &db,
bool RecoverWal(const fs::path &wal_dir, database::GraphDb &db,
RecoveryData &recovery_data) {
// Get paths to all the WAL files and sort them (on date).
std::vector<fs::path> wal_files;
@ -173,9 +173,9 @@ bool RecoverWal(const fs::path &wal_dir, GraphDb &db,
!utils::Contains(tx_sn, tx_id));
};
std::unordered_map<tx::transaction_id_t, GraphDbAccessor> accessors;
std::unordered_map<tx::transaction_id_t, database::GraphDbAccessor> accessors;
auto get_accessor =
[&accessors](tx::transaction_id_t tx_id) -> GraphDbAccessor & {
[&accessors](tx::transaction_id_t tx_id) -> database::GraphDbAccessor & {
auto found = accessors.find(tx_id);
CHECK(found != accessors.end())
<< "Accessor does not exist for transaction";
@ -227,7 +227,7 @@ bool RecoverWal(const fs::path &wal_dir, GraphDb &db,
}
} // anonymous namespace
bool Recover(const fs::path &durability_dir, GraphDb &db) {
bool Recover(const fs::path &durability_dir, database::GraphDb &db) {
RecoveryData recovery_data;
// Attempt to recover from snapshot files in reverse order (from newest
@ -257,7 +257,7 @@ bool Recover(const fs::path &durability_dir, GraphDb &db) {
RecoverWal(durability_dir / kWalDir, db, recovery_data);
// Index recovery.
GraphDbAccessor db_accessor_indices{db};
database::GraphDbAccessor db_accessor_indices{db};
for (const auto &label_prop : recovery_data.indexes)
db_accessor_indices.BuildIndex(
db_accessor_indices.Label(label_prop.first),

View File

@ -26,5 +26,5 @@ bool ReadSnapshotSummary(HashedFileReader &buffer, int64_t &vertex_count,
* @return - If recovery was succesful.
*/
bool Recover(const std::experimental::filesystem::path &durability_dir,
GraphDb &db);
database::GraphDb &db);
} // namespace durability

View File

@ -16,7 +16,8 @@ namespace fs = std::experimental::filesystem;
namespace durability {
namespace {
bool Encode(const fs::path &snapshot_file, GraphDb &db, GraphDbAccessor &dba) {
bool Encode(const fs::path &snapshot_file, database::GraphDb &db,
database::GraphDbAccessor &dba) {
try {
HashedFileWriter buffer(snapshot_file);
communication::bolt::BaseEncoder<HashedFileWriter> encoder(buffer);
@ -28,8 +29,8 @@ bool Encode(const fs::path &snapshot_file, GraphDb &db, GraphDbAccessor &dba) {
// Write the number of generated vertex and edges, used to recover
// generators internal states
encoder.WriteInt(db.VertexGenerator().LocalCount());
encoder.WriteInt(db.EdgeGenerator().LocalCount());
encoder.WriteInt(db.storage().VertexGenerator().LocalCount());
encoder.WriteInt(db.storage().EdgeGenerator().LocalCount());
// Write the ID of the transaction doing the snapshot.
encoder.WriteInt(dba.transaction_id());
@ -115,12 +116,12 @@ fs::path MakeSnapshotPath(const fs::path &durability_dir) {
return durability_dir / kSnapshotDir / date_str;
}
bool MakeSnapshot(GraphDb &db, const fs::path &durability_dir,
bool MakeSnapshot(database::GraphDb &db, const fs::path &durability_dir,
const int snapshot_max_retained) {
if (!EnsureDir(durability_dir / kSnapshotDir)) return false;
const auto snapshot_file = MakeSnapshotPath(durability_dir);
if (fs::exists(snapshot_file)) return false;
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
if (Encode(snapshot_file, db, dba)) {
RemoveOldSnapshots(durability_dir / kSnapshotDir, snapshot_max_retained);
RemoveOldWals(durability_dir / kWalDir, dba.transaction());

View File

@ -18,6 +18,6 @@ path MakeSnapshotPath(const path &durability_dir);
* @param durability_dir - directory where durability data is stored.
* @param snapshot_max_retained - maximum number of snapshots to retain.
*/
bool MakeSnapshot(GraphDb &db, const path &durability_dir,
bool MakeSnapshot(database::GraphDb &db, const path &durability_dir,
int snapshot_max_retained);
} // namespace durability

View File

@ -11,8 +11,8 @@
#include "communication/bolt/v1/decoder/decoder.hpp"
#include "communication/bolt/v1/encoder/primitive_encoder.hpp"
#include "data_structures/ring_buffer.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/state_delta.hpp"
#include "database/types.hpp"
#include "storage/gid.hpp"
#include "storage/property_value.hpp"
#include "transactions/type.hpp"
@ -33,7 +33,8 @@ class WriteAheadLog {
bool durability_enabled);
~WriteAheadLog();
/** Enables the WAL. Called at the end of GraphDb construction, after
/** Enables the WAL. Called at the end of database::GraphDb construction,
* after
* (optional) recovery. */
void Enable() { enabled_ = true; }
@ -44,7 +45,7 @@ class WriteAheadLog {
/** Groups the logic of WAL file handling (flushing, naming, rotating) */
class WalFile {
public:
WalFile(const std::experimental::filesystem::path &wal__dir);
explicit WalFile(const std::experimental::filesystem::path &wal__dir);
~WalFile();
/** Initializes the WAL file. Must be called before first flush. Can be

View File

@ -9,6 +9,7 @@
#include "communication/messaging/distributed.hpp"
#include "communication/server.hpp"
#include "config.hpp"
#include "database/graph_db.hpp"
#include "distributed/coordination_master.hpp"
#include "distributed/coordination_worker.hpp"
#include "io/network/network_endpoint.hpp"
@ -23,11 +24,10 @@
#include "version.hpp"
namespace fs = std::experimental::filesystem;
using communication::bolt::SessionData;
using io::network::NetworkEndpoint;
using io::network::Socket;
using communication::bolt::SessionData;
using SessionT = communication::bolt::Session<Socket>;
using ResultStreamT = SessionT::ResultStreamT;
using ServerT = communication::Server<SessionT, SessionData>;
// General purpose flags.
@ -50,32 +50,15 @@ DEFINE_uint64(memory_warning_threshold, 1024,
DEFINE_HIDDEN_bool(
master, false,
"If this Memgraph server is the master in a distributed deployment.");
DEFINE_HIDDEN_string(master_host, "0.0.0.0",
"For master node indicates the host served on. For worker "
"node indicates the master location.");
DEFINE_VALIDATED_HIDDEN_int32(
master_port, 0,
"For master node the port on which to serve. For "
"worker node indicates the master's port.",
FLAG_IN_RANGE(0, std::numeric_limits<uint16_t>::max()));
DEFINE_HIDDEN_bool(
worker, false,
"If this Memgraph server is a worker in a distributed deployment.");
DEFINE_HIDDEN_string(worker_host, "0.0.0.0",
"For worker node indicates the host served on. For master "
"node this flag is not used.");
DEFINE_VALIDATED_HIDDEN_int32(
worker_port, 0,
"For master node it's unused. For worker node "
"indicates the port on which to serve. If zero (default value), a port is "
"chosen at random. Sent to the master when registring worker node.",
FLAG_IN_RANGE(0, std::numeric_limits<uint16_t>::max()));
// Needed to correctly handle memgraph destruction from a signal handler.
// Without having some sort of a flag, it is possible that a signal is handled
// when we are exiting main, inside destructors of GraphDb and similar. The
// signal handler may then initiate another shutdown on memgraph which is in
// half destructed state, causing invalid memory access and crash.
// when we are exiting main, inside destructors of database::GraphDb and
// similar. The signal handler may then initiate another shutdown on memgraph
// which is in half destructed state, causing invalid memory access and crash.
volatile sig_atomic_t is_shutting_down = 0;
// Registers the given shutdown function with the appropriate signal handlers.
@ -117,65 +100,47 @@ void StartMemWarningLogger() {
void MasterMain() {
google::SetUsageMessage("Memgraph distributed master");
// RPC for worker registration, shutdown and endpoint info exchange.
communication::messaging::System system(FLAGS_master_host, FLAGS_master_port);
distributed::MasterCoordination master(system);
// Bolt server stuff.
SessionData session_data{system, master};
NetworkEndpoint endpoint(FLAGS_interface, FLAGS_port);
ServerT server(endpoint, session_data, FLAGS_num_workers);
database::Master db;
SessionData session_data{db};
ServerT server({FLAGS_interface, FLAGS_port}, session_data,
FLAGS_num_workers);
// Handler for regular termination signals
auto shutdown = [&server, &session_data] {
auto shutdown = [&server] {
if (is_shutting_down) return;
is_shutting_down = 1;
// Server needs to be shutdown first and then the database. This prevents a
// race condition when a transaction is accepted during server shutdown.
server.Shutdown();
session_data.db.Shutdown();
};
InitSignalHandlers(shutdown);
StartMemWarningLogger();
server.AwaitShutdown();
}
void WorkerMain() {
google::SetUsageMessage("Memgraph distributed worker");
// RPC for worker registration, shutdown and endpoint info exchange.
communication::messaging::System system(FLAGS_worker_host, FLAGS_worker_port);
io::network::NetworkEndpoint master_endpoint{
FLAGS_master_host, static_cast<uint16_t>(FLAGS_master_port)};
distributed::WorkerCoordination worker(system, master_endpoint);
auto worker_id = worker.RegisterWorker();
// The GraphDb destructor shuts some RPC down. Ensure correct ordering.
{
GraphDb db{system, worker_id, worker, master_endpoint};
query::Interpreter interpreter;
StartMemWarningLogger();
// Wait for the shutdown command from the master.
worker.WaitForShutdown();
}
database::Worker db;
StartMemWarningLogger();
db.WaitForShutdown();
}
void SingleNodeMain() {
google::SetUsageMessage("Memgraph single-node database server");
SessionData session_data;
NetworkEndpoint endpoint(FLAGS_interface, FLAGS_port);
ServerT server(endpoint, session_data, FLAGS_num_workers);
database::SingleNode db;
SessionData session_data{db};
ServerT server({FLAGS_interface, FLAGS_port}, session_data,
FLAGS_num_workers);
// Handler for regular termination signals
auto shutdown = [&server, &session_data] {
auto shutdown = [&server] {
if (is_shutting_down) return;
is_shutting_down = 1;
// Server needs to be shutdown first and then the database. This prevents a
// race condition when a transaction is accepted during server shutdown.
server.Shutdown();
session_data.db.Shutdown();
};
InitSignalHandlers(shutdown);

View File

@ -48,7 +48,7 @@ std::string ReadLine(const char *prompt) {
#endif // HAS_READLINE
void query::Repl(GraphDb &db) {
void query::Repl(database::GraphDb &db) {
std::cout
<< "Welcome to *Awesome* Memgraph Read Evaluate Print Loop (AM-REPL)"
<< std::endl;
@ -63,7 +63,7 @@ void query::Repl(GraphDb &db) {
// regular cypher queries
try {
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
ResultStreamFaker results;
interpeter(command, dba, {}, false).PullAll(results);
std::cout << results;

View File

@ -6,9 +6,9 @@ namespace query {
/**
* Read Evaluate Print Loop, for interacting with a database (the database in
* the given GraphDb). Immediately starts the user-input loop and interprets the
* entered queries.
* the given database::GraphDb). Immediately starts the user-input loop and
* interprets the entered queries.
*/
void Repl(GraphDb &);
void Repl(database::GraphDb &);
} // namespace query

View File

@ -16,8 +16,9 @@ class Context {
Context(Context &&) = default;
Context &operator=(Context &&) = default;
explicit Context(GraphDbAccessor &db_accessor) : db_accessor_(db_accessor) {}
GraphDbAccessor &db_accessor_;
explicit Context(database::GraphDbAccessor &db_accessor)
: db_accessor_(db_accessor) {}
database::GraphDbAccessor &db_accessor_;
SymbolTable symbol_table_;
Parameters parameters_;
bool is_query_cached_ = false;

View File

@ -12,7 +12,7 @@
#include "glog/logging.h"
#include "database/graph_db.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "query/frontend/ast/ast_visitor.hpp"
#include "query/frontend/semantic/symbol.hpp"
#include "query/interpret/awesome_memgraph_functions.hpp"
@ -23,15 +23,15 @@
// Hash function for the key in pattern atom property maps.
namespace std {
template <>
struct hash<std::pair<std::string, GraphDbTypes::Property>> {
struct hash<std::pair<std::string, database::Property>> {
size_t operator()(
const std::pair<std::string, GraphDbTypes::Property> &pair) const {
const std::pair<std::string, database::Property> &pair) const {
return string_hash(pair.first) ^ property_hash(pair.second);
};
private:
std::hash<std::string> string_hash{};
std::hash<GraphDbTypes::Property> property_hash{};
std::hash<database::Property> property_hash{};
};
} // namespace std
@ -1060,15 +1060,14 @@ class MapLiteral : public BaseLiteral {
}
// maps (property_name, property) to expressions
std::unordered_map<std::pair<std::string, GraphDbTypes::Property>,
Expression *>
std::unordered_map<std::pair<std::string, database::Property>, Expression *>
elements_;
protected:
explicit MapLiteral(int uid) : BaseLiteral(uid) {}
MapLiteral(
int uid,
const std::unordered_map<std::pair<std::string, GraphDbTypes::Property>,
const std::unordered_map<std::pair<std::string, database::Property>,
Expression *> &elements)
: BaseLiteral(uid), elements_(elements) {}
@ -1095,7 +1094,7 @@ class MapLiteral : public BaseLiteral {
size_t size = 0;
ar >> size;
for (size_t i = 0; i < size; ++i) {
std::pair<std::string, GraphDbTypes::Property> property;
std::pair<std::string, database::Property> property;
ar >> property.first;
ar >> property.second;
Expression *expression = nullptr;
@ -1165,18 +1164,17 @@ class PropertyLookup : public Expression {
Expression *expression_ = nullptr;
std::string property_name_;
GraphDbTypes::Property property_;
database::Property property_;
protected:
PropertyLookup(int uid, Expression *expression,
const std::string &property_name,
GraphDbTypes::Property property)
const std::string &property_name, database::Property property)
: Expression(uid),
expression_(expression),
property_name_(property_name),
property_(property) {}
PropertyLookup(int uid, Expression *expression,
const std::pair<std::string, GraphDbTypes::Property> &property)
const std::pair<std::string, database::Property> &property)
: Expression(uid),
expression_(expression),
property_name_(property.first),
@ -1226,11 +1224,11 @@ class LabelsTest : public Expression {
}
Expression *expression_ = nullptr;
std::vector<GraphDbTypes::Label> labels_;
std::vector<database::Label> labels_;
protected:
LabelsTest(int uid, Expression *expression,
const std::vector<GraphDbTypes::Label> &labels)
const std::vector<database::Label> &labels)
: Expression(uid), expression_(expression), labels_(labels) {}
private:
@ -1298,7 +1296,8 @@ class Function : public Expression {
private:
std::string function_name_;
std::function<TypedValue(const std::vector<TypedValue> &, GraphDbAccessor &)>
std::function<TypedValue(const std::vector<TypedValue> &,
database::GraphDbAccessor &)>
function_;
friend class boost::serialization::access;
@ -1598,10 +1597,9 @@ class NodeAtom : public PatternAtom {
return node_atom;
}
std::vector<GraphDbTypes::Label> labels_;
std::vector<database::Label> labels_;
// maps (property_name, property) to an expression
std::unordered_map<std::pair<std::string, GraphDbTypes::Property>,
Expression *>
std::unordered_map<std::pair<std::string, database::Property>, Expression *>
properties_;
protected:
@ -1632,7 +1630,7 @@ class NodeAtom : public PatternAtom {
size_t size = 0;
ar >> size;
for (size_t i = 0; i < size; ++i) {
std::pair<std::string, GraphDbTypes::Property> property;
std::pair<std::string, database::Property> property;
ar >> property.first;
ar >> property.second;
Expression *expression = nullptr;
@ -1706,9 +1704,8 @@ class EdgeAtom : public PatternAtom {
Type type_ = Type::SINGLE;
Direction direction_ = Direction::BOTH;
std::vector<GraphDbTypes::EdgeType> edge_types_;
std::unordered_map<std::pair<std::string, GraphDbTypes::Property>,
Expression *>
std::vector<database::EdgeType> edge_types_;
std::unordered_map<std::pair<std::string, database::Property>, Expression *>
properties_;
// Used in variable length and BFS expansions. Bounds can be nullptr. Inner
@ -1725,9 +1722,9 @@ class EdgeAtom : public PatternAtom {
EdgeAtom(int uid, Identifier *identifier, Type type, Direction direction)
: PatternAtom(uid, identifier), type_(type), direction_(direction) {}
// Creates an edge atom for a SINGLE expansion with the given types.
// Creates an edge atom for a SINGLE expansion with the given .
EdgeAtom(int uid, Identifier *identifier, Type type, Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types)
const std::vector<database::EdgeType> &edge_types)
: PatternAtom(uid, identifier),
type_(type),
direction_(direction),
@ -1767,7 +1764,7 @@ class EdgeAtom : public PatternAtom {
size_t size = 0;
ar >> size;
for (size_t i = 0; i < size; ++i) {
std::pair<std::string, GraphDbTypes::Property> property;
std::pair<std::string, database::Property> property;
ar >> property.first;
ar >> property.second;
Expression *expression = nullptr;
@ -2506,12 +2503,12 @@ class SetLabels : public Clause {
}
Identifier *identifier_ = nullptr;
std::vector<GraphDbTypes::Label> labels_;
std::vector<database::Label> labels_;
protected:
explicit SetLabels(int uid) : Clause(uid) {}
SetLabels(int uid, Identifier *identifier,
const std::vector<GraphDbTypes::Label> &labels)
const std::vector<database::Label> &labels)
: Clause(uid), identifier_(identifier), labels_(labels) {}
private:
@ -2601,12 +2598,12 @@ class RemoveLabels : public Clause {
}
Identifier *identifier_ = nullptr;
std::vector<GraphDbTypes::Label> labels_;
std::vector<database::Label> labels_;
protected:
explicit RemoveLabels(int uid) : Clause(uid) {}
RemoveLabels(int uid, Identifier *identifier,
const std::vector<GraphDbTypes::Label> &labels)
const std::vector<database::Label> &labels)
: Clause(uid), identifier_(identifier), labels_(labels) {}
private:
@ -2767,13 +2764,12 @@ class CreateIndex : public Clause {
return storage.Create<CreateIndex>(label_, property_);
}
GraphDbTypes::Label label_;
GraphDbTypes::Property property_;
database::Label label_;
database::Property property_;
protected:
explicit CreateIndex(int uid) : Clause(uid) {}
CreateIndex(int uid, GraphDbTypes::Label label,
GraphDbTypes::Property property)
CreateIndex(int uid, database::Label label, database::Property property)
: Clause(uid), label_(label), property_(property) {}
private:
@ -2836,10 +2832,9 @@ LOAD_AND_CONSTRUCT(query::PrimitiveLiteral, 0);
LOAD_AND_CONSTRUCT(query::ListLiteral, 0);
LOAD_AND_CONSTRUCT(query::MapLiteral, 0);
LOAD_AND_CONSTRUCT(query::Identifier, 0, "");
LOAD_AND_CONSTRUCT(query::PropertyLookup, 0, nullptr, "",
GraphDbTypes::Property());
LOAD_AND_CONSTRUCT(query::PropertyLookup, 0, nullptr, "", database::Property());
LOAD_AND_CONSTRUCT(query::LabelsTest, 0, nullptr,
std::vector<GraphDbTypes::Label>());
std::vector<database::Label>());
LOAD_AND_CONSTRUCT(query::Function, 0);
LOAD_AND_CONSTRUCT(query::Aggregation, 0, nullptr, nullptr,
query::Aggregation::Op::COUNT);

View File

@ -213,7 +213,7 @@ antlrcpp::Any CypherMainVisitor::visitCreate(CypherParser::CreateContext *ctx) {
*/
antlrcpp::Any CypherMainVisitor::visitCreateIndex(
CypherParser::CreateIndexContext *ctx) {
std::pair<std::string, GraphDbTypes::Property> key =
std::pair<std::string, database::Property> key =
ctx->propertyKeyName()->accept(this);
return storage_.Create<CreateIndex>(
ctx_.db_accessor_.Label(ctx->labelName()->accept(this)), key.second);
@ -305,21 +305,21 @@ antlrcpp::Any CypherMainVisitor::visitNodePattern(
}
if (ctx->nodeLabels()) {
node->labels_ =
ctx->nodeLabels()->accept(this).as<std::vector<GraphDbTypes::Label>>();
ctx->nodeLabels()->accept(this).as<std::vector<database::Label>>();
}
if (ctx->properties()) {
node->properties_ = ctx->properties()
->accept(this)
.as<std::unordered_map<
std::pair<std::string, GraphDbTypes::Property>,
Expression *>>();
node->properties_ =
ctx->properties()
->accept(this)
.as<std::unordered_map<std::pair<std::string, database::Property>,
Expression *>>();
}
return node;
}
antlrcpp::Any CypherMainVisitor::visitNodeLabels(
CypherParser::NodeLabelsContext *ctx) {
std::vector<GraphDbTypes::Label> labels;
std::vector<database::Label> labels;
for (auto *node_label : ctx->nodeLabel()) {
labels.push_back(ctx_.db_accessor_.Label(node_label->accept(this)));
}
@ -341,11 +341,10 @@ antlrcpp::Any CypherMainVisitor::visitProperties(
antlrcpp::Any CypherMainVisitor::visitMapLiteral(
CypherParser::MapLiteralContext *ctx) {
std::unordered_map<std::pair<std::string, GraphDbTypes::Property>,
Expression *>
std::unordered_map<std::pair<std::string, database::Property>, Expression *>
map;
for (int i = 0; i < static_cast<int>(ctx->propertyKeyName().size()); ++i) {
std::pair<std::string, GraphDbTypes::Property> key =
std::pair<std::string, database::Property> key =
ctx->propertyKeyName()[i]->accept(this);
Expression *value = ctx->expression()[i]->accept(this);
if (!map.insert({key, value}).second) {
@ -500,7 +499,7 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipPattern(
edge->edge_types_ = ctx->relationshipDetail()
->relationshipTypes()
->accept(this)
.as<std::vector<GraphDbTypes::EdgeType>>();
.as<std::vector<database::EdgeType>>();
}
auto relationshipLambdas = relationshipDetail->relationshipLambda();
@ -540,9 +539,8 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipPattern(
edge->properties_ =
properties[0]
->accept(this)
.as<std::unordered_map<
std::pair<std::string, GraphDbTypes::Property>,
Expression *>>();
.as<std::unordered_map<std::pair<std::string, database::Property>,
Expression *>>();
break;
}
default:
@ -566,7 +564,7 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipLambda(
antlrcpp::Any CypherMainVisitor::visitRelationshipTypes(
CypherParser::RelationshipTypesContext *ctx) {
std::vector<GraphDbTypes::EdgeType> types;
std::vector<database::EdgeType> types;
for (auto *edge_type : ctx->relTypeName()) {
types.push_back(ctx_.db_accessor_.EdgeType(edge_type->accept(this)));
}
@ -817,7 +815,7 @@ antlrcpp::Any CypherMainVisitor::visitExpression2a(
Expression *expression = ctx->expression2b()->accept(this);
if (ctx->nodeLabels()) {
auto labels =
ctx->nodeLabels()->accept(this).as<std::vector<GraphDbTypes::Label>>();
ctx->nodeLabels()->accept(this).as<std::vector<database::Label>>();
expression = storage_.Create<LabelsTest>(expression, labels);
}
return expression;
@ -827,7 +825,7 @@ antlrcpp::Any CypherMainVisitor::visitExpression2b(
CypherParser::Expression2bContext *ctx) {
Expression *expression = ctx->atom()->accept(this);
for (auto *lookup : ctx->propertyLookup()) {
std::pair<std::string, GraphDbTypes::Property> key = lookup->accept(this);
std::pair<std::string, database::Property> key = lookup->accept(this);
auto property_lookup =
storage_.Create<PropertyLookup>(expression, key.first, key.second);
expression = property_lookup;
@ -914,9 +912,8 @@ antlrcpp::Any CypherMainVisitor::visitLiteral(
return static_cast<Expression *>(storage_.Create<MapLiteral>(
ctx->mapLiteral()
->accept(this)
.as<std::unordered_map<
std::pair<std::string, GraphDbTypes::Property>,
Expression *>>()));
.as<std::unordered_map<std::pair<std::string, database::Property>,
Expression *>>()));
}
return visitChildren(ctx);
}
@ -1075,7 +1072,7 @@ antlrcpp::Any CypherMainVisitor::visitSetItem(
set_labels->identifier_ = storage_.Create<Identifier>(
ctx->variable()->accept(this).as<std::string>());
set_labels->labels_ =
ctx->nodeLabels()->accept(this).as<std::vector<GraphDbTypes::Label>>();
ctx->nodeLabels()->accept(this).as<std::vector<database::Label>>();
return static_cast<Clause *>(set_labels);
}
@ -1101,7 +1098,7 @@ antlrcpp::Any CypherMainVisitor::visitRemoveItem(
remove_labels->identifier_ = storage_.Create<Identifier>(
ctx->variable()->accept(this).as<std::string>());
remove_labels->labels_ =
ctx->nodeLabels()->accept(this).as<std::vector<GraphDbTypes::Label>>();
ctx->nodeLabels()->accept(this).as<std::vector<database::Label>>();
return static_cast<Clause *>(remove_labels);
}
@ -1109,7 +1106,7 @@ antlrcpp::Any CypherMainVisitor::visitPropertyExpression(
CypherParser::PropertyExpressionContext *ctx) {
Expression *expression = ctx->atom()->accept(this);
for (auto *lookup : ctx->propertyLookup()) {
std::pair<std::string, GraphDbTypes::Property> key = lookup->accept(this);
std::pair<std::string, database::Property> key = lookup->accept(this);
auto property_lookup =
storage_.Create<PropertyLookup>(expression, key.first, key.second);
expression = property_lookup;

View File

@ -214,12 +214,12 @@ class CypherMainVisitor : public antlropencypher::CypherBaseVisitor {
CypherParser::NodePatternContext *ctx) override;
/**
* @return vector<GraphDbTypes::Label>
* @return vector<database::Label>
*/
antlrcpp::Any visitNodeLabels(CypherParser::NodeLabelsContext *ctx) override;
/**
* @return unordered_map<GraphDbTypes::Property, Expression*>
* @return unordered_map<database::Property, Expression*>
*/
antlrcpp::Any visitProperties(CypherParser::PropertiesContext *ctx) override;
@ -235,7 +235,7 @@ class CypherMainVisitor : public antlropencypher::CypherBaseVisitor {
CypherParser::ListLiteralContext *ctx) override;
/**
* @return GraphDbTypes::Property
* @return database::Property
*/
antlrcpp::Any visitPropertyKeyName(
CypherParser::PropertyKeyNameContext *ctx) override;
@ -290,7 +290,7 @@ class CypherMainVisitor : public antlropencypher::CypherBaseVisitor {
CypherParser::RelationshipLambdaContext *ctx) override;
/**
* @return vector<GraphDbTypes::EdgeType>
* @return vector<database::EdgeType>
*/
antlrcpp::Any visitRelationshipTypes(
CypherParser::RelationshipTypesContext *ctx) override;

View File

@ -33,7 +33,8 @@ namespace {
// TODO: Implement degrees, haversin, radians
// TODO: Implement string and spatial functions
TypedValue Coalesce(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Coalesce(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() == 0U) {
throw QueryRuntimeException("coalesce requires at least one argument");
}
@ -45,7 +46,8 @@ TypedValue Coalesce(const std::vector<TypedValue> &args, GraphDbAccessor &) {
return TypedValue::Null;
}
TypedValue EndNode(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue EndNode(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("endNode requires one argument");
}
@ -59,7 +61,8 @@ TypedValue EndNode(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue Head(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Head(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("head requires one argument");
}
@ -76,7 +79,8 @@ TypedValue Head(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue Last(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Last(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("last requires one argument");
}
@ -94,7 +98,7 @@ TypedValue Last(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
TypedValue Properties(const std::vector<TypedValue> &args,
GraphDbAccessor &db_accessor) {
database::GraphDbAccessor &db_accessor) {
if (args.size() != 1U) {
throw QueryRuntimeException("properties requires one argument");
}
@ -117,7 +121,8 @@ TypedValue Properties(const std::vector<TypedValue> &args,
}
}
TypedValue Size(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Size(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("size requires one argument");
}
@ -141,7 +146,8 @@ TypedValue Size(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue StartNode(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue StartNode(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("startNode requires one argument");
}
@ -155,7 +161,8 @@ TypedValue StartNode(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue Degree(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Degree(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("degree requires one argument");
}
@ -171,7 +178,8 @@ TypedValue Degree(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue ToBoolean(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue ToBoolean(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("toBoolean requires one argument");
}
@ -195,7 +203,8 @@ TypedValue ToBoolean(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue ToFloat(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue ToFloat(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("toFloat requires one argument");
}
@ -217,7 +226,8 @@ TypedValue ToFloat(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue ToInteger(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue ToInteger(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("toInteger requires one argument");
}
@ -245,7 +255,7 @@ TypedValue ToInteger(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
TypedValue Type(const std::vector<TypedValue> &args,
GraphDbAccessor &db_accessor) {
database::GraphDbAccessor &db_accessor) {
if (args.size() != 1U) {
throw QueryRuntimeException("type requires one argument");
}
@ -260,7 +270,7 @@ TypedValue Type(const std::vector<TypedValue> &args,
}
TypedValue Keys(const std::vector<TypedValue> &args,
GraphDbAccessor &db_accessor) {
database::GraphDbAccessor &db_accessor) {
if (args.size() != 1U) {
throw QueryRuntimeException("keys requires one argument");
}
@ -284,7 +294,7 @@ TypedValue Keys(const std::vector<TypedValue> &args,
}
TypedValue Labels(const std::vector<TypedValue> &args,
GraphDbAccessor &db_accessor) {
database::GraphDbAccessor &db_accessor) {
if (args.size() != 1U) {
throw QueryRuntimeException("labels requires one argument");
}
@ -303,7 +313,8 @@ TypedValue Labels(const std::vector<TypedValue> &args,
}
}
TypedValue Nodes(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Nodes(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("nodes requires one argument");
}
@ -316,7 +327,7 @@ TypedValue Nodes(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
TypedValue Relationships(const std::vector<TypedValue> &args,
GraphDbAccessor &) {
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("relationships requires one argument");
}
@ -328,7 +339,8 @@ TypedValue Relationships(const std::vector<TypedValue> &args,
return std::vector<TypedValue>(edges.begin(), edges.end());
}
TypedValue Range(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Range(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 2U && args.size() != 3U) {
throw QueryRuntimeException("range requires two or three arguments");
}
@ -361,7 +373,8 @@ TypedValue Range(const std::vector<TypedValue> &args, GraphDbAccessor &) {
return list;
}
TypedValue Tail(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Tail(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("tail requires one argument");
}
@ -379,7 +392,8 @@ TypedValue Tail(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue Abs(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Abs(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("abs requires one argument");
}
@ -397,7 +411,8 @@ TypedValue Abs(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
#define WRAP_CMATH_FLOAT_FUNCTION(name, lowercased_name) \
TypedValue name(const std::vector<TypedValue> &args, GraphDbAccessor &) { \
TypedValue name(const std::vector<TypedValue> &args, \
database::GraphDbAccessor &) { \
if (args.size() != 1U) { \
throw QueryRuntimeException(#lowercased_name " requires one argument"); \
} \
@ -432,7 +447,8 @@ WRAP_CMATH_FLOAT_FUNCTION(Tan, tan)
#undef WRAP_CMATH_FLOAT_FUNCTION
TypedValue Atan2(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Atan2(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 2U) {
throw QueryRuntimeException("atan2 requires two arguments");
}
@ -453,7 +469,8 @@ TypedValue Atan2(const std::vector<TypedValue> &args, GraphDbAccessor &) {
return atan2(y, x);
}
TypedValue Sign(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Sign(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 1U) {
throw QueryRuntimeException("sign requires one argument");
}
@ -470,21 +487,23 @@ TypedValue Sign(const std::vector<TypedValue> &args, GraphDbAccessor &) {
}
}
TypedValue E(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue E(const std::vector<TypedValue> &args, database::GraphDbAccessor &) {
if (args.size() != 0U) {
throw QueryRuntimeException("e shouldn't be called with arguments");
}
return M_E;
}
TypedValue Pi(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Pi(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() != 0U) {
throw QueryRuntimeException("pi shouldn't be called with arguments");
}
return M_PI;
}
TypedValue Rand(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Rand(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
static thread_local std::mt19937 pseudo_rand_gen_{std::random_device{}()};
static thread_local std::uniform_real_distribution<> rand_dist_{0, 1};
if (args.size() != 0U) {
@ -495,7 +514,7 @@ TypedValue Rand(const std::vector<TypedValue> &args, GraphDbAccessor &) {
template <bool (*Predicate)(const std::string &s1, const std::string &s2)>
TypedValue StringMatchOperator(const std::vector<TypedValue> &args,
GraphDbAccessor &) {
database::GraphDbAccessor &) {
if (args.size() != 2U) {
throw QueryRuntimeException(
"startsWith shouldn't be called with 2 arguments");
@ -537,7 +556,8 @@ bool ContainsPredicate(const std::string &s1, const std::string &s2) {
}
auto Contains = StringMatchOperator<ContainsPredicate>;
TypedValue Assert(const std::vector<TypedValue> &args, GraphDbAccessor &) {
TypedValue Assert(const std::vector<TypedValue> &args,
database::GraphDbAccessor &) {
if (args.size() < 1U || args.size() > 2U) {
throw QueryRuntimeException("assert takes one or two arguments");
}
@ -553,7 +573,8 @@ TypedValue Assert(const std::vector<TypedValue> &args, GraphDbAccessor &) {
return args[0];
}
TypedValue Counter(const std::vector<TypedValue> &args, GraphDbAccessor &dba) {
TypedValue Counter(const std::vector<TypedValue> &args,
database::GraphDbAccessor &dba) {
if (args.size() != 1U) {
throw QueryRuntimeException("counter takes one argument");
}
@ -564,7 +585,7 @@ TypedValue Counter(const std::vector<TypedValue> &args, GraphDbAccessor &dba) {
}
TypedValue CounterSet(const std::vector<TypedValue> &args,
GraphDbAccessor &dba) {
database::GraphDbAccessor &dba) {
if (args.size() != 2U) {
throw QueryRuntimeException("counterSet takes two arguments");
}
@ -578,7 +599,7 @@ TypedValue CounterSet(const std::vector<TypedValue> &args,
}
TypedValue IndexInfo(const std::vector<TypedValue> &args,
GraphDbAccessor &dba) {
database::GraphDbAccessor &dba) {
if (args.size() != 0U)
throw QueryRuntimeException("indexInfo takes zero arguments");
@ -587,7 +608,8 @@ TypedValue IndexInfo(const std::vector<TypedValue> &args,
}
} // namespace
std::function<TypedValue(const std::vector<TypedValue> &, GraphDbAccessor &)>
std::function<TypedValue(const std::vector<TypedValue> &,
database::GraphDbAccessor &)>
NameToFunction(const std::string &function_name) {
if (function_name == "COALESCE") return Coalesce;
if (function_name == "ENDNODE") return EndNode;

View File

@ -13,6 +13,7 @@ const char kEndsWith[] = "ENDSWITH";
const char kContains[] = "CONTAINS";
}
std::function<TypedValue(const std::vector<TypedValue> &, GraphDbAccessor &)>
std::function<TypedValue(const std::vector<TypedValue> &,
database::GraphDbAccessor &)>
NameToFunction(const std::string &function_name);
}

View File

@ -20,7 +20,7 @@ class ExpressionEvaluator : public TreeVisitor<TypedValue> {
public:
ExpressionEvaluator(Frame &frame, const Parameters &parameters,
const SymbolTable &symbol_table,
GraphDbAccessor &db_accessor,
database::GraphDbAccessor &db_accessor,
GraphView graph_view = GraphView::AS_IS)
: frame_(frame),
parameters_(parameters),
@ -436,7 +436,7 @@ class ExpressionEvaluator : public TreeVisitor<TypedValue> {
Frame &frame_;
const Parameters &parameters_;
const SymbolTable &symbol_table_;
GraphDbAccessor &db_accessor_;
database::GraphDbAccessor &db_accessor_;
// which switching approach should be used when evaluating
const GraphView graph_view_;
};

View File

@ -21,7 +21,7 @@ DEFINE_VALIDATED_int32(query_plan_cache_ttl, 60,
namespace query {
Interpreter::Results Interpreter::operator()(
const std::string &query, GraphDbAccessor &db_accessor,
const std::string &query, database::GraphDbAccessor &db_accessor,
const std::map<std::string, TypedValue> &params,
bool in_explicit_transaction) {
utils::Timer frontend_timer;
@ -187,7 +187,7 @@ AstTreeStorage Interpreter::QueryToAst(const StrippedQuery &stripped,
std::pair<std::unique_ptr<plan::LogicalOperator>, double>
Interpreter::MakeLogicalPlan(AstTreeStorage &ast_storage,
const GraphDbAccessor &db_accessor,
const database::GraphDbAccessor &db_accessor,
Context &context) {
std::unique_ptr<plan::LogicalOperator> logical_plan;
auto vertex_counts = plan::MakeVertexCountCache(db_accessor);

View File

@ -154,7 +154,8 @@ class Interpreter {
* Generates an Results object for the parameters. The resulting object
* can the be Pulled with it's results written to an arbitrary stream.
*/
Results operator()(const std::string &query, GraphDbAccessor &db_accessor,
Results operator()(const std::string &query,
database::GraphDbAccessor &db_accessor,
const std::map<std::string, TypedValue> &params,
bool in_explicit_transaction);
@ -165,7 +166,7 @@ class Interpreter {
// high level tree -> (logical plan, plan cost)
// AstTreeStorage and SymbolTable may be modified during planning.
std::pair<std::unique_ptr<plan::LogicalOperator>, double> MakeLogicalPlan(
AstTreeStorage &, const GraphDbAccessor &, Context &);
AstTreeStorage &, const database::GraphDbAccessor &, Context &);
ConcurrentMap<HashType, AstTreeStorage> ast_cache_;
ConcurrentMap<HashType, std::shared_ptr<CachedPlan>> plan_cache_;

View File

@ -30,7 +30,7 @@ namespace {
// TypedValue cannot be converted to PropertyValue,
// QueryRuntimeException is raised.
template <class TRecordAccessor>
void PropsSetChecked(TRecordAccessor &record, GraphDbTypes::Property key,
void PropsSetChecked(TRecordAccessor &record, database::Property key,
TypedValue value) {
try {
record.PropsSet(key, value);
@ -70,7 +70,7 @@ bool Once::OnceCursor::Pull(Frame &, Context &) {
return false;
}
std::unique_ptr<Cursor> Once::MakeCursor(GraphDbAccessor &) const {
std::unique_ptr<Cursor> Once::MakeCursor(database::GraphDbAccessor &) const {
return std::make_unique<OnceCursor>();
}
@ -82,12 +82,13 @@ CreateNode::CreateNode(const NodeAtom *node_atom,
ACCEPT_WITH_INPUT(CreateNode)
std::unique_ptr<Cursor> CreateNode::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> CreateNode::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<CreateNodeCursor>(*this, db);
}
CreateNode::CreateNodeCursor::CreateNodeCursor(const CreateNode &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool CreateNode::CreateNodeCursor::Pull(Frame &frame, Context &context) {
@ -124,12 +125,13 @@ CreateExpand::CreateExpand(const NodeAtom *node_atom, const EdgeAtom *edge_atom,
ACCEPT_WITH_INPUT(CreateExpand)
std::unique_ptr<Cursor> CreateExpand::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> CreateExpand::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<CreateExpandCursor>(*this, db);
}
CreateExpand::CreateExpandCursor::CreateExpandCursor(const CreateExpand &self,
GraphDbAccessor &db)
CreateExpand::CreateExpandCursor::CreateExpandCursor(
const CreateExpand &self, database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool CreateExpand::CreateExpandCursor::Pull(Frame &frame, Context &context) {
@ -208,7 +210,8 @@ class ScanAllCursor : public Cursor {
public:
explicit ScanAllCursor(Symbol output_symbol,
std::unique_ptr<Cursor> &&input_cursor,
TVerticesFun &&get_vertices, GraphDbAccessor &db)
TVerticesFun &&get_vertices,
database::GraphDbAccessor &db)
: output_symbol_(output_symbol),
input_cursor_(std::move(input_cursor)),
get_vertices_(std::move(get_vertices)),
@ -246,7 +249,7 @@ class ScanAllCursor : public Cursor {
typename std::result_of<TVerticesFun(Frame &, Context &)>::type>
vertices_;
std::experimental::optional<decltype(vertices_.value().begin())> vertices_it_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
};
ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input,
@ -260,7 +263,8 @@ ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(ScanAll)
std::unique_ptr<Cursor> ScanAll::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> ScanAll::MakeCursor(
database::GraphDbAccessor &db) const {
auto vertices = [this, &db](Frame &, Context &) {
return db.Vertices(graph_view_ == GraphView::NEW);
};
@ -269,13 +273,14 @@ std::unique_ptr<Cursor> ScanAll::MakeCursor(GraphDbAccessor &db) const {
}
ScanAllByLabel::ScanAllByLabel(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, GraphDbTypes::Label label,
Symbol output_symbol, database::Label label,
GraphView graph_view)
: ScanAll(input, output_symbol, graph_view), label_(label) {}
ACCEPT_WITH_INPUT(ScanAllByLabel)
std::unique_ptr<Cursor> ScanAllByLabel::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> ScanAllByLabel::MakeCursor(
database::GraphDbAccessor &db) const {
auto vertices = [this, &db](Frame &, Context &) {
return db.Vertices(label_, graph_view_ == GraphView::NEW);
};
@ -285,7 +290,7 @@ std::unique_ptr<Cursor> ScanAllByLabel::MakeCursor(GraphDbAccessor &db) const {
ScanAllByLabelPropertyRange::ScanAllByLabelPropertyRange(
const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
GraphDbTypes::Label label, GraphDbTypes::Property property,
database::Label label, database::Property property,
std::experimental::optional<Bound> lower_bound,
std::experimental::optional<Bound> upper_bound, GraphView graph_view)
: ScanAll(input, output_symbol, graph_view),
@ -299,16 +304,16 @@ ScanAllByLabelPropertyRange::ScanAllByLabelPropertyRange(
ACCEPT_WITH_INPUT(ScanAllByLabelPropertyRange)
std::unique_ptr<Cursor> ScanAllByLabelPropertyRange::MakeCursor(
GraphDbAccessor &db) const {
database::GraphDbAccessor &db) const {
auto vertices = [this, &db](Frame &frame, Context &context) {
ExpressionEvaluator evaluator(frame, context.parameters_,
context.symbol_table_, db, graph_view_);
auto convert = [&evaluator](const auto &bound)
-> std::experimental::optional<utils::Bound<PropertyValue>> {
if (!bound) return std::experimental::nullopt;
return std::experimental::make_optional(utils::Bound<PropertyValue>(
bound.value().value()->Accept(evaluator), bound.value().type()));
};
if (!bound) return std::experimental::nullopt;
return std::experimental::make_optional(utils::Bound<PropertyValue>(
bound.value().value()->Accept(evaluator), bound.value().type()));
};
return db.Vertices(label_, property_, convert(lower_bound()),
convert(upper_bound()), graph_view_ == GraphView::NEW);
};
@ -318,8 +323,8 @@ std::unique_ptr<Cursor> ScanAllByLabelPropertyRange::MakeCursor(
ScanAllByLabelPropertyValue::ScanAllByLabelPropertyValue(
const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
GraphDbTypes::Label label, GraphDbTypes::Property property,
Expression *expression, GraphView graph_view)
database::Label label, database::Property property, Expression *expression,
GraphView graph_view)
: ScanAll(input, output_symbol, graph_view),
label_(label),
property_(property),
@ -332,7 +337,7 @@ ACCEPT_WITH_INPUT(ScanAllByLabelPropertyValue)
class ScanAllByLabelPropertyValueCursor : public Cursor {
public:
ScanAllByLabelPropertyValueCursor(const ScanAllByLabelPropertyValue &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input()->MakeCursor(db_)) {}
bool Pull(Frame &frame, Context &context) override {
@ -370,7 +375,7 @@ class ScanAllByLabelPropertyValueCursor : public Cursor {
private:
const ScanAllByLabelPropertyValue &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
std::experimental::optional<decltype(
db_.Vertices(self_.label(), self_.property(), TypedValue::Null, false))>
@ -379,15 +384,16 @@ class ScanAllByLabelPropertyValueCursor : public Cursor {
};
std::unique_ptr<Cursor> ScanAllByLabelPropertyValue::MakeCursor(
GraphDbAccessor &db) const {
database::GraphDbAccessor &db) const {
return std::make_unique<ScanAllByLabelPropertyValueCursor>(*this, db);
}
ExpandCommon::ExpandCommon(
Symbol node_symbol, Symbol edge_symbol, EdgeAtom::Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types,
const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
bool existing_node, GraphView graph_view)
ExpandCommon::ExpandCommon(Symbol node_symbol, Symbol edge_symbol,
EdgeAtom::Direction direction,
const std::vector<database::EdgeType> &edge_types,
const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, bool existing_node,
GraphView graph_view)
: node_symbol_(node_symbol),
edge_symbol_(edge_symbol),
direction_(direction),
@ -413,11 +419,13 @@ bool ExpandCommon::HandleExistingNode(const VertexAccessor &new_node,
ACCEPT_WITH_INPUT(Expand)
std::unique_ptr<Cursor> Expand::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Expand::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<ExpandCursor>(*this, db);
}
Expand::ExpandCursor::ExpandCursor(const Expand &self, GraphDbAccessor &db)
Expand::ExpandCursor::ExpandCursor(const Expand &self,
database::GraphDbAccessor &db)
: self_(self), input_cursor_(self.input_->MakeCursor(db)), db_(db) {}
bool Expand::ExpandCursor::Pull(Frame &frame, Context &context) {
@ -550,7 +558,7 @@ bool Expand::ExpandCursor::InitEdges(Frame &frame, Context &context) {
ExpandVariable::ExpandVariable(
Symbol node_symbol, Symbol edge_symbol, EdgeAtom::Type type,
EdgeAtom::Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types, bool is_reverse,
const std::vector<database::EdgeType> &edge_types, bool is_reverse,
Expression *lower_bound, Expression *upper_bound,
const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
bool existing_node, Symbol inner_edge_symbol, Symbol inner_node_symbol,
@ -586,7 +594,7 @@ namespace {
*/
auto ExpandFromVertex(const VertexAccessor &vertex,
EdgeAtom::Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types) {
const std::vector<database::EdgeType> &edge_types) {
// wraps an EdgeAccessor into a pair <accessor, direction>
auto wrapper = [](EdgeAtom::Direction direction, auto &&vertices) {
return iter::imap(
@ -637,7 +645,8 @@ int64_t EvaluateInt(ExpressionEvaluator &evaluator, Expression *expr,
class ExpandVariableCursor : public Cursor {
public:
ExpandVariableCursor(const ExpandVariable &self, GraphDbAccessor &db)
ExpandVariableCursor(const ExpandVariable &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool Pull(Frame &frame, Context &context) override {
@ -673,7 +682,7 @@ class ExpandVariableCursor : public Cursor {
private:
const ExpandVariable &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
// bounds. in the cursor they are not optional but set to
// default values if missing in the ExpandVariable operator
@ -858,7 +867,8 @@ class ExpandVariableCursor : public Cursor {
class ExpandBreadthFirstCursor : public query::plan::Cursor {
public:
ExpandBreadthFirstCursor(const ExpandVariable &self, GraphDbAccessor &db)
ExpandBreadthFirstCursor(const ExpandVariable &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Pull(Frame &frame, Context &context) override {
@ -995,7 +1005,7 @@ class ExpandBreadthFirstCursor : public query::plan::Cursor {
private:
const ExpandVariable &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<query::plan::Cursor> input_cursor_;
// Depth bounds. Calculated on each pull from the input, the initial value is
@ -1013,7 +1023,8 @@ class ExpandBreadthFirstCursor : public query::plan::Cursor {
std::deque<std::pair<EdgeAccessor, VertexAccessor>> to_visit_next_;
};
std::unique_ptr<Cursor> ExpandVariable::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> ExpandVariable::MakeCursor(
database::GraphDbAccessor &db) const {
if (type_ == EdgeAtom::Type::BREADTH_FIRST)
return std::make_unique<ExpandBreadthFirstCursor>(*this, db);
else
@ -1022,7 +1033,8 @@ std::unique_ptr<Cursor> ExpandVariable::MakeCursor(GraphDbAccessor &db) const {
class ConstructNamedPathCursor : public Cursor {
public:
ConstructNamedPathCursor(const ConstructNamedPath &self, GraphDbAccessor &db)
ConstructNamedPathCursor(const ConstructNamedPath &self,
database::GraphDbAccessor &db)
: self_(self), input_cursor_(self_.input()->MakeCursor(db)) {}
bool Pull(Frame &frame, Context &context) override {
@ -1101,7 +1113,7 @@ class ConstructNamedPathCursor : public Cursor {
ACCEPT_WITH_INPUT(ConstructNamedPath)
std::unique_ptr<Cursor> ConstructNamedPath::MakeCursor(
GraphDbAccessor &db) const {
database::GraphDbAccessor &db) const {
return std::make_unique<ConstructNamedPathCursor>(*this, db);
}
@ -1112,11 +1124,13 @@ Filter::Filter(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Filter)
std::unique_ptr<Cursor> Filter::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Filter::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<FilterCursor>(*this, db);
}
Filter::FilterCursor::FilterCursor(const Filter &self, GraphDbAccessor &db)
Filter::FilterCursor::FilterCursor(const Filter &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Filter::FilterCursor::Pull(Frame &frame, Context &context) {
@ -1139,7 +1153,8 @@ Produce::Produce(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Produce)
std::unique_ptr<Cursor> Produce::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Produce::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<ProduceCursor>(*this, db);
}
@ -1156,7 +1171,8 @@ const std::vector<NamedExpression *> &Produce::named_expressions() {
return named_expressions_;
}
Produce::ProduceCursor::ProduceCursor(const Produce &self, GraphDbAccessor &db)
Produce::ProduceCursor::ProduceCursor(const Produce &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Produce::ProduceCursor::Pull(Frame &frame, Context &context) {
@ -1179,11 +1195,13 @@ Delete::Delete(const std::shared_ptr<LogicalOperator> &input_,
ACCEPT_WITH_INPUT(Delete)
std::unique_ptr<Cursor> Delete::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Delete::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<DeleteCursor>(*this, db);
}
Delete::DeleteCursor::DeleteCursor(const Delete &self, GraphDbAccessor &db)
Delete::DeleteCursor::DeleteCursor(const Delete &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Delete::DeleteCursor::Pull(Frame &frame, Context &context) {
@ -1245,12 +1263,13 @@ SetProperty::SetProperty(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(SetProperty)
std::unique_ptr<Cursor> SetProperty::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> SetProperty::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<SetPropertyCursor>(*this, db);
}
SetProperty::SetPropertyCursor::SetPropertyCursor(const SetProperty &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool SetProperty::SetPropertyCursor::Pull(Frame &frame, Context &context) {
@ -1294,12 +1313,13 @@ SetProperties::SetProperties(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(SetProperties)
std::unique_ptr<Cursor> SetProperties::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> SetProperties::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<SetPropertiesCursor>(*this, db);
}
SetProperties::SetPropertiesCursor::SetPropertiesCursor(
const SetProperties &self, GraphDbAccessor &db)
const SetProperties &self, database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool SetProperties::SetPropertiesCursor::Pull(Frame &frame, Context &context) {
@ -1368,17 +1388,18 @@ template void SetProperties::SetPropertiesCursor::Set(
SetLabels::SetLabels(const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol,
const std::vector<GraphDbTypes::Label> &labels)
const std::vector<database::Label> &labels)
: input_(input), input_symbol_(input_symbol), labels_(labels) {}
ACCEPT_WITH_INPUT(SetLabels)
std::unique_ptr<Cursor> SetLabels::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> SetLabels::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<SetLabelsCursor>(*this, db);
}
SetLabels::SetLabelsCursor::SetLabelsCursor(const SetLabels &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), input_cursor_(self.input_->MakeCursor(db)) {}
bool SetLabels::SetLabelsCursor::Pull(Frame &frame, Context &context) {
@ -1403,12 +1424,13 @@ RemoveProperty::RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(RemoveProperty)
std::unique_ptr<Cursor> RemoveProperty::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> RemoveProperty::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<RemovePropertyCursor>(*this, db);
}
RemoveProperty::RemovePropertyCursor::RemovePropertyCursor(
const RemoveProperty &self, GraphDbAccessor &db)
const RemoveProperty &self, database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool RemoveProperty::RemovePropertyCursor::Pull(Frame &frame,
@ -1441,17 +1463,18 @@ void RemoveProperty::RemovePropertyCursor::Reset() { input_cursor_->Reset(); }
RemoveLabels::RemoveLabels(const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol,
const std::vector<GraphDbTypes::Label> &labels)
const std::vector<database::Label> &labels)
: input_(input), input_symbol_(input_symbol), labels_(labels) {}
ACCEPT_WITH_INPUT(RemoveLabels)
std::unique_ptr<Cursor> RemoveLabels::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> RemoveLabels::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<RemoveLabelsCursor>(*this, db);
}
RemoveLabels::RemoveLabelsCursor::RemoveLabelsCursor(const RemoveLabels &self,
GraphDbAccessor &db)
RemoveLabels::RemoveLabelsCursor::RemoveLabelsCursor(
const RemoveLabels &self, database::GraphDbAccessor &db)
: self_(self), input_cursor_(self.input_->MakeCursor(db)) {}
bool RemoveLabels::RemoveLabelsCursor::Pull(Frame &frame, Context &context) {
@ -1483,14 +1506,14 @@ ACCEPT_WITH_INPUT(ExpandUniquenessFilter<TAccessor>)
template <typename TAccessor>
std::unique_ptr<Cursor> ExpandUniquenessFilter<TAccessor>::MakeCursor(
GraphDbAccessor &db) const {
database::GraphDbAccessor &db) const {
return std::make_unique<ExpandUniquenessFilterCursor>(*this, db);
}
template <typename TAccessor>
ExpandUniquenessFilter<TAccessor>::ExpandUniquenessFilterCursor::
ExpandUniquenessFilterCursor(const ExpandUniquenessFilter &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), input_cursor_(self.input_->MakeCursor(db)) {}
namespace {
@ -1600,12 +1623,13 @@ Accumulate::Accumulate(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Accumulate)
std::unique_ptr<Cursor> Accumulate::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Accumulate::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<Accumulate::AccumulateCursor>(*this, db);
}
Accumulate::AccumulateCursor::AccumulateCursor(const Accumulate &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool Accumulate::AccumulateCursor::Pull(Frame &frame, Context &context) {
@ -1652,12 +1676,13 @@ Aggregate::Aggregate(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Aggregate)
std::unique_ptr<Cursor> Aggregate::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Aggregate::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<AggregateCursor>(*this, db);
}
Aggregate::AggregateCursor::AggregateCursor(const Aggregate &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
namespace {
@ -1924,7 +1949,7 @@ Skip::Skip(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Skip)
std::unique_ptr<Cursor> Skip::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Skip::MakeCursor(database::GraphDbAccessor &db) const {
return std::make_unique<SkipCursor>(*this, db);
}
@ -1933,7 +1958,7 @@ std::vector<Symbol> Skip::OutputSymbols(const SymbolTable &symbol_table) const {
return input_->OutputSymbols(symbol_table);
}
Skip::SkipCursor::SkipCursor(const Skip &self, GraphDbAccessor &db)
Skip::SkipCursor::SkipCursor(const Skip &self, database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Skip::SkipCursor::Pull(Frame &frame, Context &context) {
@ -1971,7 +1996,7 @@ Limit::Limit(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Limit)
std::unique_ptr<Cursor> Limit::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Limit::MakeCursor(database::GraphDbAccessor &db) const {
return std::make_unique<LimitCursor>(*this, db);
}
@ -1981,7 +2006,8 @@ std::vector<Symbol> Limit::OutputSymbols(
return input_->OutputSymbols(symbol_table);
}
Limit::LimitCursor::LimitCursor(const Limit &self, GraphDbAccessor &db)
Limit::LimitCursor::LimitCursor(const Limit &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Limit::LimitCursor::Pull(Frame &frame, Context &context) {
@ -2031,7 +2057,8 @@ OrderBy::OrderBy(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(OrderBy)
std::unique_ptr<Cursor> OrderBy::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> OrderBy::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<OrderByCursor>(*this, db);
}
@ -2041,7 +2068,8 @@ std::vector<Symbol> OrderBy::OutputSymbols(
return input_->OutputSymbols(symbol_table);
}
OrderBy::OrderByCursor::OrderByCursor(const OrderBy &self, GraphDbAccessor &db)
OrderBy::OrderByCursor::OrderByCursor(const OrderBy &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool OrderBy::OrderByCursor::Pull(Frame &frame, Context &context) {
@ -2176,11 +2204,12 @@ bool Merge::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
return visitor.PostVisit(*this);
}
std::unique_ptr<Cursor> Merge::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Merge::MakeCursor(database::GraphDbAccessor &db) const {
return std::make_unique<MergeCursor>(*this, db);
}
Merge::MergeCursor::MergeCursor(const Merge &self, GraphDbAccessor &db)
Merge::MergeCursor::MergeCursor(const Merge &self,
database::GraphDbAccessor &db)
: input_cursor_(self.input_->MakeCursor(db)),
merge_match_cursor_(self.merge_match_->MakeCursor(db)),
merge_create_cursor_(self.merge_create_->MakeCursor(db)) {}
@ -2242,12 +2271,13 @@ bool Optional::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
return visitor.PostVisit(*this);
}
std::unique_ptr<Cursor> Optional::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Optional::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<OptionalCursor>(*this, db);
}
Optional::OptionalCursor::OptionalCursor(const Optional &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self),
input_cursor_(self.input_->MakeCursor(db)),
optional_cursor_(self.optional_->MakeCursor(db)) {}
@ -2301,11 +2331,13 @@ Unwind::Unwind(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Unwind)
std::unique_ptr<Cursor> Unwind::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Unwind::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<UnwindCursor>(*this, db);
}
Unwind::UnwindCursor::UnwindCursor(const Unwind &self, GraphDbAccessor &db)
Unwind::UnwindCursor::UnwindCursor(const Unwind &self,
database::GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool Unwind::UnwindCursor::Pull(Frame &frame, Context &context) {
@ -2346,7 +2378,8 @@ Distinct::Distinct(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Distinct)
std::unique_ptr<Cursor> Distinct::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Distinct::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<DistinctCursor>(*this, db);
}
@ -2357,7 +2390,7 @@ std::vector<Symbol> Distinct::OutputSymbols(
}
Distinct::DistinctCursor::DistinctCursor(const Distinct &self,
GraphDbAccessor &db)
database::GraphDbAccessor &db)
: self_(self), input_cursor_(self.input_->MakeCursor(db)) {}
bool Distinct::DistinctCursor::Pull(Frame &frame, Context &context) {
@ -2377,8 +2410,7 @@ void Distinct::DistinctCursor::Reset() {
seen_rows_.clear();
}
CreateIndex::CreateIndex(GraphDbTypes::Label label,
GraphDbTypes::Property property)
CreateIndex::CreateIndex(database::Label label, database::Property property)
: label_(label), property_(property) {}
bool CreateIndex::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
@ -2387,7 +2419,7 @@ bool CreateIndex::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
class CreateIndexCursor : public Cursor {
public:
CreateIndexCursor(const CreateIndex &self, GraphDbAccessor &db)
CreateIndexCursor(const CreateIndex &self, database::GraphDbAccessor &db)
: self_(self), db_(db) {}
bool Pull(Frame &, Context &ctx) override {
@ -2397,7 +2429,7 @@ class CreateIndexCursor : public Cursor {
}
try {
db_.BuildIndex(self_.label(), self_.property());
} catch (const IndexExistsException &) {
} catch (const database::IndexExistsException &) {
// Ignore creating an existing index.
}
ctx.is_index_created_ = did_create_ = true;
@ -2408,11 +2440,12 @@ class CreateIndexCursor : public Cursor {
private:
const CreateIndex &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
bool did_create_ = false;
};
std::unique_ptr<Cursor> CreateIndex::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> CreateIndex::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<CreateIndexCursor>(*this, db);
}
@ -2427,7 +2460,7 @@ Union::Union(const std::shared_ptr<LogicalOperator> &left_op,
left_symbols_(left_symbols),
right_symbols_(right_symbols) {}
std::unique_ptr<Cursor> Union::MakeCursor(GraphDbAccessor &db) const {
std::unique_ptr<Cursor> Union::MakeCursor(database::GraphDbAccessor &db) const {
return std::make_unique<Union::UnionCursor>(*this, db);
}
@ -2444,7 +2477,8 @@ std::vector<Symbol> Union::OutputSymbols(const SymbolTable &) const {
return union_symbols_;
}
Union::UnionCursor::UnionCursor(const Union &self, GraphDbAccessor &db)
Union::UnionCursor::UnionCursor(const Union &self,
database::GraphDbAccessor &db)
: self_(self),
left_cursor_(self.left_op_->MakeCursor(db)),
right_cursor_(self.right_op_->MakeCursor(db)) {}

View File

@ -14,7 +14,7 @@
#include <vector>
#include "database/graph_db_accessor.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "query/common.hpp"
#include "query/exceptions.hpp"
#include "query/frontend/semantic/symbol_table.hpp"
@ -128,9 +128,11 @@ class LogicalOperator
public:
/** @brief Constructs a @c Cursor which is used to run this operator.
*
* @param GraphDbAccessor Used to perform operations on the database.
* @param database::GraphDbAccessor Used to perform operations on the
* database.
*/
virtual std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const = 0;
virtual std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const = 0;
/** @brief Return @c Symbol vector where the results will be stored.
*
@ -157,7 +159,8 @@ class LogicalOperator
class Once : public LogicalOperator {
public:
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
class OnceCursor : public Cursor {
@ -192,7 +195,8 @@ class CreateNode : public LogicalOperator {
CreateNode(const NodeAtom *node_atom,
const std::shared_ptr<LogicalOperator> &input);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const NodeAtom *node_atom_ = nullptr;
@ -200,13 +204,13 @@ class CreateNode : public LogicalOperator {
class CreateNodeCursor : public Cursor {
public:
CreateNodeCursor(const CreateNode &self, GraphDbAccessor &db);
CreateNodeCursor(const CreateNode &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const CreateNode &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
/**
@ -247,7 +251,8 @@ class CreateExpand : public LogicalOperator {
const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, bool existing_node);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
// info on what's getting expanded
@ -265,13 +270,13 @@ class CreateExpand : public LogicalOperator {
class CreateExpandCursor : public Cursor {
public:
CreateExpandCursor(const CreateExpand &self, GraphDbAccessor &db);
CreateExpandCursor(const CreateExpand &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const CreateExpand &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
/**
@ -316,7 +321,8 @@ class ScanAll : public LogicalOperator {
ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
auto input() const { return input_; }
auto output_symbol() const { return output_symbol_; }
@ -347,15 +353,16 @@ class ScanAll : public LogicalOperator {
class ScanAllByLabel : public ScanAll {
public:
ScanAllByLabel(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, GraphDbTypes::Label label,
Symbol output_symbol, database::Label label,
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
GraphDbTypes::Label label() const { return label_; }
database::Label label() const { return label_; }
private:
const GraphDbTypes::Label label_;
const database::Label label_;
};
/**
@ -385,14 +392,15 @@ class ScanAllByLabelPropertyRange : public ScanAll {
* @param graph_view GraphView used when obtaining vertices.
*/
ScanAllByLabelPropertyRange(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, GraphDbTypes::Label label,
GraphDbTypes::Property property,
Symbol output_symbol, database::Label label,
database::Property property,
std::experimental::optional<Bound> lower_bound,
std::experimental::optional<Bound> upper_bound,
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
auto label() const { return label_; }
auto property() const { return property_; }
@ -400,8 +408,8 @@ class ScanAllByLabelPropertyRange : public ScanAll {
auto upper_bound() const { return upper_bound_; }
private:
const GraphDbTypes::Label label_;
const GraphDbTypes::Property property_;
const database::Label label_;
const database::Property property_;
std::experimental::optional<Bound> lower_bound_;
std::experimental::optional<Bound> upper_bound_;
};
@ -427,21 +435,22 @@ class ScanAllByLabelPropertyValue : public ScanAll {
* @param graph_view GraphView used when obtaining vertices.
*/
ScanAllByLabelPropertyValue(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, GraphDbTypes::Label label,
GraphDbTypes::Property property,
Symbol output_symbol, database::Label label,
database::Property property,
Expression *expression,
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
auto label() const { return label_; }
auto property() const { return property_; }
auto expression() const { return expression_; }
private:
const GraphDbTypes::Label label_;
const GraphDbTypes::Property property_;
const database::Label label_;
const database::Property property_;
Expression *expression_;
};
@ -478,7 +487,7 @@ class ExpandCommon {
* @param direction EdgeAtom::Direction determining the direction of edge
* expansion. The direction is relative to the starting vertex for each
* expansion.
* @param edge_types GraphDbTypes::EdgeType specifying which edges we
* @param edge_types database::EdgeType specifying which edges we
* want to expand. If empty, all edges are valid. If not empty, only edges
* with one of the given types are valid.
* @param input Optional LogicalOperator that preceeds this one.
@ -489,7 +498,7 @@ class ExpandCommon {
*/
ExpandCommon(Symbol node_symbol, Symbol edge_symbol,
EdgeAtom::Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types,
const std::vector<database::EdgeType> &edge_types,
const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, bool existing_node,
GraphView graph_view = GraphView::AS_IS);
@ -505,7 +514,7 @@ class ExpandCommon {
const Symbol node_symbol_;
const Symbol edge_symbol_;
const EdgeAtom::Direction direction_;
const std::vector<GraphDbTypes::EdgeType> edge_types_;
const std::vector<database::EdgeType> edge_types_;
// the input op and the symbol under which the op's result
// can be found in the frame
@ -553,18 +562,19 @@ class Expand : public LogicalOperator, public ExpandCommon {
using ExpandCommon::ExpandCommon;
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
class ExpandCursor : public Cursor {
public:
ExpandCursor(const Expand &self, GraphDbAccessor &db);
ExpandCursor(const Expand &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Expand &self_;
const std::unique_ptr<Cursor> input_cursor_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
// The iterable over edges and the current edge iterator are referenced via
// optional because they can not be initialized in the constructor of
@ -629,7 +639,7 @@ class ExpandVariable : public LogicalOperator, public ExpandCommon {
*/
ExpandVariable(Symbol node_symbol, Symbol edge_symbol, EdgeAtom::Type type,
EdgeAtom::Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types,
const std::vector<database::EdgeType> &edge_types,
bool is_reverse, Expression *lower_bound,
Expression *upper_bound,
const std::shared_ptr<LogicalOperator> &input,
@ -639,7 +649,8 @@ class ExpandVariable : public LogicalOperator, public ExpandCommon {
GraphView graph_view = GraphView::AS_IS);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
auto type() const { return type_; }
@ -672,7 +683,8 @@ class ConstructNamedPath : public LogicalOperator {
path_symbol_(path_symbol),
path_elements_(path_elements) {}
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
const auto &input() const { return input_; }
const auto &path_symbol() const { return path_symbol_; }
@ -697,7 +709,8 @@ class Filter : public LogicalOperator {
Filter(const std::shared_ptr<LogicalOperator> &input_,
Expression *expression_);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -705,13 +718,13 @@ class Filter : public LogicalOperator {
class FilterCursor : public Cursor {
public:
FilterCursor(const Filter &self, GraphDbAccessor &db);
FilterCursor(const Filter &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Filter &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
};
};
@ -732,7 +745,8 @@ class Produce : public LogicalOperator {
Produce(const std::shared_ptr<LogicalOperator> &input,
const std::vector<NamedExpression *> &named_expressions);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
const std::vector<NamedExpression *> &named_expressions();
@ -742,13 +756,13 @@ class Produce : public LogicalOperator {
class ProduceCursor : public Cursor {
public:
ProduceCursor(const Produce &self, GraphDbAccessor &db);
ProduceCursor(const Produce &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Produce &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
};
};
@ -764,7 +778,8 @@ class Delete : public LogicalOperator {
Delete(const std::shared_ptr<LogicalOperator> &input_,
const std::vector<Expression *> &expressions, bool detach_);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -776,13 +791,13 @@ class Delete : public LogicalOperator {
class DeleteCursor : public Cursor {
public:
DeleteCursor(const Delete &self, GraphDbAccessor &db);
DeleteCursor(const Delete &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Delete &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
};
};
@ -798,7 +813,8 @@ class SetProperty : public LogicalOperator {
SetProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs, Expression *rhs);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -807,13 +823,13 @@ class SetProperty : public LogicalOperator {
class SetPropertyCursor : public Cursor {
public:
SetPropertyCursor(const SetProperty &self, GraphDbAccessor &db);
SetPropertyCursor(const SetProperty &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const SetProperty &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
};
};
@ -845,7 +861,8 @@ class SetProperties : public LogicalOperator {
SetProperties(const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, Expression *rhs, Op op);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -855,13 +872,14 @@ class SetProperties : public LogicalOperator {
class SetPropertiesCursor : public Cursor {
public:
SetPropertiesCursor(const SetProperties &self, GraphDbAccessor &db);
SetPropertiesCursor(const SetProperties &self,
database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const SetProperties &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
/** Helper function that sets the given values on either
@ -883,18 +901,19 @@ class SetProperties : public LogicalOperator {
class SetLabels : public LogicalOperator {
public:
SetLabels(const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
const std::vector<GraphDbTypes::Label> &labels);
const std::vector<database::Label> &labels);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
const Symbol input_symbol_;
const std::vector<GraphDbTypes::Label> labels_;
const std::vector<database::Label> labels_;
class SetLabelsCursor : public Cursor {
public:
SetLabelsCursor(const SetLabels &self, GraphDbAccessor &db);
SetLabelsCursor(const SetLabels &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -913,7 +932,8 @@ class RemoveProperty : public LogicalOperator {
RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -921,13 +941,14 @@ class RemoveProperty : public LogicalOperator {
class RemovePropertyCursor : public Cursor {
public:
RemovePropertyCursor(const RemoveProperty &self, GraphDbAccessor &db);
RemovePropertyCursor(const RemoveProperty &self,
database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const RemoveProperty &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
};
};
@ -941,19 +962,19 @@ class RemoveProperty : public LogicalOperator {
class RemoveLabels : public LogicalOperator {
public:
RemoveLabels(const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol,
const std::vector<GraphDbTypes::Label> &labels);
Symbol input_symbol, const std::vector<database::Label> &labels);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
const Symbol input_symbol_;
const std::vector<GraphDbTypes::Label> labels_;
const std::vector<database::Label> labels_;
class RemoveLabelsCursor : public Cursor {
public:
RemoveLabelsCursor(const RemoveLabels &self, GraphDbAccessor &db);
RemoveLabelsCursor(const RemoveLabels &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -993,7 +1014,8 @@ class ExpandUniquenessFilter : public LogicalOperator {
Symbol expand_symbol,
const std::vector<Symbol> &previous_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -1003,7 +1025,7 @@ class ExpandUniquenessFilter : public LogicalOperator {
class ExpandUniquenessFilterCursor : public Cursor {
public:
ExpandUniquenessFilterCursor(const ExpandUniquenessFilter &self,
GraphDbAccessor &db);
database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1045,7 +1067,8 @@ class Accumulate : public LogicalOperator {
Accumulate(const std::shared_ptr<LogicalOperator> &input,
const std::vector<Symbol> &symbols, bool advance_command = false);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
const auto &symbols() const { return symbols_; };
@ -1056,13 +1079,13 @@ class Accumulate : public LogicalOperator {
class AccumulateCursor : public Cursor {
public:
AccumulateCursor(const Accumulate &self, GraphDbAccessor &db);
AccumulateCursor(const Accumulate &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Accumulate &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
std::vector<std::vector<TypedValue>> cache_;
decltype(cache_.begin()) cache_it_ = cache_.begin();
@ -1111,7 +1134,8 @@ class Aggregate : public LogicalOperator {
const std::vector<Expression *> &group_by,
const std::vector<Symbol> &remember);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
const auto &aggregations() const { return aggregations_; }
const auto &group_by() const { return group_by_; }
@ -1124,7 +1148,7 @@ class Aggregate : public LogicalOperator {
class AggregateCursor : public Cursor {
public:
AggregateCursor(const Aggregate &self, GraphDbAccessor &db);
AggregateCursor(const Aggregate &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1146,7 +1170,7 @@ class Aggregate : public LogicalOperator {
};
const Aggregate &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
// storage for aggregated data
// map key is the vector of group-by values
@ -1218,7 +1242,8 @@ class Skip : public LogicalOperator {
public:
Skip(const std::shared_ptr<LogicalOperator> &input, Expression *expression);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
private:
@ -1227,13 +1252,13 @@ class Skip : public LogicalOperator {
class SkipCursor : public Cursor {
public:
SkipCursor(const Skip &self, GraphDbAccessor &db);
SkipCursor(const Skip &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Skip &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
// init to_skip_ to -1, indicating
// that it's still unknown (input has not been Pulled yet)
@ -1261,7 +1286,8 @@ class Limit : public LogicalOperator {
public:
Limit(const std::shared_ptr<LogicalOperator> &input, Expression *expression);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
private:
@ -1270,13 +1296,13 @@ class Limit : public LogicalOperator {
class LimitCursor : public Cursor {
public:
LimitCursor(const Limit &self, GraphDbAccessor &db);
LimitCursor(const Limit &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Limit &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
std::unique_ptr<Cursor> input_cursor_;
// init limit_ to -1, indicating
// that it's still unknown (Cursor has not been Pulled yet)
@ -1302,7 +1328,8 @@ class OrderBy : public LogicalOperator {
const std::vector<std::pair<Ordering, Expression *>> &order_by,
const std::vector<Symbol> &output_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
const auto &output_symbols() const { return output_symbols_; }
@ -1338,13 +1365,13 @@ class OrderBy : public LogicalOperator {
class OrderByCursor : public Cursor {
public:
OrderByCursor(const OrderBy &self, GraphDbAccessor &db);
OrderByCursor(const OrderBy &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const OrderBy &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
bool did_pull_all_{false};
// a cache of elements pulled from the input
@ -1377,7 +1404,8 @@ class Merge : public LogicalOperator {
const std::shared_ptr<LogicalOperator> &merge_match,
const std::shared_ptr<LogicalOperator> &merge_create);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
auto input() const { return input_; }
auto merge_match() const { return merge_match_; }
@ -1390,7 +1418,7 @@ class Merge : public LogicalOperator {
class MergeCursor : public Cursor {
public:
MergeCursor(const Merge &self, GraphDbAccessor &db);
MergeCursor(const Merge &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1422,7 +1450,8 @@ class Optional : public LogicalOperator {
const std::shared_ptr<LogicalOperator> &optional,
const std::vector<Symbol> &optional_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
auto input() const { return input_; }
auto optional() const { return optional_; }
@ -1435,7 +1464,7 @@ class Optional : public LogicalOperator {
class OptionalCursor : public Cursor {
public:
OptionalCursor(const Optional &self, GraphDbAccessor &db);
OptionalCursor(const Optional &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1463,7 +1492,8 @@ class Unwind : public LogicalOperator {
Unwind(const std::shared_ptr<LogicalOperator> &input,
Expression *input_expression_, Symbol output_symbol);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
Expression *input_expression() const { return input_expression_; }
@ -1474,13 +1504,13 @@ class Unwind : public LogicalOperator {
class UnwindCursor : public Cursor {
public:
UnwindCursor(const Unwind &self, GraphDbAccessor &db);
UnwindCursor(const Unwind &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
const Unwind &self_;
GraphDbAccessor &db_;
database::GraphDbAccessor &db_;
const std::unique_ptr<Cursor> input_cursor_;
// typed values we are unwinding and yielding
std::vector<TypedValue> input_value_;
@ -1502,7 +1532,8 @@ class Distinct : public LogicalOperator {
Distinct(const std::shared_ptr<LogicalOperator> &input,
const std::vector<Symbol> &value_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
private:
@ -1511,7 +1542,7 @@ class Distinct : public LogicalOperator {
class DistinctCursor : public Cursor {
public:
DistinctCursor(const Distinct &self, GraphDbAccessor &db);
DistinctCursor(const Distinct &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1539,16 +1570,17 @@ class Distinct : public LogicalOperator {
*/
class CreateIndex : public LogicalOperator {
public:
CreateIndex(GraphDbTypes::Label label, GraphDbTypes::Property property);
CreateIndex(database::Label label, database::Property property);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
auto label() const { return label_; }
auto property() const { return property_; }
private:
GraphDbTypes::Label label_;
GraphDbTypes::Property property_;
database::Label label_;
database::Property property_;
};
/**
@ -1566,7 +1598,8 @@ class Union : public LogicalOperator {
const std::vector<Symbol> &left_symbols,
const std::vector<Symbol> &right_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
private:
@ -1575,7 +1608,7 @@ class Union : public LogicalOperator {
class UnionCursor : public Cursor {
public:
UnionCursor(const Union &self, GraphDbAccessor &db);
UnionCursor(const Union &self, database::GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;

View File

@ -174,7 +174,7 @@ auto SplitExpressionOnAnd(Expression *expression) {
PropertyFilter::PropertyFilter(const SymbolTable &symbol_table,
const Symbol &symbol,
const GraphDbTypes::Property &property,
const database::Property &property,
Expression *value)
: symbol_(symbol), property_(property), value_(value) {
UsedSymbolsCollector collector(symbol_table);
@ -184,7 +184,7 @@ PropertyFilter::PropertyFilter(const SymbolTable &symbol_table,
PropertyFilter::PropertyFilter(
const SymbolTable &symbol_table, const Symbol &symbol,
const GraphDbTypes::Property &property,
const database::Property &property,
const std::experimental::optional<PropertyFilter::Bound> &lower_bound,
const std::experimental::optional<PropertyFilter::Bound> &upper_bound)
: symbol_(symbol),
@ -212,7 +212,7 @@ void Filters::EraseFilter(const FilterInfo &filter) {
}
void Filters::EraseLabelFilter(const Symbol &symbol,
const GraphDbTypes::Label &label) {
const database::Label &label) {
for (auto filter_it = all_filters_.begin();
filter_it != all_filters_.end();) {
if (filter_it->type != FilterInfo::Type::Label) {

View File

@ -37,15 +37,15 @@ class PropertyFilter {
using Bound = ScanAllByLabelPropertyRange::Bound;
PropertyFilter(const SymbolTable &, const Symbol &,
const GraphDbTypes::Property &, Expression *);
const database::Property &, Expression *);
PropertyFilter(const SymbolTable &, const Symbol &,
const GraphDbTypes::Property &,
const database::Property &,
const std::experimental::optional<Bound> &,
const std::experimental::optional<Bound> &);
/// Symbol whose property is looked up.
Symbol symbol_;
GraphDbTypes::Property property_;
database::Property property_;
/// True if the same symbol is used in expressions for value or bounds.
bool is_symbol_in_value_ = false;
/// Expression which when evaluated produces the value a property must
@ -70,7 +70,7 @@ struct FilterInfo {
/// Set of used symbols by the filter @c expression.
std::unordered_set<Symbol> used_symbols;
/// Labels for Type::Label filtering.
std::vector<GraphDbTypes::Label> labels;
std::vector<database::Label> labels;
/// Property information for Type::Property filtering.
std::experimental::optional<PropertyFilter> property_filter;
};
@ -101,7 +101,7 @@ class Filters {
}
auto FilteredLabels(const Symbol &symbol) const {
std::unordered_set<GraphDbTypes::Label> labels;
std::unordered_set<database::Label> labels;
for (const auto &filter : all_filters_) {
if (filter.type == FilterInfo::Type::Label &&
utils::Contains(filter.used_symbols, symbol)) {
@ -119,7 +119,7 @@ class Filters {
void EraseFilter(const FilterInfo &);
// Remove a label filter for symbol; may invalidate iterators.
void EraseLabelFilter(const Symbol &, const GraphDbTypes::Label &);
void EraseLabelFilter(const Symbol &, const database::Label &);
// Returns a vector of FilterInfo for properties.
auto PropertyFilters(const Symbol &symbol) const {

View File

@ -196,7 +196,7 @@ class RuleBasedPlanner {
TPlanningContext &context_;
struct LabelPropertyIndex {
GraphDbTypes::Label label;
database::Label label;
// FilterInfo with PropertyFilter.
FilterInfo filter;
int64_t vertex_count;
@ -240,8 +240,8 @@ class RuleBasedPlanner {
return found;
}
const GraphDbTypes::Label &FindBestLabelIndex(
const std::unordered_set<GraphDbTypes::Label> &labels) {
const database::Label &FindBestLabelIndex(
const std::unordered_set<database::Label> &labels) {
DCHECK(!labels.empty())
<< "Trying to find the best label without any labels.";
return *std::min_element(labels.begin(), labels.end(),

View File

@ -3,7 +3,7 @@
#include <experimental/optional>
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "storage/property_value.hpp"
#include "utils/bound.hpp"
#include "utils/hashing/fnv.hpp"
@ -22,14 +22,14 @@ class VertexCountCache {
return *vertices_count_;
}
int64_t VerticesCount(const GraphDbTypes::Label &label) const {
int64_t VerticesCount(const database::Label &label) const {
if (label_vertex_count_.find(label) == label_vertex_count_.end())
label_vertex_count_[label] = db_.VerticesCount(label);
return label_vertex_count_.at(label);
}
int64_t VerticesCount(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property) const {
int64_t VerticesCount(const database::Label &label,
const database::Property &property) const {
auto key = std::make_pair(label, property);
if (label_property_vertex_count_.find(key) ==
label_property_vertex_count_.end())
@ -37,8 +37,8 @@ class VertexCountCache {
return label_property_vertex_count_.at(key);
}
int64_t VerticesCount(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property,
int64_t VerticesCount(const database::Label &label,
const database::Property &property,
const PropertyValue &value) const {
auto label_prop = std::make_pair(label, property);
auto &value_vertex_count = property_value_vertex_count_[label_prop];
@ -48,7 +48,7 @@ class VertexCountCache {
}
int64_t VerticesCount(
const GraphDbTypes::Label &label, const GraphDbTypes::Property &property,
const database::Label &label, const database::Property &property,
const std::experimental::optional<utils::Bound<PropertyValue>> &lower,
const std::experimental::optional<utils::Bound<PropertyValue>> &upper)
const {
@ -61,19 +61,18 @@ class VertexCountCache {
return bounds_vertex_count.at(bounds);
}
bool LabelPropertyIndexExists(const GraphDbTypes::Label &label,
const GraphDbTypes::Property &property) const {
bool LabelPropertyIndexExists(const database::Label &label,
const database::Property &property) const {
return db_.LabelPropertyIndexExists(label, property);
}
private:
typedef std::pair<GraphDbTypes::Label, GraphDbTypes::Property>
LabelPropertyKey;
typedef std::pair<database::Label, database::Property> LabelPropertyKey;
struct LabelPropertyHash {
size_t operator()(const LabelPropertyKey &key) const {
return HashCombine<GraphDbTypes::Label, GraphDbTypes::Property>{}(
key.first, key.second);
return HashCombine<database::Label, database::Property>{}(key.first,
key.second);
}
};
@ -113,7 +112,7 @@ class VertexCountCache {
const TDbAccessor &db_;
mutable std::experimental::optional<int64_t> vertices_count_;
mutable std::unordered_map<GraphDbTypes::Label, int64_t> label_vertex_count_;
mutable std::unordered_map<database::Label, int64_t> label_vertex_count_;
mutable std::unordered_map<LabelPropertyKey, int64_t, LabelPropertyHash>
label_property_vertex_count_;
mutable std::unordered_map<

View File

@ -10,7 +10,7 @@ namespace storage {
* for the master (single source of truth) and worker (must query master).
* Both implementations must be concurrent.
*
* @TParam TId - One of GraphDb types (Label, EdgeType, Property).
* @TParam TId - One of database::GraphDb types (Label, EdgeType, Property).
*/
template <typename TId>
class ConcurrentIdMapper {

View File

@ -1,6 +1,6 @@
#include "glog/logging.h"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "storage/concurrent_id_mapper_master.hpp"
#include "storage/concurrent_id_mapper_rpc_messages.hpp"
@ -22,7 +22,7 @@ void RegisterRpc(MasterConcurrentIdMapper<TId> &mapper,
}); \
}
using namespace GraphDbTypes;
using namespace database;
ID_VALUE_RPC_CALLS(Label)
ID_VALUE_RPC_CALLS(EdgeType)
ID_VALUE_RPC_CALLS(Property)

View File

@ -13,7 +13,7 @@ namespace storage {
template <typename TId>
class MasterConcurrentIdMapper : public SingleNodeConcurrentIdMapper<TId> {
public:
MasterConcurrentIdMapper(communication::messaging::System &system);
explicit MasterConcurrentIdMapper(communication::messaging::System &system);
private:
communication::rpc::Server rpc_server_;

View File

@ -3,7 +3,7 @@
#include <chrono>
#include "communication/rpc/rpc.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "transactions/commit_log.hpp"
#include "transactions/snapshot.hpp"
#include "transactions/type.hpp"
@ -16,10 +16,10 @@ const auto kConcurrentIdMapperRpcTimeout = 300ms;
#define ID_VALUE_RPC(type) \
RPC_SINGLE_MEMBER_MESSAGE(type##IdReq, std::string); \
RPC_SINGLE_MEMBER_MESSAGE(type##IdRes, GraphDbTypes::type); \
RPC_SINGLE_MEMBER_MESSAGE(type##IdRes, database::type); \
using type##IdRpc = \
communication::rpc::RequestResponse<type##IdReq, type##IdRes>; \
RPC_SINGLE_MEMBER_MESSAGE(Id##type##Req, GraphDbTypes::type); \
RPC_SINGLE_MEMBER_MESSAGE(Id##type##Req, database::type); \
RPC_SINGLE_MEMBER_MESSAGE(Id##type##Res, std::string); \
using Id##type##Rpc = \
communication::rpc::RequestResponse<Id##type##Req, Id##type##Res>;

View File

@ -1,7 +1,7 @@
#include "glog/logging.h"
#include "concurrent_id_mapper_worker.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "storage/concurrent_id_mapper_rpc_messages.hpp"
namespace storage {
@ -24,7 +24,7 @@ namespace storage {
return response->member; \
}
using namespace GraphDbTypes;
using namespace database;
ID_VALUE_RPC_CALLS(Label)
ID_VALUE_RPC_CALLS(EdgeType)
ID_VALUE_RPC_CALLS(Property)

View File

@ -1,6 +1,6 @@
#pragma once
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "mvcc/record.hpp"
#include "mvcc/version_list.hpp"
#include "storage/address.hpp"
@ -12,7 +12,7 @@ class Edge : public mvcc::Record<Edge> {
using VertexAddress = storage::Address<mvcc::VersionList<Vertex>>;
public:
Edge(VertexAddress from, VertexAddress to, GraphDbTypes::EdgeType edge_type)
Edge(VertexAddress from, VertexAddress to, database::EdgeType edge_type)
: from_(from), to_(to), edge_type_(edge_type) {}
// Returns new Edge with copy of data stored in this Edge, but without
@ -21,7 +21,7 @@ class Edge : public mvcc::Record<Edge> {
VertexAddress from_;
VertexAddress to_;
GraphDbTypes::EdgeType edge_type_;
database::EdgeType edge_type_;
PropertyValueStore properties_;
private:

View File

@ -4,7 +4,7 @@
#include "storage/vertex_accessor.hpp"
#include "utils/algorithm.hpp"
GraphDbTypes::EdgeType EdgeAccessor::EdgeType() const { return edge_type_; }
database::EdgeType EdgeAccessor::EdgeType() const { return edge_type_; }
VertexAccessor EdgeAccessor::from() const {
return VertexAccessor(from_, db_accessor());
@ -27,10 +27,9 @@ bool EdgeAccessor::is_cycle() const { return to_ == from_; }
std::ostream &operator<<(std::ostream &os, const EdgeAccessor &ea) {
os << "E[" << ea.db_accessor().EdgeTypeName(ea.EdgeType());
os << " {";
utils::PrintIterable(os, ea.Properties(), ", ",
[&](auto &stream, const auto &pair) {
stream << ea.db_accessor().PropertyName(pair.first)
<< ": " << pair.second;
});
utils::PrintIterable(os, ea.Properties(), ", ", [&](auto &stream,
const auto &pair) {
stream << ea.db_accessor().PropertyName(pair.first) << ": " << pair.second;
});
return os << "}]";
}

View File

@ -25,7 +25,7 @@ class EdgeAccessor : public RecordAccessor<Edge> {
public:
/** Constructor that reads data from the random memory location (lower
* performance, see class docs). */
EdgeAccessor(EdgeAddress address, GraphDbAccessor &db_accessor)
EdgeAccessor(EdgeAddress address, database::GraphDbAccessor &db_accessor)
: RecordAccessor(address, db_accessor),
from_(nullptr),
to_(nullptr),
@ -40,15 +40,15 @@ class EdgeAccessor : public RecordAccessor<Edge> {
/** Constructor that does NOT data from the random memory location (better
* performance, see class docs). */
EdgeAccessor(EdgeAddress address, GraphDbAccessor &db_accessor,
EdgeAccessor(EdgeAddress address, database::GraphDbAccessor &db_accessor,
VertexAddress from, VertexAddress to,
GraphDbTypes::EdgeType edge_type)
database::EdgeType edge_type)
: RecordAccessor(address, db_accessor),
from_(from),
to_(to),
edge_type_(edge_type) {}
GraphDbTypes::EdgeType EdgeType() const;
database::EdgeType EdgeType() const;
/** Returns an accessor to the originating Vertex of this edge. */
VertexAccessor from() const;
@ -71,7 +71,7 @@ class EdgeAccessor : public RecordAccessor<Edge> {
private:
VertexAddress from_;
VertexAddress to_;
GraphDbTypes::EdgeType edge_type_;
database::EdgeType edge_type_;
};
std::ostream &operator<<(std::ostream &, const EdgeAccessor &);

View File

@ -3,8 +3,9 @@
#include <utility>
#include <vector>
#include "database/graph_db_datatypes.hpp"
#include "glog/logging.h"
#include "database/types.hpp"
#include "mvcc/version_list.hpp"
#include "storage/address.hpp"
#include "utils/algorithm.hpp"
@ -27,7 +28,7 @@ class Edges {
struct Element {
VertexAddress vertex;
EdgeAddress edge;
GraphDbTypes::EdgeType edge_type;
database::EdgeType edge_type;
};
/** Custom iterator that takes care of skipping edges when the destination
@ -55,7 +56,7 @@ class Edges {
*/
Iterator(std::vector<Element>::const_iterator position,
std::vector<Element>::const_iterator end, VertexAddress vertex,
const std::vector<GraphDbTypes::EdgeType> *edge_types)
const std::vector<database::EdgeType> *edge_types)
: position_(position),
end_(end),
vertex_(vertex),
@ -87,7 +88,7 @@ class Edges {
// iterator. Only one can be not-null in the current implementation.
VertexAddress vertex_{nullptr};
// For edge types we use a vector pointer because it's optional.
const std::vector<GraphDbTypes::EdgeType> *edge_types_ = nullptr;
const std::vector<database::EdgeType> *edge_types_ = nullptr;
/** Helper function that skips edges that don't satisfy the predicate
* present in this iterator. */
@ -115,7 +116,7 @@ class Edges {
* @param edge_type - Type of the edge.
*/
void emplace(VertexAddress vertex, EdgeAddress edge,
GraphDbTypes::EdgeType edge_type) {
database::EdgeType edge_type) {
storage_.emplace_back(Element{vertex, edge, edge_type});
}
@ -145,7 +146,7 @@ class Edges {
* If nullptr edges are not filtered on type.
*/
auto begin(VertexAddress vertex,
const std::vector<GraphDbTypes::EdgeType> *edge_types) const {
const std::vector<database::EdgeType> *edge_types) const {
if (edge_types && edge_types->empty()) edge_types = nullptr;
return Iterator(storage_.begin(), storage_.end(), vertex, edge_types);
}

View File

@ -4,7 +4,7 @@
#include <map>
#include <vector>
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "property_value.hpp"
/**
@ -14,7 +14,7 @@
* The underlying implementation is not necessarily std::map.
*/
class PropertyValueStore {
using Property = GraphDbTypes::Property;
using Property = database::Property;
public:
/**

View File

@ -9,17 +9,17 @@ using database::StateDelta;
template <typename TRecord>
RecordAccessor<TRecord>::RecordAccessor(AddressT address,
GraphDbAccessor &db_accessor)
database::GraphDbAccessor &db_accessor)
: db_accessor_(&db_accessor), address_(address) {}
template <typename TRecord>
const PropertyValue &RecordAccessor<TRecord>::PropsAt(
GraphDbTypes::Property key) const {
database::Property key) const {
return current().properties_.at(key);
}
template <>
void RecordAccessor<Vertex>::PropsSet(GraphDbTypes::Property key,
void RecordAccessor<Vertex>::PropsSet(database::Property key,
PropertyValue value) {
Vertex &vertex = update();
vertex.properties_.set(key, value);
@ -33,7 +33,7 @@ void RecordAccessor<Vertex>::PropsSet(GraphDbTypes::Property key,
}
template <>
void RecordAccessor<Edge>::PropsSet(GraphDbTypes::Property key,
void RecordAccessor<Edge>::PropsSet(database::Property key,
PropertyValue value) {
update().properties_.set(key, value);
auto &dba = db_accessor();
@ -43,7 +43,7 @@ void RecordAccessor<Edge>::PropsSet(GraphDbTypes::Property key,
}
template <>
size_t RecordAccessor<Vertex>::PropsErase(GraphDbTypes::Property key) {
size_t RecordAccessor<Vertex>::PropsErase(database::Property key) {
auto &dba = db_accessor();
// TODO use the delta for handling.
dba.wal().Emplace(StateDelta::PropsSetVertex(
@ -52,7 +52,7 @@ size_t RecordAccessor<Vertex>::PropsErase(GraphDbTypes::Property key) {
}
template <>
size_t RecordAccessor<Edge>::PropsErase(GraphDbTypes::Property key) {
size_t RecordAccessor<Edge>::PropsErase(database::Property key) {
auto &dba = db_accessor();
// TODO use the delta for handling.
dba.wal().Emplace(StateDelta::PropsSetEdge(
@ -96,7 +96,7 @@ bool RecordAccessor<TRecord>::operator==(const RecordAccessor &other) const {
}
template <typename TRecord>
GraphDbAccessor &RecordAccessor<TRecord>::db_accessor() const {
database::GraphDbAccessor &RecordAccessor<TRecord>::db_accessor() const {
return *db_accessor_;
}

View File

@ -2,7 +2,7 @@
#include "glog/logging.h"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "mvcc/version_list.hpp"
#include "storage/address.hpp"
#include "storage/gid.hpp"
@ -10,7 +10,9 @@
#include "storage/property_value_store.hpp"
#include "utils/total_ordering.hpp"
namespace database {
class GraphDbAccessor;
};
/// Mock class for a DB delta.
// TODO replace with the real thing.
@ -37,16 +39,19 @@ class GraphStateDelta {
*/
template <typename TRecord>
class RecordAccessor : public TotalOrdering<RecordAccessor<TRecord>> {
protected:
using AddressT = storage::Address<mvcc::VersionList<TRecord>>;
/**
* The GraphDbAccessor is friend to this accessor so it can
* The database::GraphDbAccessor is friend to this accessor so it can
* operate on it's data (mvcc version-list and the record itself).
* This is legitemate because GraphDbAccessor creates RecordAccessors
* This is legitemate because database::GraphDbAccessor creates
* RecordAccessors
* and is semantically their parent/owner. It is necessary because
* the GraphDbAccessor handles insertions and deletions, and these
* the database::GraphDbAccessor handles insertions and deletions, and these
* operations modify data intensively.
*/
friend GraphDbAccessor;
friend database::GraphDbAccessor;
public:
/**
@ -54,7 +59,7 @@ class RecordAccessor : public TotalOrdering<RecordAccessor<TRecord>> {
* accessor.
* @param db_accessor The DB accessor that "owns" this record accessor.
*/
RecordAccessor(AddressT address, GraphDbAccessor &db_accessor);
RecordAccessor(AddressT address, database::GraphDbAccessor &db_accessor);
// this class is default copyable, movable and assignable
RecordAccessor(const RecordAccessor &other) = default;
@ -63,13 +68,13 @@ class RecordAccessor : public TotalOrdering<RecordAccessor<TRecord>> {
RecordAccessor &operator=(RecordAccessor &&other) = default;
/** Gets the property for the given key. */
const PropertyValue &PropsAt(GraphDbTypes::Property key) const;
const PropertyValue &PropsAt(database::Property key) const;
/** Sets a value on the record for the given property. */
void PropsSet(GraphDbTypes::Property key, PropertyValue value);
void PropsSet(database::Property key, PropertyValue value);
/** Erases the property for the given key. */
size_t PropsErase(GraphDbTypes::Property key);
size_t PropsErase(database::Property key);
/** Removes all the properties from this record. */
void PropsClear();
@ -80,7 +85,7 @@ class RecordAccessor : public TotalOrdering<RecordAccessor<TRecord>> {
bool operator==(const RecordAccessor &other) const;
/** Returns a GraphDB accessor of this record accessor. */
GraphDbAccessor &db_accessor() const;
database::GraphDbAccessor &db_accessor() const;
/**
* Returns a globally-unique ID of this vertex or edge. Note that vertices
@ -176,7 +181,7 @@ class RecordAccessor : public TotalOrdering<RecordAccessor<TRecord>> {
// The database accessor for which this record accessor is created
// Provides means of getting to the transaction and database functions.
// Immutable, set in the constructor and never changed.
GraphDbAccessor *db_accessor_;
database::GraphDbAccessor *db_accessor_;
AddressT address_;

View File

@ -1,6 +1,6 @@
#pragma once
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "mvcc/record.hpp"
#include "mvcc/version_list.hpp"
#include "storage/address.hpp"
@ -16,7 +16,7 @@ class Vertex : public mvcc::Record<Vertex> {
Edges out_;
Edges in_;
std::vector<GraphDbTypes::Label> labels_;
std::vector<database::Label> labels_;
PropertyValueStore properties_;
private:

View File

@ -9,7 +9,7 @@ size_t VertexAccessor::out_degree() const { return current().out_.size(); }
size_t VertexAccessor::in_degree() const { return current().in_.size(); }
bool VertexAccessor::add_label(GraphDbTypes::Label label) {
bool VertexAccessor::add_label(database::Label label) {
auto &labels_view = current().labels_;
auto found = std::find(labels_view.begin(), labels_view.end(), label);
if (found != labels_view.end()) return false;
@ -25,7 +25,7 @@ bool VertexAccessor::add_label(GraphDbTypes::Label label) {
return true;
}
size_t VertexAccessor::remove_label(GraphDbTypes::Label label) {
size_t VertexAccessor::remove_label(database::Label label) {
auto &labels = update().labels_;
auto found = std::find(labels.begin(), labels.end(), label);
if (found == labels.end()) return 0;
@ -39,12 +39,12 @@ size_t VertexAccessor::remove_label(GraphDbTypes::Label label) {
return 1;
}
bool VertexAccessor::has_label(GraphDbTypes::Label label) const {
bool VertexAccessor::has_label(database::Label label) const {
auto &labels = this->current().labels_;
return std::find(labels.begin(), labels.end(), label) != labels.end();
}
const std::vector<GraphDbTypes::Label> &VertexAccessor::labels() const {
const std::vector<database::Label> &VertexAccessor::labels() const {
return this->current().labels_;
}

View File

@ -30,9 +30,9 @@ class VertexAccessor : public RecordAccessor<Vertex> {
// @param db_accessor - database accessor
// @return - Iterator over EdgeAccessors
template <typename TIterator>
static inline auto MakeAccessorIterator(TIterator &&begin, TIterator &&end,
bool from, VertexAddress vertex,
GraphDbAccessor &db_accessor) {
static inline auto MakeAccessorIterator(
TIterator &&begin, TIterator &&end, bool from, VertexAddress vertex,
database::GraphDbAccessor &db_accessor) {
return iter::imap(
[from, vertex, &db_accessor](auto &edges_element) {
if (from) {
@ -49,7 +49,7 @@ class VertexAccessor : public RecordAccessor<Vertex> {
}
public:
VertexAccessor(VertexAddress address, GraphDbAccessor &db_accessor)
VertexAccessor(VertexAddress address, database::GraphDbAccessor &db_accessor)
: RecordAccessor(address, db_accessor) {
RecordAccessor::Reconstruct();
}
@ -63,17 +63,17 @@ class VertexAccessor : public RecordAccessor<Vertex> {
/** Adds a label to the Vertex. If the Vertex already has that label the call
* has no effect. */
// TODO revise return value, is it necessary?
bool add_label(GraphDbTypes::Label label);
bool add_label(database::Label label);
/** Removes a label from the Vertex. Return number of removed (0, 1). */
// TODO reves return value, is it necessary?
size_t remove_label(GraphDbTypes::Label label);
size_t remove_label(database::Label label);
/** Indicates if the Vertex has the given label. */
bool has_label(GraphDbTypes::Label label) const;
bool has_label(database::Label label) const;
/** Returns all the Labels of the Vertex. */
const std::vector<GraphDbTypes::Label> &labels() const;
const std::vector<database::Label> &labels() const;
/** Returns EdgeAccessors for all incoming edges. */
auto in() const {
@ -88,9 +88,8 @@ class VertexAccessor : public RecordAccessor<Vertex> {
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto in(
const VertexAccessor &dest,
const std::vector<GraphDbTypes::EdgeType> *edge_types = nullptr) const {
auto in(const VertexAccessor &dest,
const std::vector<database::EdgeType> *edge_types = nullptr) const {
return MakeAccessorIterator(current().in_.begin(dest.address(), edge_types),
current().in_.end(), false, address(),
db_accessor());
@ -102,7 +101,7 @@ class VertexAccessor : public RecordAccessor<Vertex> {
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto in(const std::vector<GraphDbTypes::EdgeType> *edge_types) const {
auto in(const std::vector<database::EdgeType> *edge_types) const {
return MakeAccessorIterator(current().in_.begin(nullptr, edge_types),
current().in_.end(), false, address(),
db_accessor());
@ -122,9 +121,8 @@ class VertexAccessor : public RecordAccessor<Vertex> {
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto out(
const VertexAccessor &dest,
const std::vector<GraphDbTypes::EdgeType> *edge_types = nullptr) const {
auto out(const VertexAccessor &dest,
const std::vector<database::EdgeType> *edge_types = nullptr) const {
return MakeAccessorIterator(
current().out_.begin(dest.address(), edge_types), current().out_.end(),
true, address(), db_accessor());
@ -136,7 +134,7 @@ class VertexAccessor : public RecordAccessor<Vertex> {
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto out(const std::vector<GraphDbTypes::EdgeType> *edge_types) const {
auto out(const std::vector<database::EdgeType> *edge_types) const {
return MakeAccessorIterator(current().out_.begin(nullptr, edge_types),
current().out_.end(), true, address(),
db_accessor());

View File

@ -43,6 +43,7 @@
/// of the flag as a string is implicitly bound to the `flagname` variable.
#include <cstdint>
#include <iostream>
#include <string>
#include "gflags/gflags.h"

View File

@ -1,8 +1,3 @@
//
// Copyright 2017 Memgraph
// Created by Florijan Stamenkovic on 23.03.17.
//
#pragma once
#include <algorithm>
@ -14,7 +9,7 @@
#include "data_structures/concurrent/skiplist.hpp"
#include "database/graph_db_accessor.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "mvcc/version_list.hpp"
#include "storage/property_value.hpp"
#include "storage/vertex_accessor.hpp"
@ -38,7 +33,7 @@ auto RandomIntGenerator(int from, int to) {
*/
class RandomGraphGenerator {
public:
explicit RandomGraphGenerator(GraphDb &db) : db_(db) {}
explicit RandomGraphGenerator(database::GraphDb &db) : db_(db) {}
/**
* Adds a progress listener that gets notified when
@ -66,13 +61,13 @@ class RandomGraphGenerator {
*/
void AddVertices(int count, const std::vector<std::string> &label_names,
int thread_count, int batch_size = 2000) {
GraphDbAccessor dba(db_);
std::vector<GraphDbTypes::Label> labels;
database::GraphDbAccessor dba(db_);
std::vector<database::Label> labels;
for (const auto &label_name : label_names)
labels.push_back(dba.Label(label_name));
Map(
[&labels, this](GraphDbAccessor &dba) {
[&labels, this](database::GraphDbAccessor &dba) {
auto vertex = dba.InsertVertex();
for (auto label : labels) vertex.add_label(label);
NotifyProgressListeners();
@ -86,7 +81,7 @@ class RandomGraphGenerator {
* regardless of their labels.
*/
int64_t VertexCount() const {
GraphDbAccessor accessor(db_);
database::GraphDbAccessor accessor(db_);
return CountIterable(accessor.Vertices(true));
}
@ -111,7 +106,7 @@ class RandomGraphGenerator {
auto vertices_from = FilterVertices(from_filter);
auto vertices_to = FilterVertices(to_filter);
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
auto edge_type = dba.EdgeType(edge_type_name);
// for small vertex counts reduce the batch size
@ -119,7 +114,8 @@ class RandomGraphGenerator {
std::min(batch_size, static_cast<int>(dba.VerticesCount() / 1000 + 1));
Map(
[&vertices_from, &vertices_to, edge_type, this](GraphDbAccessor &dba) {
[&vertices_from, &vertices_to, edge_type,
this](database::GraphDbAccessor &dba) {
auto from =
dba.Transfer(vertices_from[rand() % vertices_from.size()]);
auto to = dba.Transfer(vertices_to[rand() % vertices_to.size()]);
@ -137,7 +133,7 @@ class RandomGraphGenerator {
* regardless of their types and origin/destination labels.
*/
int64_t EdgeCount() const {
GraphDbAccessor accessor(db_);
database::GraphDbAccessor accessor(db_);
return CountIterable(accessor.Edges(true));
}
@ -155,7 +151,7 @@ class RandomGraphGenerator {
const std::string &prop_name, std::function<TValue()> value_generator,
std::function<bool(VertexAccessor &va)> predicate = {}) {
if (!predicate) predicate = [](VertexAccessor &) { return true; };
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
auto property = dba.Property(prop_name);
for (VertexAccessor va : dba.Vertices(false))
if (predicate(va)) va.PropsSet(property, value_generator());
@ -163,7 +159,7 @@ class RandomGraphGenerator {
}
private:
GraphDb &db_;
database::GraphDb &db_;
// progress listeners, they get notified about vertices and edges being
// created
@ -184,7 +180,7 @@ class RandomGraphGenerator {
std::function<bool(VertexAccessor &item)> predicate = {}) {
if (!predicate) predicate = [](VertexAccessor &) { return true; };
std::vector<VertexAccessor> r_val;
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
for (VertexAccessor &item : dba.Vertices(false))
if (predicate(item)) r_val.emplace_back(item);
@ -201,7 +197,7 @@ class RandomGraphGenerator {
* threads. Returns only once all of the threads have
* finished.
*/
void Map(std::function<void(GraphDbAccessor &)> f, int count,
void Map(std::function<void(database::GraphDbAccessor &)> f, int count,
int thread_count, int elements_per_commit) {
DCHECK(thread_count > 0) << "Can't work on less then 1 thread";
@ -215,7 +211,7 @@ 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) {
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
try {
int apply_count =
std::min(elements_per_commit, count_per_thread - i);

View File

@ -3,19 +3,21 @@
#include <glog/logging.h>
#include "communication/result_stream_faker.hpp"
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "query/interpreter.hpp"
#include "query/typed_value.hpp"
class ExpansionBenchFixture : public benchmark::Fixture {
protected:
// GraphDb shouldn't be global constructed/destructed. See documentation in
// database/graph_db.hpp for details.
std::experimental::optional<GraphDb> db_;
// GraphDb shouldn't be global constructed/destructed. See
// documentation in database/graph_db.hpp for details.
std::experimental::optional<database::SingleNode> db_;
query::Interpreter interpeter_;
void SetUp(const benchmark::State &state) override {
db_.emplace();
GraphDbAccessor dba(*db_);
database::GraphDbAccessor dba(*db_);
for (int i = 0; i < state.range(0); i++) dba.InsertVertex();
// the fixed part is one vertex expanding to 1000 others
@ -30,7 +32,7 @@ class ExpansionBenchFixture : public benchmark::Fixture {
}
void TearDown(const benchmark::State &) override {
GraphDbAccessor dba(*db_);
database::GraphDbAccessor dba(*db_);
for (auto vertex : dba.Vertices(false)) dba.DetachRemoveVertex(vertex);
dba.Commit();
db_ = std::experimental::nullopt;
@ -39,7 +41,7 @@ class ExpansionBenchFixture : public benchmark::Fixture {
BENCHMARK_DEFINE_F(ExpansionBenchFixture, Match)(benchmark::State &state) {
auto query = "MATCH (s:Start) return s";
GraphDbAccessor dba(*db_);
database::GraphDbAccessor dba(*db_);
while (state.KeepRunning()) {
ResultStreamFaker results;
interpeter_(query, dba, {}, false).PullAll(results);
@ -53,7 +55,7 @@ BENCHMARK_REGISTER_F(ExpansionBenchFixture, Match)
BENCHMARK_DEFINE_F(ExpansionBenchFixture, Expand)(benchmark::State &state) {
auto query = "MATCH (s:Start) WITH s MATCH (s)--(d) RETURN count(d)";
GraphDbAccessor dba(*db_);
database::GraphDbAccessor dba(*db_);
while (state.KeepRunning()) {
ResultStreamFaker results;
interpeter_(query, dba, {}, false).PullAll(results);

View File

@ -2,6 +2,7 @@
#include <benchmark/benchmark_api.h>
#include "database/graph_db.hpp"
#include "query/frontend/semantic/symbol_generator.hpp"
#include "query/plan/cost_estimator.hpp"
#include "query/plan/planner.hpp"
@ -29,8 +30,8 @@ static void AddChainedMatches(int num_matches, query::AstTreeStorage &storage) {
}
static void BM_PlanChainedMatches(benchmark::State &state) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
while (state.KeepRunning()) {
state.PauseTiming();
query::AstTreeStorage storage;
@ -61,8 +62,8 @@ BENCHMARK(BM_PlanChainedMatches)
->Unit(benchmark::kMillisecond);
static void AddIndexedMatches(
int num_matches, const GraphDbTypes::Label &label,
const std::pair<std::string, GraphDbTypes::Property> &property,
int num_matches, const database::Label &label,
const std::pair<std::string, database::Property> &property,
query::AstTreeStorage &storage) {
for (int i = 0; i < num_matches; ++i) {
auto *match = storage.Create<query::Match>();
@ -82,11 +83,11 @@ static void AddIndexedMatches(
}
static auto CreateIndexedVertices(int index_count, int vertex_count,
GraphDb &db) {
auto label = GraphDbAccessor(db).Label("label");
auto prop = GraphDbAccessor(db).Property("prop");
GraphDbAccessor(db).BuildIndex(label, prop);
GraphDbAccessor dba(db);
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);
for (int vi = 0; vi < vertex_count; ++vi) {
for (int index = 0; index < index_count; ++index) {
auto vertex = dba.InsertVertex();
@ -99,13 +100,13 @@ static auto CreateIndexedVertices(int index_count, int vertex_count,
}
static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
GraphDb db;
GraphDbTypes::Label label;
GraphDbTypes::Property prop;
database::SingleNode db;
database::Label label;
database::Property prop;
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
Parameters parameters;
while (state.KeepRunning()) {
state.PauseTiming();
@ -132,13 +133,13 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(
benchmark::State &state) {
GraphDb db;
GraphDbTypes::Label label;
GraphDbTypes::Property prop;
database::SingleNode db;
database::Label label;
database::Property prop;
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex_counts = query::plan::MakeVertexCountCache(dba);
Parameters parameters;
while (state.KeepRunning()) {

View File

@ -5,6 +5,7 @@
#include <gflags/gflags.h>
#include <glog/logging.h>
#include "database/graph_db.hpp"
#include "query/console.hpp"
#include "query/interpreter.hpp"
#include "utils/random_graph_generator.hpp"
@ -43,7 +44,8 @@ class ProgressReporter {
std::mutex mutex_{};
};
void random_generate(GraphDb &db, int64_t node_count, int64_t edge_count) {
void random_generate(database::GraphDb &db, int64_t node_count,
int64_t edge_count) {
utils::RandomGraphGenerator generator(db);
ProgressReporter reporter(node_count, edge_count,
std::max(1l, (node_count + edge_count) / 100));
@ -69,7 +71,7 @@ int main(int argc, char *argv[]) {
// TODO switch to GFlags, once finally available
if (argc > 3) google::InitGoogleLogging(argv[0]);
GraphDb db;
database::SingleNode db;
std::cout << "Generating graph..." << std::endl;
// fill_db;
random_generate(db, node_count, edge_count);

View File

@ -10,6 +10,8 @@
#include "gflags/gflags.h"
#include "glog/logging.h"
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "query/context.hpp"
#include "query/frontend/ast/ast.hpp"
#include "query/frontend/ast/cypher_main_visitor.hpp"
@ -132,13 +134,13 @@ class Timer {
// Dummy DbAccessor which forwards user input for various vertex counts.
class InteractiveDbAccessor {
public:
InteractiveDbAccessor(GraphDbAccessor &dba, int64_t vertices_count,
InteractiveDbAccessor(database::GraphDbAccessor &dba, int64_t vertices_count,
Timer &timer)
: dba_(dba), vertices_count_(vertices_count), timer_(timer) {}
int64_t VerticesCount() const { return vertices_count_; }
int64_t VerticesCount(const GraphDbTypes::Label &label_id) const {
int64_t VerticesCount(const database::Label &label_id) const {
auto label = dba_.LabelName(label_id);
if (label_vertex_count_.find(label) == label_vertex_count_.end()) {
label_vertex_count_[label] = ReadVertexCount("label '" + label + "'");
@ -146,8 +148,8 @@ class InteractiveDbAccessor {
return label_vertex_count_.at(label);
}
int64_t VerticesCount(const GraphDbTypes::Label &label_id,
const GraphDbTypes::Property &property_id) const {
int64_t VerticesCount(const database::Label &label_id,
const database::Property &property_id) const {
auto label = dba_.LabelName(label_id);
auto property = dba_.PropertyName(property_id);
auto key = std::make_pair(label, property);
@ -159,8 +161,8 @@ class InteractiveDbAccessor {
return label_property_vertex_count_.at(key);
}
int64_t VerticesCount(const GraphDbTypes::Label &label_id,
const GraphDbTypes::Property &property_id,
int64_t VerticesCount(const database::Label &label_id,
const database::Property &property_id,
const PropertyValue &value) const {
auto label = dba_.LabelName(label_id);
auto property = dba_.PropertyName(property_id);
@ -180,8 +182,7 @@ class InteractiveDbAccessor {
}
int64_t VerticesCount(
const GraphDbTypes::Label &label_id,
const GraphDbTypes::Property &property_id,
const database::Label &label_id, const database::Property &property_id,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper)
const {
@ -201,9 +202,8 @@ class InteractiveDbAccessor {
"' in range " + range_string.str());
}
bool LabelPropertyIndexExists(
const GraphDbTypes::Label &label_id,
const GraphDbTypes::Property &property_id) const {
bool LabelPropertyIndexExists(const database::Label &label_id,
const database::Property &property_id) const {
auto label = dba_.LabelName(label_id);
auto property = dba_.PropertyName(property_id);
auto key = std::make_pair(label, property);
@ -318,7 +318,7 @@ class InteractiveDbAccessor {
private:
typedef std::pair<std::string, std::string> LabelPropertyKey;
GraphDbAccessor &dba_;
database::GraphDbAccessor &dba_;
int64_t vertices_count_;
Timer &timer_;
mutable std::map<std::string, int64_t> label_vertex_count_;
@ -368,7 +368,7 @@ class PlanPrinter : public query::plan::HierarchicalLogicalOperatorVisitor {
using HierarchicalLogicalOperatorVisitor::PreVisit;
using HierarchicalLogicalOperatorVisitor::Visit;
explicit PlanPrinter(GraphDbAccessor &dba) : dba_(dba) {}
explicit PlanPrinter(database::GraphDbAccessor &dba) : dba_(dba) {}
#define PRE_VISIT(TOp) \
bool PreVisit(query::plan::TOp &) override { \
@ -517,7 +517,7 @@ class PlanPrinter : public query::plan::HierarchicalLogicalOperatorVisitor {
}
int depth_ = 0;
GraphDbAccessor &dba_;
database::GraphDbAccessor &dba_;
};
// Shorthand for a vector of pairs (logical_plan, cost).
@ -529,7 +529,8 @@ typedef std::vector<
struct Command {
typedef std::vector<std::string> Args;
// Function of this command
std::function<void(GraphDbAccessor &, PlansWithCost &, const Args &)>
std::function<void(database::GraphDbAccessor &, PlansWithCost &,
const Args &)>
function;
// Number of arguments the function works with.
int arg_count;
@ -537,8 +538,8 @@ struct Command {
std::string documentation;
};
#define DEFCOMMAND(Name) \
void Name##Command(GraphDbAccessor &dba, PlansWithCost &plans, \
#define DEFCOMMAND(Name) \
void Name##Command(database::GraphDbAccessor &dba, PlansWithCost &plans, \
const Command::Args &args)
DEFCOMMAND(Top) {
@ -591,7 +592,7 @@ DEFCOMMAND(Help) {
#undef DEFCOMMAND
void ExaminePlans(
GraphDbAccessor &dba,
database::GraphDbAccessor &dba,
std::vector<std::pair<std::unique_ptr<query::plan::LogicalOperator>,
double>> &plans) {
while (true) {
@ -617,7 +618,8 @@ void ExaminePlans(
}
}
query::AstTreeStorage MakeAst(const std::string &query, GraphDbAccessor &dba) {
query::AstTreeStorage MakeAst(const std::string &query,
database::GraphDbAccessor &dba) {
query::Context ctx(dba);
// query -> AST
auto parser = std::make_unique<query::frontend::opencypher::Parser>(query);
@ -672,8 +674,8 @@ int main(int argc, char *argv[]) {
std::cerr << "File '" << in_db_filename << "' does not exist!" << std::endl;
std::exit(EXIT_FAILURE);
}
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
Timer planning_timer;
InteractiveDbAccessor interactive_db(
dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0,

View File

@ -11,8 +11,8 @@ int main(int argc, char *argv[]) {
std::cout << "Usage: ./single_query 'RETURN \"query here\"'" << std::endl;
exit(1);
}
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
ResultStreamFaker results;
query::Interpreter()(argv[1], dba, {}, false).PullAll(results);
std::cout << results;

View File

@ -24,12 +24,12 @@ RC_GTEST_PROP(RandomGraph, RandomGraph, (std::vector<std::string> vertex_labels,
int vertices_num = vertex_labels.size();
int edges_num = edge_types.size();
GraphDb db;
database::SingleNode db;
std::vector<VertexAccessor> vertices;
std::unordered_map<VertexAccessor, std::string> vertex_label_map;
std::unordered_map<EdgeAccessor, std::string> edge_type_map;
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
for (auto label : vertex_labels) {
auto vertex_accessor = dba.InsertVertex();

View File

@ -164,8 +164,8 @@ TEST(BoltEncoder, VertexAndEdge) {
output.clear();
// create vertex
GraphDb db;
GraphDbAccessor db_accessor(db);
database::SingleNode db;
database::GraphDbAccessor db_accessor(db);
auto va1 = db_accessor.InsertVertex();
auto va2 = db_accessor.InsertVertex();
auto l1 = db_accessor.Label("label1");

View File

@ -4,12 +4,14 @@
#include "bolt_common.hpp"
#include "communication/bolt/v1/encoder/result_stream.hpp"
#include "communication/bolt/v1/session.hpp"
#include "database/graph_db.hpp"
// TODO: This could be done in fixture.
// Shortcuts for writing variable initializations in tests
#define INIT_VARS \
TestSocket socket(10); \
SessionData session_data; \
database::SingleNode db; \
SessionData session_data{db}; \
SessionT session(std::move(socket), session_data); \
std::vector<uint8_t> &output = session.socket_.output;

View File

@ -3,13 +3,13 @@
#include "gtest/gtest.h"
#include "communication/messaging/distributed.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "storage/concurrent_id_mapper_master.hpp"
#include "storage/concurrent_id_mapper_worker.hpp"
using namespace communication::messaging;
using namespace storage;
using namespace GraphDbTypes;
using namespace database;
template <typename TId>
class DistributedConcurrentIdMapperTest : public ::testing::Test {
@ -31,7 +31,6 @@ class DistributedConcurrentIdMapperTest : public ::testing::Test {
}
};
using namespace GraphDbTypes;
typedef ::testing::Types<Label, EdgeType, Property> GraphDbTestTypes;
TYPED_TEST_CASE(DistributedConcurrentIdMapperTest, GraphDbTestTypes);

View File

@ -5,10 +5,10 @@
#include "gmock/gmock.h"
#include "gtest/gtest.h"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "storage/concurrent_id_mapper_single_node.hpp"
using Id = GraphDbTypes::Label;
using Id = database::Label;
using Mapper = storage::SingleNodeConcurrentIdMapper<Id>;
TEST(ConcurrentIdMapper, SameValueGivesSameId) {

View File

@ -31,8 +31,8 @@ using testing::UnorderedElementsAre;
class Base {
public:
explicit Base(const std::string &query) : query_string_(query) {}
GraphDb db_;
GraphDbAccessor db_accessor_{db_};
database::SingleNode db_;
database::GraphDbAccessor db_accessor_{db_};
Context context_{db_accessor_};
std::string query_string_;
@ -889,7 +889,7 @@ TYPED_TEST(CypherMainVisitorTest, NodePattern) {
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, GraphDbTypes::Property>, int64_t> properties;
std::map<std::pair<std::string, database::Property>, int64_t> properties;
for (auto x : node->properties_) {
TypedValue value = LiteralValue(ast_generator.context_, x.second);
ASSERT_TRUE(value.type() == TypedValue::Type::Int);
@ -989,7 +989,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternDetails) {
edge->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"),
ast_generator.db_accessor_.EdgeType("type2")));
std::map<std::pair<std::string, GraphDbTypes::Property>, int64_t> properties;
std::map<std::pair<std::string, database::Property>, int64_t> properties;
for (auto x : edge->properties_) {
TypedValue value = LiteralValue(ast_generator.context_, x.second);
ASSERT_TRUE(value.type() == TypedValue::Type::Int);

View File

@ -1,8 +1,9 @@
#include "gmock/gmock.h"
#include "gtest/gtest.h"
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "storage/vertex.hpp"
#include "transactions/engine_single_node.hpp"
@ -12,10 +13,11 @@ using testing::UnorderedElementsAreArray;
// Test index does it insert everything uniquely
TEST(LabelsIndex, UniqueInsert) {
KeyIndex<GraphDbTypes::Label, Vertex> index;
GraphDb db;
GraphDbAccessor dba(db);
database::KeyIndex<database::Label, Vertex> index;
database::SingleNode db;
database::GraphDbAccessor dba(db);
tx::SingleNodeEngine engine;
auto t1 = engine.Begin();
mvcc::VersionList<Vertex> vlist(*t1, 0);
engine.Commit(*t1);
@ -40,9 +42,9 @@ TEST(LabelsIndex, UniqueInsert) {
// Check if index filters duplicates.
TEST(LabelsIndex, UniqueFilter) {
GraphDb db;
KeyIndex<GraphDbTypes::Label, Vertex> index;
GraphDbAccessor dba(db);
database::SingleNode db;
database::KeyIndex<database::Label, Vertex> index;
database::GraphDbAccessor dba(db);
tx::SingleNodeEngine engine;
auto t1 = engine.Begin();
@ -80,9 +82,9 @@ TEST(LabelsIndex, UniqueFilter) {
// Delete not anymore relevant recods from index.
TEST(LabelsIndex, Refresh) {
KeyIndex<GraphDbTypes::Label, Vertex> index;
GraphDb db;
GraphDbAccessor access(db);
database::KeyIndex<database::Label, Vertex> index;
database::SingleNode db;
database::GraphDbAccessor access(db);
tx::SingleNodeEngine engine;
// add two vertices to database
@ -120,8 +122,8 @@ TEST(LabelsIndex, Refresh) {
// Transaction hasn't ended and so the vertex is not visible.
TEST(LabelsIndexDb, AddGetZeroLabels) {
GraphDb db;
GraphDbAccessor dba(db);
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);
@ -133,9 +135,9 @@ TEST(LabelsIndexDb, AddGetZeroLabels) {
// Test label index by adding and removing one vertex, and removing label from
// another, while the third one with an irrelevant label exists.
TEST(LabelsIndexDb, AddGetRemoveLabel) {
GraphDb db;
database::SingleNode db;
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex1 = dba.InsertVertex();
vertex1.add_label(dba.Label("test"));
@ -149,7 +151,7 @@ TEST(LabelsIndexDb, AddGetRemoveLabel) {
dba.Commit();
} // Finish transaction.
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto filtered = dba.Vertices(dba.Label("test"), false);
std::vector<VertexAccessor> collection(filtered.begin(), filtered.end());
@ -178,7 +180,7 @@ TEST(LabelsIndexDb, AddGetRemoveLabel) {
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto filtered = dba.Vertices(dba.Label("test"), false);
std::vector<VertexAccessor> collection(filtered.begin(), filtered.end());

View File

@ -2,12 +2,13 @@
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/indexes/label_property_index.hpp"
#include "transactions/engine_single_node.hpp"
#include "database/types.hpp"
#include "mvcc_gc_common.hpp"
using namespace database;
class LabelPropertyIndexComplexTest : public ::testing::Test {
protected:
virtual void SetUp() {
@ -40,7 +41,7 @@ class LabelPropertyIndexComplexTest : public ::testing::Test {
}
public:
GraphDb db_;
SingleNode db_;
LabelPropertyIndex index;
LabelPropertyIndex::Key *key;
@ -50,14 +51,14 @@ class LabelPropertyIndexComplexTest : public ::testing::Test {
mvcc::VersionList<Vertex> *vlist;
Vertex *vertex;
GraphDbTypes::Label label;
GraphDbTypes::Property property;
GraphDbTypes::Label label2;
GraphDbTypes::Property property2;
Label label;
Property property;
Label label2;
Property property2;
};
TEST(LabelPropertyIndex, CreateIndex) {
GraphDb db;
SingleNode db;
GraphDbAccessor accessor(db);
LabelPropertyIndex::Key key(accessor.Label("test"),
accessor.Property("test2"));
@ -67,7 +68,7 @@ TEST(LabelPropertyIndex, CreateIndex) {
}
TEST(LabelPropertyIndex, IndexExistance) {
GraphDb db;
SingleNode db;
GraphDbAccessor accessor(db);
LabelPropertyIndex::Key key(accessor.Label("test"),
accessor.Property("test2"));
@ -80,7 +81,7 @@ TEST(LabelPropertyIndex, IndexExistance) {
}
TEST(LabelPropertyIndex, Count) {
GraphDb db;
SingleNode db;
GraphDbAccessor accessor(db);
auto label = accessor.Label("label");
auto property = accessor.Property("property");

View File

@ -9,7 +9,7 @@ DECLARE_int32(query_execution_time_sec);
TEST(TransactionTimeout, TransactionTimeout) {
FLAGS_query_execution_time_sec = 3;
GraphDb db;
database::SingleNode db;
query::Interpreter interpreter;
auto interpret = [&](auto &dba, const std::string &query) {
ResultStreamFaker stream;
@ -17,16 +17,16 @@ TEST(TransactionTimeout, TransactionTimeout) {
};
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
interpret(dba, "MATCH (n) RETURN n");
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
std::this_thread::sleep_for(std::chrono::seconds(5));
ASSERT_THROW(interpret(dba, "MATCH (n) RETURN n"), query::HintedAbortError);
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
interpret(dba, "MATCH (n) RETURN n");
}
}

View File

@ -4,6 +4,7 @@
#include "boost/archive/binary_iarchive.hpp"
#include "boost/archive/binary_oarchive.hpp"
#include "database/types.hpp"
#include "distributed/serialization.hpp"
#include "mvcc/version_list.hpp"
#include "query/typed_value.hpp"
@ -12,7 +13,7 @@
#include "storage/vertex.hpp"
#include "transactions/engine_single_node.hpp"
using namespace GraphDbTypes;
using namespace database;
template <typename TAddress>
TAddress ToGlobal(const TAddress &address, int worker_id) {

View File

@ -38,7 +38,7 @@ class DbGenerator {
}
public:
DbGenerator(GraphDbAccessor &dba) : dba_(dba) {}
DbGenerator(database::GraphDbAccessor &dba) : dba_(dba) {}
void BuildIndex(int seq_number) {
dba_.BuildIndex(Label(seq_number % kLabelCount),
@ -113,7 +113,7 @@ class DbGenerator {
}
private:
GraphDbAccessor &dba_;
database::GraphDbAccessor &dba_;
std::vector<int64_t> vertex_ids_;
std::vector<int64_t> edge_ids_;
@ -150,9 +150,9 @@ class DbGenerator {
/** Checks if the given databases have the same contents (indices,
* vertices and edges). */
void CompareDbs(GraphDb &a, GraphDb &b) {
GraphDbAccessor dba_a(a);
GraphDbAccessor dba_b(b);
void CompareDbs(database::GraphDb &a, database::GraphDb &b) {
database::GraphDbAccessor dba_a(a);
database::GraphDbAccessor dba_b(b);
{
auto index_a = dba_a.IndexInfo();
@ -235,7 +235,8 @@ fs::path GetLastFile(fs::path dir) {
return *std::max_element(files.begin(), files.end());
}
void MakeDb(GraphDbAccessor &dba, int scale, std::vector<int> indices = {}) {
void MakeDb(database::GraphDbAccessor &dba, int scale,
std::vector<int> indices = {}) {
DbGenerator generator{dba};
for (int i = 0; i < scale; i++) generator.InsertVertex();
for (int i = 0; i < scale * 2; i++) generator.InsertEdge();
@ -260,8 +261,8 @@ void MakeDb(GraphDbAccessor &dba, int scale, std::vector<int> indices = {}) {
for (auto index : indices) generator.BuildIndex(index);
}
void MakeDb(GraphDb &db, int scale, std::vector<int> indices = {}) {
GraphDbAccessor dba{db};
void MakeDb(database::GraphDb &db, int scale, std::vector<int> indices = {}) {
database::GraphDbAccessor dba{db};
MakeDb(dba, scale, indices);
dba.Commit();
}
@ -278,7 +279,7 @@ class Durability : public ::testing::Test {
}
auto DbConfig() {
GraphDb::Config config;
database::Config config;
config.durability_enabled = false;
config.durability_directory = durability_dir_;
config.snapshot_on_exit = false;
@ -287,7 +288,7 @@ class Durability : public ::testing::Test {
return config;
}
void MakeSnapshot(GraphDb &db, int snapshot_max_retained = -1) {
void MakeSnapshot(database::GraphDb &db, int snapshot_max_retained = -1) {
ASSERT_TRUE(
durability::MakeSnapshot(db, durability_dir_, snapshot_max_retained));
}
@ -312,8 +313,8 @@ TEST_F(Durability, WalEncoding) {
{
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
GraphDbAccessor dba(db);
database::SingleNode db{config};
database::GraphDbAccessor dba(db);
auto v0 = dba.InsertVertex();
ASSERT_EQ(v0.gid(), gid0);
v0.add_label(dba.Label("l0"));
@ -378,8 +379,8 @@ TEST_F(Durability, SnapshotEncoding) {
auto gid1 = generator.Next();
auto gid2 = generator.Next();
{
GraphDb db{DbConfig()};
GraphDbAccessor dba(db);
database::SingleNode db{DbConfig()};
database::GraphDbAccessor dba(db);
auto v0 = dba.InsertVertex();
ASSERT_EQ(v0.gid(), gid0);
v0.add_label(dba.Label("l0"));
@ -487,7 +488,7 @@ TEST_F(Durability, SnapshotEncoding) {
}
TEST_F(Durability, SnapshotRecovery) {
GraphDb db{DbConfig()};
database::SingleNode db{DbConfig()};
MakeDb(db, 300, {0, 1, 2});
MakeDb(db, 300);
MakeDb(db, 300, {3, 4});
@ -495,20 +496,20 @@ TEST_F(Durability, SnapshotRecovery) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
database::SingleNode recovered{recovered_config};
CompareDbs(db, recovered);
}
}
TEST_F(Durability, SnapshotNoVerticesIdRecovery) {
GraphDb db{DbConfig()};
database::SingleNode db{DbConfig()};
MakeDb(db, 10);
// Erase all vertices, this should cause snapshot to not have any more
// 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
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
for (auto vertex : dba.Vertices(false)) dba.RemoveVertex(vertex);
dba.Commit();
}
@ -517,18 +518,18 @@ TEST_F(Durability, SnapshotNoVerticesIdRecovery) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
EXPECT_EQ(db.VertexGenerator().LocalCount(),
recovered.VertexGenerator().LocalCount());
EXPECT_EQ(db.EdgeGenerator().LocalCount(),
recovered.EdgeGenerator().LocalCount());
database::SingleNode recovered{recovered_config};
EXPECT_EQ(db.storage().VertexGenerator().LocalCount(),
recovered.storage().VertexGenerator().LocalCount());
EXPECT_EQ(db.storage().EdgeGenerator().LocalCount(),
recovered.storage().EdgeGenerator().LocalCount());
}
}
TEST_F(Durability, SnapshotAndWalIdRecovery) {
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
database::SingleNode db{config};
MakeDb(db, 300);
MakeSnapshot(db);
MakeDb(db, 300);
@ -539,18 +540,18 @@ TEST_F(Durability, SnapshotAndWalIdRecovery) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
EXPECT_EQ(db.VertexGenerator().LocalCount(),
recovered.VertexGenerator().LocalCount());
EXPECT_EQ(db.EdgeGenerator().LocalCount(),
recovered.EdgeGenerator().LocalCount());
database::SingleNode recovered{recovered_config};
EXPECT_EQ(db.storage().VertexGenerator().LocalCount(),
recovered.storage().VertexGenerator().LocalCount());
EXPECT_EQ(db.storage().EdgeGenerator().LocalCount(),
recovered.storage().EdgeGenerator().LocalCount());
}
}
TEST_F(Durability, OnlyWalIdRecovery) {
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
database::SingleNode db{config};
MakeDb(db, 300);
// Sleep to ensure the WAL gets flushed.
std::this_thread::sleep_for(std::chrono::milliseconds(50));
@ -559,18 +560,18 @@ TEST_F(Durability, OnlyWalIdRecovery) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
EXPECT_EQ(db.VertexGenerator().LocalCount(),
recovered.VertexGenerator().LocalCount());
EXPECT_EQ(db.EdgeGenerator().LocalCount(),
recovered.EdgeGenerator().LocalCount());
database::SingleNode recovered{recovered_config};
EXPECT_EQ(db.storage().VertexGenerator().LocalCount(),
recovered.storage().VertexGenerator().LocalCount());
EXPECT_EQ(db.storage().EdgeGenerator().LocalCount(),
recovered.storage().EdgeGenerator().LocalCount());
}
}
TEST_F(Durability, WalRecovery) {
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
database::SingleNode db{config};
MakeDb(db, 300, {0, 1, 2});
MakeDb(db, 300);
MakeDb(db, 300, {3, 4});
@ -583,7 +584,7 @@ TEST_F(Durability, WalRecovery) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
database::SingleNode recovered{recovered_config};
CompareDbs(db, recovered);
}
}
@ -591,7 +592,7 @@ TEST_F(Durability, WalRecovery) {
TEST_F(Durability, SnapshotAndWalRecovery) {
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
database::SingleNode db{config};
MakeDb(db, 300, {0, 1, 2});
MakeDb(db, 300);
MakeSnapshot(db);
@ -607,7 +608,7 @@ TEST_F(Durability, SnapshotAndWalRecovery) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
database::SingleNode recovered{recovered_config};
CompareDbs(db, recovered);
}
}
@ -615,19 +616,19 @@ TEST_F(Durability, SnapshotAndWalRecovery) {
TEST_F(Durability, SnapshotAndWalRecoveryAfterComplexTxSituation) {
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
database::SingleNode db{config};
// The first transaction modifies and commits.
GraphDbAccessor dba_1{db};
database::GraphDbAccessor dba_1{db};
MakeDb(dba_1, 100);
dba_1.Commit();
// The second transaction will commit after snapshot.
GraphDbAccessor dba_2{db};
database::GraphDbAccessor dba_2{db};
MakeDb(dba_2, 100);
// The third transaction modifies and commits.
GraphDbAccessor dba_3{db};
database::GraphDbAccessor dba_3{db};
MakeDb(dba_3, 100);
dba_3.Commit();
@ -635,16 +636,16 @@ TEST_F(Durability, SnapshotAndWalRecoveryAfterComplexTxSituation) {
dba_2.Commit();
// The fifth transaction starts and commits after snapshot.
GraphDbAccessor dba_5{db};
database::GraphDbAccessor dba_5{db};
MakeDb(dba_5, 100);
dba_5.Commit();
// The sixth transaction will not commit at all.
GraphDbAccessor dba_6{db};
database::GraphDbAccessor dba_6{db};
MakeDb(dba_6, 100);
auto VisibleVertexCount = [](GraphDb &db) {
GraphDbAccessor dba{db};
auto VisibleVertexCount = [](database::GraphDb &db) {
database::GraphDbAccessor dba{db};
auto vertices = dba.Vertices(false);
return std::distance(vertices.begin(), vertices.end());
};
@ -657,7 +658,7 @@ TEST_F(Durability, SnapshotAndWalRecoveryAfterComplexTxSituation) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
database::SingleNode recovered{recovered_config};
ASSERT_EQ(VisibleVertexCount(recovered), 400);
CompareDbs(db, recovered);
}
@ -668,7 +669,7 @@ TEST_F(Durability, NoWalDuringRecovery) {
{
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
database::SingleNode db{config};
MakeDb(db, 300, {0, 1, 2});
// Sleep to ensure the WAL gets flushed.
@ -680,13 +681,13 @@ TEST_F(Durability, NoWalDuringRecovery) {
{
auto recovered_config = DbConfig();
recovered_config.db_recover_on_startup = true;
GraphDb recovered{recovered_config};
database::SingleNode recovered{recovered_config};
EXPECT_EQ(DirFiles(wal_dir_).size(), wal_files_before);
}
}
TEST_F(Durability, SnapshotRetention) {
GraphDb db{DbConfig()};
database::SingleNode db{DbConfig()};
for (auto &pair : {std::pair<int, int>{5, 10}, {5, 3}, {7, -1}}) {
CleanDurability();
int count, retain;
@ -713,7 +714,7 @@ TEST_F(Durability, WalRetention) {
{
auto config = DbConfig();
config.durability_enabled = true;
GraphDb db{config};
database::SingleNode db{config};
MakeDb(db, 100);
MakeSnapshot(db);
MakeDb(db, 100);
@ -732,7 +733,7 @@ TEST_F(Durability, SnapshotOnExit) {
{
auto config = DbConfig();
config.snapshot_on_exit = true;
GraphDb graph_db{config};
database::SingleNode graph_db{config};
}
EXPECT_EQ(DirFiles(snapshot_dir_).size(), 1);
}

View File

@ -4,19 +4,19 @@
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/indexes/label_property_index.hpp"
#include "database/types.hpp"
TEST(GraphDbTest, GarbageCollectIndices) {
GraphDb::Config config;
database::Config config;
config.gc_cycle_sec = -1;
GraphDb graph_db{config};
std::unique_ptr<GraphDbAccessor> dba =
std::make_unique<GraphDbAccessor>(graph_db);
database::SingleNode graph_db{config};
std::unique_ptr<database::GraphDbAccessor> dba =
std::make_unique<database::GraphDbAccessor>(graph_db);
auto commit = [&] {
dba->Commit();
dba = std::make_unique<GraphDbAccessor>(graph_db);
dba = std::make_unique<database::GraphDbAccessor>(graph_db);
};
auto label = dba->Label("label");
auto property = dba->Property("property");

View File

@ -7,13 +7,15 @@
#include "storage/edge_accessor.hpp"
#include "storage/vertex_accessor.hpp"
using namespace database;
template <typename TIterable>
auto Count(TIterable iterable) {
return std::distance(iterable.begin(), iterable.end());
}
TEST(GraphDbAccessorTest, InsertVertex) {
GraphDb db;
SingleNode db;
GraphDbAccessor accessor(db);
gid::Generator generator(0);
@ -33,7 +35,7 @@ TEST(GraphDbAccessorTest, InsertVertex) {
}
TEST(GraphDbAccessorTest, UniqueVertexId) {
GraphDb db;
SingleNode db;
SkipList<int64_t> ids;
std::vector<std::thread> threads;
@ -50,7 +52,7 @@ TEST(GraphDbAccessorTest, UniqueVertexId) {
}
TEST(GraphDbAccessorTest, RemoveVertexSameTransaction) {
GraphDb db;
SingleNode db;
GraphDbAccessor accessor(db);
EXPECT_EQ(Count(accessor.Vertices(false)), 0);
@ -68,7 +70,7 @@ TEST(GraphDbAccessorTest, RemoveVertexSameTransaction) {
}
TEST(GraphDbAccessorTest, RemoveVertexDifferentTransaction) {
GraphDb db;
SingleNode db;
// first transaction creates a vertex
{
GraphDbAccessor accessor(db);
@ -93,7 +95,7 @@ TEST(GraphDbAccessorTest, RemoveVertexDifferentTransaction) {
}
TEST(GraphDbAccessorTest, InsertEdge) {
GraphDb db;
SingleNode db;
GraphDbAccessor dba(db);
auto va1 = dba.InsertVertex();
@ -135,7 +137,7 @@ TEST(GraphDbAccessorTest, InsertEdge) {
}
TEST(GraphDbAccessorTest, UniqueEdgeId) {
GraphDb db;
SingleNode db;
SkipList<int64_t> ids;
std::vector<std::thread> threads;
@ -156,7 +158,7 @@ TEST(GraphDbAccessorTest, UniqueEdgeId) {
}
TEST(GraphDbAccessorTest, RemoveEdge) {
GraphDb db;
SingleNode db;
GraphDbAccessor dba(db);
// setup (v1) - [:likes] -> (v2) <- [:hates] - (v3)
@ -203,7 +205,7 @@ TEST(GraphDbAccessorTest, RemoveEdge) {
}
TEST(GraphDbAccessorTest, DetachRemoveVertex) {
GraphDb db;
SingleNode db;
GraphDbAccessor dba(db);
// setup (v0)- []->(v1)<-[]-(v2)<-[]-(v3)
@ -277,7 +279,7 @@ TEST(GraphDbAccessorTest, DetachRemoveVertexMultiple) {
// This test checks that we can detach remove the
// same vertex / edge multiple times
GraphDb db;
SingleNode db;
GraphDbAccessor dba(db);
// setup: make a fully connected N graph
@ -320,10 +322,10 @@ TEST(GraphDbAccessorTest, DetachRemoveVertexMultiple) {
}
TEST(GraphDbAccessorTest, Labels) {
GraphDb db;
SingleNode db;
GraphDbAccessor dba(db);
GraphDbTypes::Label label_friend = dba.Label("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");
@ -334,10 +336,10 @@ TEST(GraphDbAccessorTest, Labels) {
}
TEST(GraphDbAccessorTest, EdgeTypes) {
GraphDb db;
SingleNode db;
GraphDbAccessor dba(db);
GraphDbTypes::EdgeType edge_type = dba.EdgeType("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");
@ -348,10 +350,10 @@ TEST(GraphDbAccessorTest, EdgeTypes) {
}
TEST(GraphDbAccessorTest, Properties) {
GraphDb db;
SingleNode db;
GraphDbAccessor dba(db);
GraphDbTypes::Property prop = dba.Property("name");
Property prop = dba.Property("name");
EXPECT_EQ(prop, dba.Property("name"));
EXPECT_NE(prop, dba.Property("surname"));
EXPECT_EQ(dba.PropertyName(prop), "name");
@ -362,7 +364,7 @@ TEST(GraphDbAccessorTest, Properties) {
}
TEST(GraphDbAccessorTest, Transfer) {
GraphDb db;
SingleNode db;
GraphDbAccessor dba1(db);
auto prop = dba1.Property("property");

View File

@ -5,6 +5,7 @@
#include <gmock/gmock.h>
#include <gtest/gtest.h>
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "utils/bound.hpp"
@ -21,11 +22,11 @@ auto Count(TIterable iterable) {
*/
class GraphDbAccessorIndex : public testing::Test {
protected:
GraphDb db;
std::experimental::optional<GraphDbAccessor> dba{db};
GraphDbTypes::Property property = dba->Property("property");
GraphDbTypes::Label label = dba->Label("label");
GraphDbTypes::EdgeType edge_type = dba->EdgeType("edge_type");
database::SingleNode db;
std::experimental::optional<database::GraphDbAccessor> dba{db};
database::Property property = dba->Property("property");
database::Label label = dba->Label("label");
database::EdgeType edge_type = dba->EdgeType("edge_type");
auto AddVertex() {
auto vertex = dba->InsertVertex();
@ -54,8 +55,9 @@ TEST_F(GraphDbAccessorIndex, LabelIndexCount) {
EXPECT_EQ(dba->VerticesCount(), 0);
for (int i = 0; i < 11; ++i) dba->InsertVertex().add_label(label);
for (int i = 0; i < 17; ++i) dba->InsertVertex().add_label(label2);
// even though xxx_count functions in GraphDbAccessor can over-estaimate
// in this situation they should be exact (nothing was ever deleted)
// even though xxx_count functions in database::GraphDbAccessor can
// over-estaimate in this situation they should be exact (nothing was ever
// deleted)
EXPECT_EQ(dba->VerticesCount(label), 11);
EXPECT_EQ(dba->VerticesCount(label2), 17);
EXPECT_EQ(dba->VerticesCount(), 28);
@ -91,8 +93,9 @@ TEST_F(GraphDbAccessorIndex, EdgesCount) {
auto v2 = AddVertex();
for (int i = 0; i < 11; ++i) dba->InsertEdge(v1, v2, edge_type);
for (int i = 0; i < 17; ++i) dba->InsertEdge(v1, v2, edge_type2);
// even though xxx_count functions in GraphDbAccessor can over-estaimate
// in this situation they should be exact (nothing was ever deleted)
// even though xxx_count functions in database::GraphDbAccessor can
// over-estaimate in this situation they should be exact (nothing was ever
// deleted)
EXPECT_EQ(dba->EdgesCount(), 28);
}
@ -134,12 +137,12 @@ TEST_F(GraphDbAccessorIndex, LabelPropertyIndexCount) {
TEST(GraphDbAccessorIndexApi, LabelPropertyBuildIndexConcurrent) {
const int ITER_COUNT = 10;
for (int iter = 0; iter < ITER_COUNT; ++iter) {
GraphDb db;
database::SingleNode db;
const int THREAD_COUNT = 10;
std::vector<std::thread> threads;
for (int index = 0; index < THREAD_COUNT; ++index) {
threads.emplace_back([&db, index]() {
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
dba.BuildIndex(dba.Label("l" + std::to_string(index)),
dba.Property("p" + std::to_string(index)));
@ -339,7 +342,7 @@ TEST_F(GraphDbAccessorIndex, LabelPropertyValueSorting) {
* (label, property) index and 100 vertices, 10 for
* each of [0, 10) property values.
*/
class GraphDbAccesssorIndexRange : public GraphDbAccessorIndex {
class GraphDbAccessorIndexRange : public GraphDbAccessorIndex {
protected:
void SetUp() override {
dba->BuildIndex(label, property);
@ -368,7 +371,7 @@ class GraphDbAccesssorIndexRange : public GraphDbAccessorIndex {
}
};
TEST_F(GraphDbAccesssorIndexRange, RangeIteration) {
TEST_F(GraphDbAccessorIndexRange, RangeIteration) {
using std::experimental::nullopt;
EXPECT_EQ(Count(Vertices(nullopt, Inclusive(7))), 80);
EXPECT_EQ(Count(Vertices(nullopt, Exclusive(7))), 70);
@ -381,7 +384,7 @@ TEST_F(GraphDbAccesssorIndexRange, RangeIteration) {
EXPECT_DEATH(Vertices(nullopt, nullopt), "bound must be provided");
}
TEST_F(GraphDbAccesssorIndexRange, RangeIterationCurrentState) {
TEST_F(GraphDbAccessorIndexRange, RangeIterationCurrentState) {
using std::experimental::nullopt;
EXPECT_EQ(Count(Vertices(nullopt, Inclusive(7))), 80);
for (int i = 0; i < 20; i++) AddVertex(2);
@ -391,7 +394,7 @@ TEST_F(GraphDbAccesssorIndexRange, RangeIterationCurrentState) {
EXPECT_EQ(Count(Vertices(nullopt, Inclusive(7))), 100);
}
TEST_F(GraphDbAccesssorIndexRange, RangeInterationIncompatibleTypes) {
TEST_F(GraphDbAccessorIndexRange, RangeInterationIncompatibleTypes) {
using std::experimental::nullopt;
// using PropertyValue::Null as a bound fails with an assertion

View File

@ -15,12 +15,12 @@
class InterpreterTest : public ::testing::Test {
protected:
query::Interpreter interpreter_;
GraphDb db_;
database::SingleNode db_;
ResultStreamFaker Interpret(
const std::string &query,
const std::map<std::string, query::TypedValue> params = {}) {
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
ResultStreamFaker result;
interpreter_(query, dba, params, false).PullAll(result);
return result;
@ -87,7 +87,7 @@ TEST_F(InterpreterTest, AstCache) {
// Run query with same ast multiple times with different parameters.
TEST_F(InterpreterTest, Parameters) {
query::Interpreter interpreter;
GraphDb db;
database::SingleNode db;
{
auto stream = Interpret("RETURN $2 + $`a b`", {{"2", 10}, {"a b", 15}});
ASSERT_EQ(stream.GetHeader().size(), 1U);
@ -146,7 +146,7 @@ TEST_F(InterpreterTest, Bfs) {
// Set up.
{
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
auto add_node = [&](int level, bool reachable) {
auto node = dba.InsertVertex();
node.PropsSet(dba.Property(kId), id++);
@ -199,7 +199,7 @@ TEST_F(InterpreterTest, Bfs) {
dba.Commit();
}
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
ResultStreamFaker stream;
interpreter_(
"MATCH (n {id: 0})-[r *bfs..5 (e, n | n.reachable and "
@ -245,7 +245,7 @@ TEST_F(InterpreterTest, Bfs) {
TEST_F(InterpreterTest, CreateIndexInMulticommandTransaction) {
ResultStreamFaker stream;
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
ASSERT_THROW(
interpreter_("CREATE INDEX ON :X(y)", dba, {}, true).PullAll(stream),
query::IndexInMulticommandTxException);

View File

@ -9,6 +9,7 @@
#include "communication/bolt/client.hpp"
#include "communication/bolt/v1/session.hpp"
#include "communication/server.hpp"
#include "database/graph_db.hpp"
#include "io/network/network_endpoint.hpp"
#include "io/network/socket.hpp"
@ -28,7 +29,8 @@ using ClientT = communication::bolt::Client<Socket>;
class RunningServer {
public:
SessionData session_data_;
database::SingleNode db_;
SessionData session_data_{db_};
NetworkEndpoint endpoint_{"127.0.0.1", "0"};
ServerT server_{endpoint_, session_data_, 1};
};

View File

@ -12,12 +12,12 @@ class PropertyValueStoreTest : public ::testing::Test {
PropertyValueStore props_;
void Set(int key, PropertyValue value) {
props_.set(GraphDbTypes::Property(key), value);
props_.set(database::Property(key), value);
}
PropertyValue At(int key) { return props_.at(GraphDbTypes::Property(key)); }
PropertyValue At(int key) { return props_.at(database::Property(key)); }
auto Erase(int key) { return props_.erase(GraphDbTypes::Property(key)); }
auto Erase(int key) { return props_.erase(database::Property(key)); }
};
TEST_F(PropertyValueStoreTest, At) {

View File

@ -5,8 +5,8 @@
/// example:
///
/// AstTreeStorage storage; // Macros rely on storage being in scope.
/// // PROPERTY_LOOKUP and PROPERTY_PAIR macros also rely on GraphDb
/// GraphDb db;
/// // PROPERTY_LOOKUP and PROPERTY_PAIR macros rely on database::SingleNode
/// database::SingleNode db;
///
/// QUERY(MATCH(PATTERN(NODE("n"), EDGE("e"), NODE("m"))),
/// WHERE(LESS(PROPERTY_LOOKUP("e", edge_prop), LITERAL(3))),
@ -28,7 +28,9 @@
#include <utility>
#include <vector>
#include "database/graph_db_datatypes.hpp"
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "database/types.hpp"
#include "query/frontend/ast/ast.hpp"
#include "query/interpret/awesome_memgraph_functions.hpp"
#include "utils/string.hpp"
@ -107,28 +109,27 @@ auto GetOrderBy(T... exprs) {
///
/// Name is used to create the Identifier which is used for property lookup.
///
auto GetPropertyLookup(AstTreeStorage &storage, GraphDb &db,
const std::string &name,
GraphDbTypes::Property property) {
GraphDbAccessor dba(db);
auto GetPropertyLookup(AstTreeStorage &storage, database::GraphDb &db,
const std::string &name, database::Property property) {
database::GraphDbAccessor dba(db);
return storage.Create<PropertyLookup>(storage.Create<Identifier>(name),
dba.PropertyName(property), property);
}
auto GetPropertyLookup(AstTreeStorage &storage, GraphDb &db, Expression *expr,
GraphDbTypes::Property property) {
GraphDbAccessor dba(db);
auto GetPropertyLookup(AstTreeStorage &storage, database::GraphDb &db,
Expression *expr, database::Property property) {
database::GraphDbAccessor dba(db);
return storage.Create<PropertyLookup>(expr, dba.PropertyName(property),
property);
}
auto GetPropertyLookup(
AstTreeStorage &storage, GraphDb &, const std::string &name,
const std::pair<std::string, GraphDbTypes::Property> &prop_pair) {
AstTreeStorage &storage, database::GraphDb &, const std::string &name,
const std::pair<std::string, database::Property> &prop_pair) {
return storage.Create<PropertyLookup>(storage.Create<Identifier>(name),
prop_pair.first, prop_pair.second);
}
auto GetPropertyLookup(
AstTreeStorage &storage, GraphDb &, Expression *expr,
const std::pair<std::string, GraphDbTypes::Property> &prop_pair) {
AstTreeStorage &storage, database::GraphDb &, Expression *expr,
const std::pair<std::string, database::Property> &prop_pair) {
return storage.Create<PropertyLookup>(expr, prop_pair.first,
prop_pair.second);
}
@ -140,7 +141,7 @@ auto GetPropertyLookup(
///
auto GetEdge(AstTreeStorage &storage, const std::string &name,
EdgeAtom::Direction dir = EdgeAtom::Direction::BOTH,
const std::vector<GraphDbTypes::EdgeType> &edge_types = {}) {
const std::vector<database::EdgeType> &edge_types = {}) {
return storage.Create<EdgeAtom>(storage.Create<Identifier>(name),
EdgeAtom::Type::SINGLE, dir, edge_types);
}
@ -153,16 +154,18 @@ auto GetEdge(AstTreeStorage &storage, const std::string &name,
///
auto GetEdgeVariable(AstTreeStorage &storage, const std::string &name,
EdgeAtom::Direction dir = EdgeAtom::Direction::BOTH,
const std::vector<GraphDbTypes::EdgeType> &edge_types = {},
const std::vector<database::EdgeType> &edge_types = {},
Identifier *inner_edge = nullptr,
Identifier *inner_node = nullptr) {
auto r_val =
storage.Create<EdgeAtom>(storage.Create<Identifier>(name),
EdgeAtom::Type::DEPTH_FIRST, dir, edge_types);
r_val->inner_edge_ = inner_edge ? inner_edge : storage.Create<Identifier>(
utils::RandomString(20));
r_val->inner_node_ = inner_node ? inner_node : storage.Create<Identifier>(
utils::RandomString(20));
r_val->inner_edge_ =
inner_edge ? inner_edge
: storage.Create<Identifier>(utils::RandomString(20));
r_val->inner_node_ =
inner_node ? inner_node
: storage.Create<Identifier>(utils::RandomString(20));
return r_val;
}
@ -172,7 +175,7 @@ auto GetEdgeVariable(AstTreeStorage &storage, const std::string &name,
/// Name is used to create the Identifier which is assigned to the node.
///
auto GetNode(AstTreeStorage &storage, const std::string &name,
std::experimental::optional<GraphDbTypes::Label> label =
std::experimental::optional<database::Label> label =
std::experimental::nullopt) {
auto node = storage.Create<NodeAtom>(storage.Create<Identifier>(name));
if (label) node->labels_.emplace_back(*label);
@ -430,7 +433,7 @@ auto GetSet(AstTreeStorage &storage, const std::string &name, Expression *expr,
/// Create a set labels clause for given identifier name and labels.
///
auto GetSet(AstTreeStorage &storage, const std::string &name,
std::vector<GraphDbTypes::Label> labels) {
std::vector<database::Label> labels) {
return storage.Create<SetLabels>(storage.Create<Identifier>(name), labels);
}
@ -445,7 +448,7 @@ auto GetRemove(AstTreeStorage &storage, PropertyLookup *prop_lookup) {
/// Create a remove labels clause for given identifier name and labels.
///
auto GetRemove(AstTreeStorage &storage, const std::string &name,
std::vector<GraphDbTypes::Label> labels) {
std::vector<database::Label> labels) {
return storage.Create<RemoveLabels>(storage.Create<Identifier>(name), labels);
}
@ -507,12 +510,13 @@ auto GetMerge(AstTreeStorage &storage, Pattern *pattern, OnMatch on_match,
#define LIST(...) \
storage.Create<query::ListLiteral>( \
std::vector<query::Expression *>{__VA_ARGS__})
#define MAP(...) \
storage.Create<query::MapLiteral>( \
std::unordered_map<std::pair<std::string, GraphDbTypes::Property>, \
#define MAP(...) \
storage.Create<query::MapLiteral>( \
std::unordered_map<std::pair<std::string, database::Property>, \
query::Expression *>{__VA_ARGS__})
#define PROPERTY_PAIR(property_name) \
std::make_pair(property_name, GraphDbAccessor(db).Property(property_name))
std::make_pair(property_name, \
database::GraphDbAccessor(db).Property(property_name))
#define PROPERTY_LOOKUP(...) \
query::test_common::GetPropertyLookup(storage, db, __VA_ARGS__)
#define NEXPR(name, expr) storage.Create<query::NamedExpression>((name), (expr))

View File

@ -10,9 +10,9 @@
using namespace query;
using namespace query::plan;
using CardParam = CostEstimator<GraphDbAccessor>::CardParam;
using CostParam = CostEstimator<GraphDbAccessor>::CostParam;
using MiscParam = CostEstimator<GraphDbAccessor>::MiscParam;
using CardParam = CostEstimator<database::GraphDbAccessor>::CardParam;
using CostParam = CostEstimator<database::GraphDbAccessor>::CostParam;
using MiscParam = CostEstimator<database::GraphDbAccessor>::MiscParam;
/** A fixture for cost estimation. Sets up the database
* and accessor (adds some vertices). Provides convenience
@ -21,10 +21,10 @@ using MiscParam = CostEstimator<GraphDbAccessor>::MiscParam;
* estimation testing. */
class QueryCostEstimator : public ::testing::Test {
protected:
GraphDb db;
std::experimental::optional<GraphDbAccessor> dba{db};
GraphDbTypes::Label label = dba->Label("label");
GraphDbTypes::Property property = dba->Property("property");
database::SingleNode db;
std::experimental::optional<database::GraphDbAccessor> dba{db};
database::Label label = dba->Label("label");
database::Property property = dba->Property("property");
// we incrementally build the logical operator plan
// start it off with Once
@ -60,7 +60,7 @@ class QueryCostEstimator : public ::testing::Test {
}
auto Cost() {
CostEstimator<GraphDbAccessor> cost_estimator(*dba, parameters_);
CostEstimator<database::GraphDbAccessor> cost_estimator(*dba, parameters_);
last_op_->Accept(cost_estimator);
return cost_estimator.cost();
}
@ -162,7 +162,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeConstExpr) {
TEST_F(QueryCostEstimator, Expand) {
MakeOp<Expand>(NextSymbol(), NextSymbol(), EdgeAtom::Direction::IN,
std::vector<GraphDbTypes::EdgeType>{}, last_op_, NextSymbol(),
std::vector<database::EdgeType>{}, last_op_, NextSymbol(),
false);
EXPECT_COST(CardParam::kExpand * CostParam::kExpand);
}
@ -170,7 +170,7 @@ TEST_F(QueryCostEstimator, Expand) {
TEST_F(QueryCostEstimator, ExpandVariable) {
MakeOp<ExpandVariable>(NextSymbol(), NextSymbol(),
EdgeAtom::Type::DEPTH_FIRST, EdgeAtom::Direction::IN,
std::vector<GraphDbTypes::EdgeType>{}, false, nullptr,
std::vector<database::EdgeType>{}, false, nullptr,
nullptr, last_op_, NextSymbol(), false, NextSymbol(),
NextSymbol(), nullptr);
EXPECT_COST(CardParam::kExpandVariable * CostParam::kExpandVariable);
@ -198,12 +198,11 @@ TEST_F(QueryCostEstimator, ExpandUniquenessFilter) {
}
TEST_F(QueryCostEstimator, UnwindLiteral) {
TEST_OP(
MakeOp<query::plan::Unwind>(
last_op_,
storage_.Create<ListLiteral>(std::vector<Expression *>(7, nullptr)),
NextSymbol()),
CostParam::kUnwind, 7);
TEST_OP(MakeOp<query::plan::Unwind>(
last_op_, storage_.Create<ListLiteral>(
std::vector<Expression *>(7, nullptr)),
NextSymbol()),
CostParam::kUnwind, 7);
}
TEST_F(QueryCostEstimator, UnwindNoLiteral) {

View File

@ -8,7 +8,7 @@
#include "gtest/gtest.h"
#include "database/graph_db_accessor.hpp"
#include "database/graph_db_datatypes.hpp"
#include "database/types.hpp"
#include "query/frontend/ast/ast.hpp"
#include "query/frontend/opencypher/parser.hpp"
#include "query/interpret/awesome_memgraph_functions.hpp"
@ -30,17 +30,18 @@ struct NoContextExpressionEvaluator {
NoContextExpressionEvaluator() {}
Frame frame{128};
SymbolTable symbol_table;
GraphDb db;
GraphDbAccessor dba{db};
database::SingleNode db;
database::GraphDbAccessor dba{db};
Parameters parameters;
ExpressionEvaluator eval{frame, parameters, symbol_table, dba};
};
TypedValue EvaluateFunction(const std::string &function_name,
const std::vector<TypedValue> &args, GraphDb &db) {
const std::vector<TypedValue> &args,
database::GraphDb &db) {
AstTreeStorage storage;
SymbolTable symbol_table;
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
Frame frame{128};
Parameters parameters;
ExpressionEvaluator eval{frame, parameters, symbol_table, dba};
@ -55,7 +56,7 @@ TypedValue EvaluateFunction(const std::string &function_name,
TypedValue EvaluateFunction(const std::string &function_name,
const std::vector<TypedValue> &args) {
GraphDb db;
database::SingleNode db;
return EvaluateFunction(function_name, args, db);
}
@ -415,10 +416,10 @@ TEST(ExpressionEvaluator, ListMapIndexingOperator) {
TEST(ExpressionEvaluator, MapIndexing) {
AstTreeStorage storage;
NoContextExpressionEvaluator eval;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto *map_literal = storage.Create<MapLiteral>(
std::unordered_map<std::pair<std::string, GraphDbTypes::Property>,
std::unordered_map<std::pair<std::string, database::Property>,
Expression *>{
{PROPERTY_PAIR("a"), storage.Create<PrimitiveLiteral>(1)},
{PROPERTY_PAIR("b"), storage.Create<PrimitiveLiteral>(2)},
@ -622,18 +623,17 @@ class ExpressionEvaluatorPropertyLookup : public testing::Test {
protected:
AstTreeStorage storage;
NoContextExpressionEvaluator eval;
GraphDb db;
GraphDbAccessor dba{db};
std::pair<std::string, GraphDbTypes::Property> prop_age =
PROPERTY_PAIR("age");
std::pair<std::string, GraphDbTypes::Property> prop_height =
database::SingleNode db;
database::GraphDbAccessor dba{db};
std::pair<std::string, database::Property> prop_age = PROPERTY_PAIR("age");
std::pair<std::string, database::Property> prop_height =
PROPERTY_PAIR("height");
Expression *identifier = storage.Create<Identifier>("element");
Symbol symbol = eval.symbol_table.CreateSymbol("element", true);
void SetUp() { eval.symbol_table[*identifier] = symbol; }
auto Value(std::pair<std::string, GraphDbTypes::Property> property) {
auto Value(std::pair<std::string, database::Property> property) {
auto *op = storage.Create<PropertyLookup>(identifier, property);
return op->Accept(eval.eval);
}
@ -671,8 +671,8 @@ TEST_F(ExpressionEvaluatorPropertyLookup, MapLiteral) {
TEST(ExpressionEvaluator, LabelsTest) {
AstTreeStorage storage;
NoContextExpressionEvaluator eval;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
v1.add_label(dba.Label("ANIMAL"));
v1.add_label(dba.Label("DOG"));
@ -683,16 +683,16 @@ TEST(ExpressionEvaluator, LabelsTest) {
eval.frame[node_symbol] = v1;
{
auto *op = storage.Create<LabelsTest>(
identifier, std::vector<GraphDbTypes::Label>{dba.Label("DOG"),
dba.Label("ANIMAL")});
identifier,
std::vector<database::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<GraphDbTypes::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"),
dba.Label("ANIMAL")});
std::vector<database::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"),
dba.Label("ANIMAL")});
auto value = op->Accept(eval.eval);
EXPECT_EQ(value.Value<bool>(), false);
}
@ -700,8 +700,8 @@ TEST(ExpressionEvaluator, LabelsTest) {
eval.frame[node_symbol] = TypedValue::Null;
auto *op = storage.Create<LabelsTest>(
identifier,
std::vector<GraphDbTypes::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"),
dba.Label("ANIMAL")});
std::vector<database::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"),
dba.Label("ANIMAL")});
auto value = op->Accept(eval.eval);
EXPECT_TRUE(value.IsNull());
}
@ -716,8 +716,8 @@ TEST(ExpressionEvaluator, Aggregation) {
symbol_table[*aggr] = aggr_sym;
Frame frame{symbol_table.max_position()};
frame[aggr_sym] = TypedValue(1);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
Parameters parameters;
ExpressionEvaluator eval{frame, parameters, symbol_table, dba};
auto value = aggr->Accept(eval);
@ -754,8 +754,8 @@ TEST(ExpressionEvaluator, FunctionEndNode) {
ASSERT_THROW(EvaluateFunction("ENDNODE", {}), QueryRuntimeException);
ASSERT_EQ(EvaluateFunction("ENDNODE", {TypedValue::Null}).type(),
TypedValue::Type::Null);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
v1.add_label(dba.Label("label1"));
auto v2 = dba.InsertVertex();
@ -783,8 +783,8 @@ TEST(ExpressionEvaluator, FunctionProperties) {
ASSERT_THROW(EvaluateFunction("PROPERTIES", {}), QueryRuntimeException);
ASSERT_EQ(EvaluateFunction("PROPERTIES", {TypedValue::Null}).type(),
TypedValue::Type::Null);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
v1.PropsSet(dba.Property("height"), 5);
v1.PropsSet(dba.Property("age"), 10);
@ -836,8 +836,8 @@ TEST(ExpressionEvaluator, FunctionSize) {
3);
ASSERT_THROW(EvaluateFunction("SIZE", {5}), QueryRuntimeException);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v0 = dba.InsertVertex();
query::Path path(v0);
EXPECT_EQ(EvaluateFunction("SIZE", {path}).ValueInt(), 0);
@ -851,8 +851,8 @@ TEST(ExpressionEvaluator, FunctionStartNode) {
ASSERT_THROW(EvaluateFunction("STARTNODE", {}), QueryRuntimeException);
ASSERT_EQ(EvaluateFunction("STARTNODE", {TypedValue::Null}).type(),
TypedValue::Type::Null);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
v1.add_label(dba.Label("label1"));
auto v2 = dba.InsertVertex();
@ -868,8 +868,8 @@ TEST(ExpressionEvaluator, FunctionDegree) {
ASSERT_THROW(EvaluateFunction("DEGREE", {}), QueryRuntimeException);
ASSERT_EQ(EvaluateFunction("DEGREE", {TypedValue::Null}).type(),
TypedValue::Type::Null);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
auto v3 = dba.InsertVertex();
@ -928,8 +928,8 @@ TEST(ExpressionEvaluator, FunctionType) {
ASSERT_THROW(EvaluateFunction("TYPE", {}), QueryRuntimeException);
ASSERT_EQ(EvaluateFunction("TYPE", {TypedValue::Null}).type(),
TypedValue::Type::Null);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
v1.add_label(dba.Label("label1"));
auto v2 = dba.InsertVertex();
@ -943,8 +943,8 @@ TEST(ExpressionEvaluator, FunctionLabels) {
ASSERT_THROW(EvaluateFunction("LABELS", {}), QueryRuntimeException);
ASSERT_EQ(EvaluateFunction("LABELS", {TypedValue::Null}).type(),
TypedValue::Type::Null);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v = dba.InsertVertex();
v.add_label(dba.Label("label1"));
v.add_label(dba.Label("label2"));
@ -965,8 +965,8 @@ TEST(ExpressionEvaluator, FunctionNodesRelationships) {
EXPECT_TRUE(EvaluateFunction("RELATIONSHIPS", {TypedValue::Null}).IsNull());
{
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
auto v3 = dba.InsertVertex();
@ -1023,8 +1023,8 @@ TEST(ExpressionEvaluator, FunctionKeys) {
ASSERT_THROW(EvaluateFunction("KEYS", {}), QueryRuntimeException);
ASSERT_EQ(EvaluateFunction("KEYS", {TypedValue::Null}).type(),
TypedValue::Type::Null);
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
v1.PropsSet(dba.Property("height"), 5);
v1.PropsSet(dba.Property("age"), 10);
@ -1257,7 +1257,7 @@ TEST(ExpressionEvaluator, ParameterLookup) {
}
TEST(ExpressionEvaluator, FunctionCounter) {
GraphDb db;
database::SingleNode db;
EXPECT_THROW(EvaluateFunction("COUNTER", {}, db), QueryRuntimeException);
EXPECT_THROW(EvaluateFunction("COUNTER", {"a", "b"}, db),
QueryRuntimeException);
@ -1269,7 +1269,7 @@ TEST(ExpressionEvaluator, FunctionCounter) {
}
TEST(ExpressionEvaluator, FunctionCounterSet) {
GraphDb db;
database::SingleNode db;
EXPECT_THROW(EvaluateFunction("COUNTERSET", {}, db), QueryRuntimeException);
EXPECT_THROW(EvaluateFunction("COUNTERSET", {"a"}, db),
QueryRuntimeException);
@ -1287,10 +1287,10 @@ TEST(ExpressionEvaluator, FunctionCounterSet) {
}
TEST(ExpressionEvaluator, FunctionIndexInfo) {
GraphDb db;
database::SingleNode db;
EXPECT_THROW(EvaluateFunction("INDEXINFO", {1}, db), QueryRuntimeException);
EXPECT_EQ(EvaluateFunction("INDEXINFO", {}, db).ValueList().size(), 0);
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
dba.InsertVertex().add_label(dba.Label("l1"));
{
auto info = ToList<std::string>(EvaluateFunction("INDEXINFO", {}, db));

View File

@ -1,8 +1,3 @@
//
// Copyright 2017 Memgraph
// Created by Florijan Stamenkovic on 14.03.17.
//
#include <algorithm>
#include <iterator>
#include <memory>
@ -12,10 +7,10 @@
#include "gtest/gtest.h"
#include "communication/result_stream_faker.hpp"
#include "database/graph_db.hpp"
#include "query/context.hpp"
#include "query/exceptions.hpp"
#include "query/plan/operator.hpp"
#include "query_plan_common.hpp"
using namespace query;
@ -32,8 +27,8 @@ TEST(QueryPlan, Accumulate) {
// with accumulation we expect them to be [[2, 2], [2, 2]]
auto check = [&](bool accumulate) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("x");
auto v1 = dba.InsertVertex();
@ -91,8 +86,8 @@ TEST(QueryPlan, AccumulateAdvance) {
// to get correct results we need to advance the command
auto check = [&](bool advance) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -153,9 +148,9 @@ std::shared_ptr<Produce> MakeAggregationProduce(
/** Test fixture for all the aggregation ops in one return. */
class QueryPlanAggregateOps : public ::testing::Test {
protected:
GraphDb db;
GraphDbAccessor dba{db};
GraphDbTypes::Property prop = dba.Property("prop");
database::SingleNode db;
database::GraphDbAccessor dba{db};
database::Property prop = dba.Property("prop");
AstTreeStorage storage;
SymbolTable symbol_table;
@ -292,8 +287,8 @@ TEST(QueryPlan, AggregateGroupByValues) {
// Tests that distinct groups are aggregated properly for values of all types.
// Also test the "remember" part of the Aggregation API as final results are
// obtained via a property lookup of a remembered node.
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// a vector of TypedValue to be set as property values on vertices
// most of them should result in a distinct group (commented where not)
@ -351,8 +346,8 @@ TEST(QueryPlan, AggregateMultipleGroupBy) {
// in this test we have 3 different properties that have different values
// for different records and assert that we get the correct combination
// of values in our groups
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop1 = dba.Property("prop1");
auto prop2 = dba.Property("prop2");
@ -386,8 +381,8 @@ TEST(QueryPlan, AggregateMultipleGroupBy) {
}
TEST(QueryPlan, AggregateNoInput) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -413,8 +408,8 @@ TEST(QueryPlan, AggregateCountEdgeCases) {
// - 2 vertices in database, property set on one
// - 2 vertices in database, property set on both
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
@ -465,8 +460,8 @@ TEST(QueryPlan, AggregateFirstValueTypes) {
// testing exceptions that get emitted by the first-value
// type check
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto prop_string = dba.Property("string");
@ -522,8 +517,8 @@ TEST(QueryPlan, AggregateTypes) {
// does not check all combinations that can result in an exception
// (that logic is defined and tested by TypedValue)
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto p1 = dba.Property("p1"); // has only string props
dba.InsertVertex().PropsSet(p1, "string");
@ -578,8 +573,8 @@ TEST(QueryPlan, AggregateTypes) {
}
TEST(QueryPlan, Unwind) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;

View File

@ -22,8 +22,8 @@ using namespace query;
using namespace query::plan;
TEST(QueryPlan, Skip) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -51,8 +51,8 @@ TEST(QueryPlan, Skip) {
}
TEST(QueryPlan, Limit) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -83,8 +83,8 @@ TEST(QueryPlan, CreateLimit) {
// CREATE (n), (m)
// MATCH (n) CREATE (m) LIMIT 1
// in the end we need to have 3 vertices in the db
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
dba.InsertVertex();
dba.InsertVertex();
dba.AdvanceCommand();
@ -104,8 +104,8 @@ TEST(QueryPlan, CreateLimit) {
}
TEST(QueryPlan, OrderBy) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
auto prop = dba.Property("prop");
@ -142,8 +142,7 @@ TEST(QueryPlan, OrderBy) {
ASSERT_FALSE(order_equal());
// create the vertices
for (const auto &value : shuffled)
dba.InsertVertex().PropsSet(prop, value);
for (const auto &value : shuffled) dba.InsertVertex().PropsSet(prop, value);
dba.AdvanceCommand();
// order by and collect results
@ -166,8 +165,8 @@ TEST(QueryPlan, OrderBy) {
}
TEST(QueryPlan, OrderByMultiple) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -222,8 +221,8 @@ TEST(QueryPlan, OrderByMultiple) {
}
TEST(QueryPlan, OrderByExceptions) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
auto prop = dba.Property("prop");

View File

@ -33,7 +33,8 @@ using Bound = ScanAllByLabelPropertyRange::Bound;
* @return
*/
std::vector<std::vector<TypedValue>> CollectProduce(
Produce *produce, SymbolTable &symbol_table, GraphDbAccessor &db_accessor) {
Produce *produce, SymbolTable &symbol_table,
database::GraphDbAccessor &db_accessor) {
ResultStreamFaker stream;
Frame frame(symbol_table.max_position());
@ -66,8 +67,8 @@ std::vector<std::vector<TypedValue>> CollectProduce(
return stream.GetResults();
}
int PullAll(std::shared_ptr<LogicalOperator> logical_op, GraphDbAccessor &db,
SymbolTable &symbol_table) {
int PullAll(std::shared_ptr<LogicalOperator> logical_op,
database::GraphDbAccessor &db, SymbolTable &symbol_table) {
Frame frame(symbol_table.max_position());
auto cursor = logical_op->MakeCursor(db);
int count = 0;
@ -115,7 +116,7 @@ ScanAllTuple MakeScanAll(AstTreeStorage &storage, SymbolTable &symbol_table,
*/
ScanAllTuple MakeScanAllByLabel(
AstTreeStorage &storage, SymbolTable &symbol_table,
const std::string &identifier, const GraphDbTypes::Label &label,
const std::string &identifier, const database::Label &label,
std::shared_ptr<LogicalOperator> input = {nullptr},
GraphView graph_view = GraphView::OLD) {
auto node = NODE(identifier);
@ -134,7 +135,7 @@ ScanAllTuple MakeScanAllByLabel(
*/
ScanAllTuple MakeScanAllByLabelPropertyRange(
AstTreeStorage &storage, SymbolTable &symbol_table, std::string identifier,
GraphDbTypes::Label label, GraphDbTypes::Property property,
database::Label label, database::Property property,
std::experimental::optional<Bound> lower_bound,
std::experimental::optional<Bound> upper_bound,
std::shared_ptr<LogicalOperator> input = {nullptr},
@ -155,8 +156,8 @@ ScanAllTuple MakeScanAllByLabelPropertyRange(
*/
ScanAllTuple MakeScanAllByLabelPropertyValue(
AstTreeStorage &storage, SymbolTable &symbol_table, std::string identifier,
GraphDbTypes::Label label, GraphDbTypes::Property property,
Expression *value, std::shared_ptr<LogicalOperator> input = {nullptr},
database::Label label, database::Property property, Expression *value,
std::shared_ptr<LogicalOperator> input = {nullptr},
GraphView graph_view = GraphView::OLD) {
auto node = NODE(identifier);
auto symbol = symbol_table.CreateSymbol(identifier, true);
@ -178,7 +179,7 @@ ExpandTuple MakeExpand(AstTreeStorage &storage, SymbolTable &symbol_table,
std::shared_ptr<LogicalOperator> input,
Symbol input_symbol, const std::string &edge_identifier,
EdgeAtom::Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types,
const std::vector<database::EdgeType> &edge_types,
const std::string &node_identifier, bool existing_node,
GraphView graph_view = GraphView::AS_IS) {
auto edge = EDGE(edge_identifier, direction);

View File

@ -22,10 +22,10 @@ using namespace query;
using namespace query::plan;
TEST(QueryPlan, CreateNodeWithAttributes) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
GraphDbTypes::Label label = dba.Label("Person");
database::Label label = dba.Label("Person");
auto property = PROPERTY_PAIR("prop");
AstTreeStorage storage;
@ -56,10 +56,10 @@ TEST(QueryPlan, CreateNodeWithAttributes) {
TEST(QueryPlan, CreateReturn) {
// test CREATE (n:Person {age: 42}) RETURN n, n.age
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
GraphDbTypes::Label label = dba.Label("Person");
database::Label label = dba.Label("Person");
auto property = PROPERTY_PAIR("property");
AstTreeStorage storage;
@ -97,13 +97,13 @@ TEST(QueryPlan, CreateReturn) {
}
TEST(QueryPlan, CreateExpand) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
GraphDbTypes::Label label_node_1 = dba.Label("Node1");
GraphDbTypes::Label label_node_2 = dba.Label("Node2");
database::Label label_node_1 = dba.Label("Node1");
database::Label label_node_2 = dba.Label("Node2");
auto property = PROPERTY_PAIR("property");
GraphDbTypes::EdgeType edge_type = dba.EdgeType("edge_type");
database::EdgeType edge_type = dba.EdgeType("edge_type");
SymbolTable symbol_table;
AstTreeStorage storage;
@ -151,7 +151,7 @@ TEST(QueryPlan, CreateExpand) {
for (VertexAccessor vertex : dba.Vertices(false)) {
EXPECT_EQ(vertex.labels().size(), 1);
GraphDbTypes::Label label = vertex.labels()[0];
database::Label label = vertex.labels()[0];
if (label == label_node_1) {
// node created by first op
EXPECT_EQ(vertex.PropsAt(property.second).Value<int64_t>(), 1);
@ -171,8 +171,8 @@ TEST(QueryPlan, CreateExpand) {
}
TEST(QueryPlan, MatchCreateNode) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// add three nodes we'll match and expand-create from
dba.InsertVertex();
@ -198,8 +198,8 @@ TEST(QueryPlan, MatchCreateNode) {
}
TEST(QueryPlan, MatchCreateExpand) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// add three nodes we'll match and expand-create from
dba.InsertVertex();
@ -207,10 +207,10 @@ TEST(QueryPlan, MatchCreateExpand) {
dba.InsertVertex();
dba.AdvanceCommand();
// GraphDbTypes::Label label_node_1 = dba.Label("Node1");
// GraphDbTypes::Label label_node_2 = dba.Label("Node2");
// GraphDbTypes::Property property = dba.Label("prop");
GraphDbTypes::EdgeType edge_type = dba.EdgeType("edge_type");
// database::Label label_node_1 = dba.Label("Node1");
// database::Label label_node_2 = dba.Label("Node2");
// database::Property property = dba.Label("prop");
database::EdgeType edge_type = dba.EdgeType("edge_type");
SymbolTable symbol_table;
AstTreeStorage storage;
@ -250,8 +250,8 @@ TEST(QueryPlan, MatchCreateExpand) {
}
TEST(QueryPlan, Delete) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// make a fully-connected (one-direction, no cycles) with 4 nodes
std::vector<VertexAccessor> vertices;
@ -339,8 +339,8 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
// MATCH (n)-[r]-(m) [DETACH] DELETE n, r, m
auto test_delete = [](bool detach) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -377,8 +377,8 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
}
TEST(QueryPlan, DeleteReturn) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// make a fully-connected (one-direction, no cycles) with 4 nodes
auto prop = PROPERTY_PAIR("property");
@ -416,8 +416,8 @@ TEST(QueryPlan, DeleteReturn) {
TEST(QueryPlan, DeleteNull) {
// test (simplified) WITH Null as x delete x
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -437,8 +437,8 @@ TEST(QueryPlan, DeleteAdvance) {
// note that Neo does not fail when the deleted
// record is not used in subsequent clauses, but
// we are not yet compatible with that
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
dba.InsertVertex();
dba.AdvanceCommand();
@ -456,8 +456,8 @@ TEST(QueryPlan, DeleteAdvance) {
}
TEST(QueryPlan, SetProperty) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// graph with 4 vertices in connected pairs
// the origin vertex in each par and both edges
@ -507,8 +507,8 @@ TEST(QueryPlan, SetProperty) {
TEST(QueryPlan, SetProperties) {
auto test_set_properties = [](bool update) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// graph: ({a: 0})-[:R {b:1}]->({c:2})
auto prop_a = dba.Property("a");
@ -576,8 +576,8 @@ TEST(QueryPlan, SetProperties) {
}
TEST(QueryPlan, SetLabels) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label1 = dba.Label("label1");
auto label2 = dba.Label("label2");
@ -591,7 +591,7 @@ TEST(QueryPlan, SetLabels) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto label_set = std::make_shared<plan::SetLabels>(
n.op_, n.sym_, std::vector<GraphDbTypes::Label>{label2, label3});
n.op_, n.sym_, std::vector<database::Label>{label2, label3});
EXPECT_EQ(2, PullAll(label_set, dba, symbol_table));
for (VertexAccessor vertex : dba.Vertices(false)) {
@ -603,8 +603,8 @@ TEST(QueryPlan, SetLabels) {
}
TEST(QueryPlan, RemoveProperty) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// graph with 4 vertices in connected pairs
// the origin vertex in each par and both edges
@ -655,8 +655,8 @@ TEST(QueryPlan, RemoveProperty) {
}
TEST(QueryPlan, RemoveLabels) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label1 = dba.Label("label1");
auto label2 = dba.Label("label2");
@ -675,7 +675,7 @@ TEST(QueryPlan, RemoveLabels) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto label_remove = std::make_shared<plan::RemoveLabels>(
n.op_, n.sym_, std::vector<GraphDbTypes::Label>{label1, label2});
n.op_, n.sym_, std::vector<database::Label>{label1, label2});
EXPECT_EQ(2, PullAll(label_remove, dba, symbol_table));
for (VertexAccessor vertex : dba.Vertices(false)) {
@ -687,8 +687,8 @@ TEST(QueryPlan, RemoveLabels) {
}
TEST(QueryPlan, NodeFilterSet) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Create a graph such that (v1 {prop: 42}) is connected to v2 and v3.
auto v1 = dba.InsertVertex();
auto prop = PROPERTY_PAIR("property");
@ -727,8 +727,8 @@ TEST(QueryPlan, NodeFilterSet) {
}
TEST(QueryPlan, FilterRemove) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Create a graph such that (v1 {prop: 42}) is connected to v2 and v3.
auto v1 = dba.InsertVertex();
auto prop = PROPERTY_PAIR("property");
@ -763,8 +763,8 @@ TEST(QueryPlan, FilterRemove) {
}
TEST(QueryPlan, SetRemove) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v = dba.InsertVertex();
auto label1 = dba.Label("label1");
auto label2 = dba.Label("label2");
@ -777,9 +777,9 @@ TEST(QueryPlan, SetRemove) {
auto scan_all = MakeScanAll(storage, symbol_table, "n");
auto set = std::make_shared<plan::SetLabels>(
scan_all.op_, scan_all.sym_,
std::vector<GraphDbTypes::Label>{label1, label2});
std::vector<database::Label>{label1, label2});
auto rem = std::make_shared<plan::RemoveLabels>(
set, scan_all.sym_, std::vector<GraphDbTypes::Label>{label1, label2});
set, scan_all.sym_, std::vector<database::Label>{label1, label2});
EXPECT_EQ(1, PullAll(rem, dba, symbol_table));
dba.AdvanceCommand();
v.Reconstruct();
@ -794,8 +794,8 @@ TEST(QueryPlan, Merge) {
// - merge_match branch looks for an expansion (any direction)
// and sets some property (for result validation)
// - merge_create branch just sets some other property
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
dba.InsertEdge(v1, v2, dba.EdgeType("Type"));
@ -839,8 +839,8 @@ TEST(QueryPlan, Merge) {
TEST(QueryPlan, MergeNoInput) {
// merge with no input, creates a single node
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -858,8 +858,8 @@ TEST(QueryPlan, MergeNoInput) {
TEST(QueryPlan, SetPropertyOnNull) {
// SET (Null).prop = 42
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
auto prop = PROPERTY_PAIR("property");
@ -873,8 +873,8 @@ TEST(QueryPlan, SetPropertyOnNull) {
TEST(QueryPlan, SetPropertiesOnNull) {
// OPTIONAL MATCH (n) SET n = n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
auto n = MakeScanAll(storage, symbol_table, "n");
@ -890,8 +890,8 @@ TEST(QueryPlan, SetPropertiesOnNull) {
TEST(QueryPlan, SetLabelsOnNull) {
// OPTIONAL MATCH (n) SET n :label
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
AstTreeStorage storage;
SymbolTable symbol_table;
@ -901,15 +901,15 @@ TEST(QueryPlan, SetLabelsOnNull) {
auto optional = std::make_shared<plan::Optional>(nullptr, n.op_,
std::vector<Symbol>{n.sym_});
auto set_op = std::make_shared<plan::SetLabels>(
optional, n.sym_, std::vector<GraphDbTypes::Label>{label});
optional, n.sym_, std::vector<database::Label>{label});
EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
EXPECT_EQ(1, PullAll(set_op, dba, symbol_table));
}
TEST(QueryPlan, RemovePropertyOnNull) {
// REMOVE (Null).prop
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
auto prop = PROPERTY_PAIR("property");
@ -922,8 +922,8 @@ TEST(QueryPlan, RemovePropertyOnNull) {
TEST(QueryPlan, RemoveLabelsOnNull) {
// OPTIONAL MATCH (n) REMOVE n :label
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
AstTreeStorage storage;
SymbolTable symbol_table;
@ -933,15 +933,15 @@ TEST(QueryPlan, RemoveLabelsOnNull) {
auto optional = std::make_shared<plan::Optional>(nullptr, n.op_,
std::vector<Symbol>{n.sym_});
auto remove_op = std::make_shared<plan::RemoveLabels>(
optional, n.sym_, std::vector<GraphDbTypes::Label>{label});
optional, n.sym_, std::vector<database::Label>{label});
EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
EXPECT_EQ(1, PullAll(remove_op, dba, symbol_table));
}
TEST(QueryPlan, CreateIndex) {
// CREATE INDEX ON :Label(property)
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto property = dba.Property("property");
EXPECT_FALSE(dba.LabelPropertyIndexExists(label, property));

View File

@ -8,14 +8,16 @@
#include "gtest/gtest.h"
#include "communication/result_stream_faker.hpp"
#include "database/graph_db.hpp"
#include "database/graph_db_accessor.hpp"
#include "query/interpreter.hpp"
DECLARE_bool(query_cost_planner);
class QueryExecution : public testing::Test {
protected:
std::experimental::optional<GraphDb> db_;
std::experimental::optional<GraphDbAccessor> dba_;
std::experimental::optional<database::SingleNode> db_;
std::experimental::optional<database::GraphDbAccessor> dba_;
void SetUp() {
db_.emplace();

View File

@ -27,8 +27,8 @@ using namespace query::plan;
class MatchReturnFixture : public testing::Test {
protected:
GraphDb db_;
GraphDbAccessor dba_{db_};
database::SingleNode db_;
database::GraphDbAccessor dba_{db_};
AstTreeStorage storage;
SymbolTable symbol_table;
@ -91,8 +91,8 @@ TEST_F(MatchReturnFixture, MatchReturnPath) {
}
TEST(QueryPlan, MatchReturnCartesian) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
dba.InsertVertex().add_label(dba.Label("l1"));
dba.InsertVertex().add_label(dba.Label("l2"));
@ -124,8 +124,8 @@ TEST(QueryPlan, MatchReturnCartesian) {
}
TEST(QueryPlan, StandaloneReturn) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// add a few nodes to the database
dba.InsertVertex();
@ -146,11 +146,11 @@ TEST(QueryPlan, StandaloneReturn) {
}
TEST(QueryPlan, NodeFilterLabelsAndProperties) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// add a few nodes to the database
GraphDbTypes::Label label = dba.Label("Label");
database::Label label = dba.Label("Label");
auto property = PROPERTY_PAIR("Property");
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -202,13 +202,13 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
}
TEST(QueryPlan, NodeFilterMultipleLabels) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// add a few nodes to the database
GraphDbTypes::Label label1 = dba.Label("label1");
GraphDbTypes::Label label2 = dba.Label("label2");
GraphDbTypes::Label label3 = dba.Label("label3");
database::Label label1 = dba.Label("label1");
database::Label label2 = dba.Label("label2");
database::Label label3 = dba.Label("label3");
// the test will look for nodes that have label1 and label2
dba.InsertVertex(); // NOT accepted
dba.InsertVertex().add_label(label1); // NOT accepted
@ -253,8 +253,8 @@ TEST(QueryPlan, NodeFilterMultipleLabels) {
class ExpandFixture : public testing::Test {
protected:
GraphDb db_;
GraphDbAccessor dba_{db_};
database::SingleNode db_;
database::GraphDbAccessor dba_{db_};
AstTreeStorage storage;
SymbolTable symbol_table;
@ -262,7 +262,7 @@ class ExpandFixture : public testing::Test {
VertexAccessor v1 = dba_.InsertVertex();
VertexAccessor v2 = dba_.InsertVertex();
VertexAccessor v3 = dba_.InsertVertex();
GraphDbTypes::EdgeType edge_type = dba_.EdgeType("Edge");
database::EdgeType edge_type = dba_.EdgeType("Edge");
EdgeAccessor r1 = dba_.InsertEdge(v1, v2, edge_type);
EdgeAccessor r2 = dba_.InsertEdge(v1, v3, edge_type);
@ -353,12 +353,12 @@ class QueryPlanExpandVariable : public testing::Test {
// a lot below in test declaration
using map_int = std::unordered_map<int, int>;
GraphDb db_;
GraphDbAccessor dba_{db_};
database::SingleNode db_;
database::GraphDbAccessor dba_{db_};
// labels for layers in the double chain
std::vector<GraphDbTypes::Label> labels;
std::vector<database::Label> labels;
// for all the edges
GraphDbTypes::EdgeType edge_type = dba_.EdgeType("edge_type");
database::EdgeType edge_type = dba_.EdgeType("edge_type");
AstTreeStorage storage;
SymbolTable symbol_table;
@ -409,7 +409,7 @@ class QueryPlanExpandVariable : public testing::Test {
std::shared_ptr<LogicalOperator> AddMatch(
std::shared_ptr<LogicalOperator> input_op, const std::string &node_from,
int layer, EdgeAtom::Direction direction,
const std::vector<GraphDbTypes::EdgeType> &edge_types,
const std::vector<database::EdgeType> &edge_types,
std::experimental::optional<size_t> lower,
std::experimental::optional<size_t> upper, Symbol edge_sym,
const std::string &node_to, GraphView graph_view = GraphView::AS_IS,
@ -418,7 +418,7 @@ class QueryPlanExpandVariable : public testing::Test {
auto filter_op = std::make_shared<Filter>(
n_from.op_, storage.Create<query::LabelsTest>(
n_from.node_->identifier_,
std::vector<GraphDbTypes::Label>{labels[layer]}));
std::vector<database::Label>{labels[layer]}));
auto n_to = NODE(node_to);
auto n_to_sym = symbol_table.CreateSymbol(node_to, true);
@ -619,15 +619,14 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessTwoVariableExpansions) {
}
TEST_F(QueryPlanExpandVariable, GraphState) {
auto test_expand =
[&](GraphView graph_view,
const std::vector<GraphDbTypes::EdgeType> &edge_types) {
auto e = Edge("r", EdgeAtom::Direction::OUT);
return GetEdgeListSizes(
AddMatch<ExpandVariable>(nullptr, "n", 0, EdgeAtom::Direction::OUT,
edge_types, 2, 2, e, "m", graph_view),
e);
};
auto test_expand = [&](GraphView graph_view,
const std::vector<database::EdgeType> &edge_types) {
auto e = Edge("r", EdgeAtom::Direction::OUT);
return GetEdgeListSizes(
AddMatch<ExpandVariable>(nullptr, "n", 0, EdgeAtom::Direction::OUT,
edge_types, 2, 2, e, "m", graph_view),
e);
};
auto new_edge_type = dba_.EdgeType("some_type");
// add two vertices branching out from the second layer
@ -698,11 +697,10 @@ class QueryPlanExpandBreadthFirst : public testing::Test {
protected:
// style-guide non-conformant name due to PROPERTY_PAIR and PROPERTY_LOOKUP
// macro requirements
GraphDb db;
GraphDbAccessor dba{db};
std::pair<std::string, GraphDbTypes::Property> prop =
PROPERTY_PAIR("property");
GraphDbTypes::EdgeType edge_type = dba.EdgeType("edge_type");
database::SingleNode db;
database::GraphDbAccessor dba{db};
std::pair<std::string, database::Property> prop = PROPERTY_PAIR("property");
database::EdgeType edge_type = dba.EdgeType("edge_type");
// make 4 vertices because we'll need to compare against them exactly
// v[0] has `prop` with the value 0
@ -767,9 +765,9 @@ class QueryPlanExpandBreadthFirst : public testing::Test {
auto edge_list_sym = symbol_table.CreateSymbol("edgelist_", true);
last_op = std::make_shared<ExpandVariable>(
node_sym, edge_list_sym, EdgeAtom::Type::BREADTH_FIRST, direction,
std::vector<GraphDbTypes::EdgeType>{}, false, nullptr,
LITERAL(max_depth), last_op, n.sym_, existing_node_input != nullptr,
inner_edge, inner_node, where, graph_view);
std::vector<database::EdgeType>{}, false, nullptr, LITERAL(max_depth),
last_op, n.sym_, existing_node_input != nullptr, inner_edge, inner_node,
where, graph_view);
Frame frame(symbol_table.max_position());
auto cursor = last_op->MakeCursor(dba);
@ -916,8 +914,8 @@ TEST_F(QueryPlanExpandBreadthFirst, ExistingNode) {
}
TEST(QueryPlan, ExpandOptional) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -975,8 +973,8 @@ TEST(QueryPlan, ExpandOptional) {
}
TEST(QueryPlan, OptionalMatchEmptyDB) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -997,8 +995,8 @@ TEST(QueryPlan, OptionalMatchEmptyDB) {
}
TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
// OPTIONAL MATCH (n)
@ -1024,8 +1022,8 @@ TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
}
TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Make a graph with 2 connected, unlabeled nodes.
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -1061,8 +1059,8 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
auto node = NODE("n");
symbol_table[*node->identifier_] = with_n_sym;
auto expand = std::make_shared<plan::Expand>(
with_n_sym, edge_sym, edge_direction,
std::vector<GraphDbTypes::EdgeType>{}, m.op_, m.sym_, true);
with_n_sym, edge_sym, edge_direction, std::vector<database::EdgeType>{},
m.op_, m.sym_, true);
// RETURN m
auto m_ne = NEXPR("m", IDENT("m"));
symbol_table[*m_ne->expression_] = m.sym_;
@ -1073,8 +1071,8 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
}
TEST(QueryPlan, ExpandExistingNode) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// make a graph (v1)->(v2) that
// has a recursive edge (v1)->(v1)
@ -1095,7 +1093,7 @@ TEST(QueryPlan, ExpandExistingNode) {
if (with_existing)
r_n.op_ = std::make_shared<Expand>(
n.sym_, r_n.edge_sym_, r_n.edge_->direction_,
std::vector<GraphDbTypes::EdgeType>{}, n.op_, n.sym_, with_existing);
std::vector<database::EdgeType>{}, n.op_, n.sym_, with_existing);
// make a named expression and a produce
auto output = NEXPR("n", IDENT("n"));
@ -1115,8 +1113,8 @@ TEST(QueryPlan, ExpandExistingNode) {
TEST(QueryPlan, ExpandBothCycleEdgeCase) {
// we're testing that expanding on BOTH
// does only one expansion for a cycle
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v = dba.InsertVertex();
dba.InsertEdge(v, v, dba.EdgeType("et"));
@ -1132,14 +1130,14 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) {
}
TEST(QueryPlan, EdgeFilter) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// make an N-star expanding from (v1)
// where only one edge will qualify
// and there are all combinations of
// (edge_type yes|no) * (property yes|absent|no)
std::vector<GraphDbTypes::EdgeType> edge_types;
std::vector<database::EdgeType> edge_types;
for (int j = 0; j < 2; ++j)
edge_types.push_back(dba.EdgeType("et" + std::to_string(j)));
std::vector<VertexAccessor> vertices;
@ -1200,8 +1198,8 @@ TEST(QueryPlan, EdgeFilter) {
}
TEST(QueryPlan, EdgeFilterMultipleTypes) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -1234,8 +1232,8 @@ TEST(QueryPlan, EdgeFilterMultipleTypes) {
}
TEST(QueryPlan, Filter) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// add a 6 nodes with property 'prop', 2 have true as value
auto property = PROPERTY_PAIR("property");
@ -1263,8 +1261,8 @@ TEST(QueryPlan, Filter) {
}
TEST(QueryPlan, ExpandUniquenessFilter) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// make a graph that has (v1)->(v2) and a recursive edge (v1)->(v1)
auto v1 = dba.InsertVertex();
@ -1309,8 +1307,8 @@ TEST(QueryPlan, Distinct) {
// test queries like
// UNWIND [1, 2, 3, 3] AS x RETURN DISTINCT x
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
SymbolTable symbol_table;
@ -1352,8 +1350,8 @@ TEST(QueryPlan, Distinct) {
}
TEST(QueryPlan, ScanAllByLabel) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Add a vertex with a label and one without.
auto label = dba.Label("label");
auto labeled_vertex = dba.InsertVertex();
@ -1379,10 +1377,10 @@ TEST(QueryPlan, ScanAllByLabel) {
}
TEST(QueryPlan, ScanAllByLabelProperty) {
GraphDb db;
database::SingleNode db;
// Add 5 vertices with same label, but with different property values.
auto label = GraphDbAccessor(db).Label("label");
auto prop = GraphDbAccessor(db).Property("prop");
auto label = database::GraphDbAccessor(db).Label("label");
auto prop = database::GraphDbAccessor(db).Property("prop");
// vertex property values that will be stored into the DB
// clang-format off
std::vector<TypedValue> values{
@ -1391,16 +1389,16 @@ TEST(QueryPlan, ScanAllByLabelProperty) {
std::vector<TypedValue>{2}};
// clang-format on
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
for (const auto &value : values) {
auto vertex = dba.InsertVertex();
vertex.add_label(label);
vertex.PropsSet(prop, value);
}
dba.Commit();
GraphDbAccessor(db).BuildIndex(label, prop);
database::GraphDbAccessor(db).BuildIndex(label, prop);
}
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
ASSERT_EQ(14, CountIterable(dba.Vertices(false)));
auto check = [&dba, label, prop](TypedValue lower, Bound::Type lower_type,
@ -1449,13 +1447,13 @@ TEST(QueryPlan, ScanAllByLabelProperty) {
}
TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
GraphDb db;
database::SingleNode db;
// Add 2 vertices with same label, but with property values that cannot be
// compared. On the other hand, equality works fine.
auto label = GraphDbAccessor(db).Label("label");
auto prop = GraphDbAccessor(db).Property("prop");
auto label = database::GraphDbAccessor(db).Label("label");
auto prop = database::GraphDbAccessor(db).Property("prop");
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto number_vertex = dba.InsertVertex();
number_vertex.add_label(label);
number_vertex.PropsSet(prop, 42);
@ -1463,9 +1461,9 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
string_vertex.add_label(label);
string_vertex.PropsSet(prop, "string");
dba.Commit();
GraphDbAccessor(db).BuildIndex(label, prop);
database::GraphDbAccessor(db).BuildIndex(label, prop);
}
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
// MATCH (n :label {prop: 42})
AstTreeStorage storage;
@ -1488,24 +1486,24 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
}
TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) {
GraphDb db;
database::SingleNode db;
// Add 2 vertices with the same label, but one has a property value while
// the
// other does not. Checking if the value is equal to null, should yield no
// results.
auto label = GraphDbAccessor(db).Label("label");
auto prop = GraphDbAccessor(db).Property("prop");
auto label = database::GraphDbAccessor(db).Label("label");
auto prop = database::GraphDbAccessor(db).Property("prop");
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex = dba.InsertVertex();
vertex.add_label(label);
auto vertex_with_prop = dba.InsertVertex();
vertex_with_prop.add_label(label);
vertex_with_prop.PropsSet(prop, 42);
dba.Commit();
GraphDbAccessor(db).BuildIndex(label, prop);
database::GraphDbAccessor(db).BuildIndex(label, prop);
}
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
// MATCH (n :label {prop: 42})
AstTreeStorage storage;

View File

@ -243,8 +243,8 @@ class ExpectScanAllByLabelPropertyValue
: public OpChecker<ScanAllByLabelPropertyValue> {
public:
ExpectScanAllByLabelPropertyValue(
GraphDbTypes::Label label,
const std::pair<std::string, GraphDbTypes::Property> &prop_pair,
database::Label label,
const std::pair<std::string, database::Property> &prop_pair,
query::Expression *expression)
: label_(label), property_(prop_pair.second), expression_(expression) {}
@ -256,8 +256,8 @@ class ExpectScanAllByLabelPropertyValue
}
private:
GraphDbTypes::Label label_;
GraphDbTypes::Property property_;
database::Label label_;
database::Property property_;
query::Expression *expression_;
};
@ -265,7 +265,7 @@ class ExpectScanAllByLabelPropertyRange
: public OpChecker<ScanAllByLabelPropertyRange> {
public:
ExpectScanAllByLabelPropertyRange(
GraphDbTypes::Label label, GraphDbTypes::Property property,
database::Label label, database::Property property,
std::experimental::optional<Bound> lower_bound,
std::experimental::optional<Bound> upper_bound)
: label_(label),
@ -290,15 +290,15 @@ class ExpectScanAllByLabelPropertyRange
}
private:
GraphDbTypes::Label label_;
GraphDbTypes::Property property_;
database::Label label_;
database::Property property_;
std::experimental::optional<Bound> lower_bound_;
std::experimental::optional<Bound> upper_bound_;
};
class ExpectCreateIndex : public OpChecker<CreateIndex> {
public:
ExpectCreateIndex(GraphDbTypes::Label label, GraphDbTypes::Property property)
ExpectCreateIndex(database::Label label, database::Property property)
: label_(label), property_(property) {}
void ExpectOp(CreateIndex &create_index, const SymbolTable &) override {
@ -307,8 +307,8 @@ class ExpectCreateIndex : public OpChecker<CreateIndex> {
}
private:
GraphDbTypes::Label label_;
GraphDbTypes::Property property_;
database::Label label_;
database::Property property_;
};
auto MakeSymbolTable(query::Query &query) {
@ -330,8 +330,8 @@ auto CheckPlan(LogicalOperator &plan, const SymbolTable &symbol_table,
template <class... TChecker>
auto CheckPlan(AstTreeStorage &storage, TChecker... checker) {
auto symbol_table = MakeSymbolTable(*storage.query());
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
auto query_parts = CollectQueryParts(symbol_table, storage);
ASSERT_TRUE(query_parts.query_parts.size() > 0);
@ -356,8 +356,8 @@ TEST(TestLogicalPlanner, CreateNodeReturn) {
QUERY(SINGLE_QUERY(CREATE(PATTERN(NODE("n"))), RETURN(ident_n, AS("n"))));
auto symbol_table = MakeSymbolTable(*query);
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
auto query_parts = CollectQueryParts(symbol_table, storage);
ASSERT_TRUE(query_parts.query_parts.size() > 0);
@ -370,8 +370,8 @@ TEST(TestLogicalPlanner, CreateNodeReturn) {
TEST(TestLogicalPlanner, CreateExpand) {
// Test CREATE (n) -[r :rel1]-> (m)
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto relationship = dba.EdgeType("relationship");
QUERY(SINGLE_QUERY(CREATE(PATTERN(
NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
@ -388,8 +388,8 @@ TEST(TestLogicalPlanner, CreateMultipleNode) {
TEST(TestLogicalPlanner, CreateNodeExpandNode) {
// Test CREATE (n) -[r :rel]-> (m), (l)
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto relationship = dba.EdgeType("rel");
QUERY(SINGLE_QUERY(CREATE(
PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")),
@ -401,8 +401,8 @@ TEST(TestLogicalPlanner, CreateNodeExpandNode) {
TEST(TestLogicalPlanner, CreateNamedPattern) {
// Test CREATE p = (n) -[r :rel]-> (m)
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto relationship = dba.EdgeType("rel");
QUERY(SINGLE_QUERY(CREATE(NAMED_PATTERN(
"p", NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
@ -413,8 +413,8 @@ TEST(TestLogicalPlanner, CreateNamedPattern) {
TEST(TestLogicalPlanner, MatchCreateExpand) {
// Test MATCH (n) CREATE (n) -[r :rel1]-> (m)
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto relationship = dba.EdgeType("relationship");
QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"))),
@ -426,8 +426,8 @@ TEST(TestLogicalPlanner, MatchCreateExpand) {
TEST(TestLogicalPlanner, MatchLabeledNodes) {
// Test MATCH (n :label) RETURN n
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))), RETURN("n")));
CheckPlan(storage, ExpectScanAllByLabel(), ExpectProduce());
@ -436,8 +436,8 @@ TEST(TestLogicalPlanner, MatchLabeledNodes) {
TEST(TestLogicalPlanner, MatchPathReturn) {
// Test MATCH (n) -[r :relationship]- (m) RETURN n
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto relationship = dba.EdgeType("relationship");
QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"), EDGE("r", Direction::BOTH, {relationship}),
@ -449,8 +449,8 @@ TEST(TestLogicalPlanner, MatchPathReturn) {
TEST(TestLogicalPlanner, MatchNamedPatternReturn) {
// Test MATCH p = (n) -[r :relationship]- (m) RETURN p
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto relationship = dba.EdgeType("relationship");
QUERY(SINGLE_QUERY(
MATCH(NAMED_PATTERN("p", NODE("n"),
@ -464,8 +464,8 @@ TEST(TestLogicalPlanner, MatchNamedPatternReturn) {
TEST(TestLogicalPlanner, MatchNamedPatternWithPredicateReturn) {
// Test MATCH p = (n) -[r :relationship]- (m) RETURN p
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto relationship = dba.EdgeType("relationship");
QUERY(SINGLE_QUERY(
MATCH(NAMED_PATTERN("p", NODE("n"),
@ -478,8 +478,8 @@ TEST(TestLogicalPlanner, MatchNamedPatternWithPredicateReturn) {
TEST(TestLogicalPlanner, OptionalMatchNamedPatternReturn) {
// Test OPTIONAL MATCH p = (n) -[r]- (m) RETURN p
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
auto node_n = NODE("n");
auto edge = EDGE("r");
@ -508,8 +508,8 @@ TEST(TestLogicalPlanner, OptionalMatchNamedPatternReturn) {
TEST(TestLogicalPlanner, MatchWhereReturn) {
// Test MATCH (n) WHERE n.property < 42 RETURN n
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto property = dba.Property("property");
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
WHERE(LESS(PROPERTY_LOOKUP("n", property), LITERAL(42))),
@ -527,8 +527,8 @@ TEST(TestLogicalPlanner, MatchDelete) {
TEST(TestLogicalPlanner, MatchNodeSet) {
// Test MATCH (n) SET n.prop = 42, n = n, n :label
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
auto label = dba.Label("label");
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
@ -541,8 +541,8 @@ TEST(TestLogicalPlanner, MatchNodeSet) {
TEST(TestLogicalPlanner, MatchRemove) {
// Test MATCH (n) REMOVE n.prop REMOVE n :label
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
auto label = dba.Label("label");
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
@ -624,8 +624,8 @@ TEST(TestLogicalPlanner, MatchWithReturn) {
TEST(TestLogicalPlanner, MatchWithWhereReturn) {
// Test MATCH (old) WITH old AS new WHERE new.prop < 42 RETURN new
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("old"))), WITH("old", AS("new")),
@ -638,8 +638,8 @@ TEST(TestLogicalPlanner, MatchWithWhereReturn) {
TEST(TestLogicalPlanner, CreateMultiExpand) {
// Test CREATE (n) -[r :r]-> (m), (n) - [p :p]-> (l)
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto r = dba.EdgeType("r");
auto p = dba.EdgeType("p");
AstTreeStorage storage;
@ -653,8 +653,8 @@ TEST(TestLogicalPlanner, CreateMultiExpand) {
TEST(TestLogicalPlanner, MatchWithSumWhereReturn) {
// Test MATCH (n) WITH SUM(n.prop) + 42 AS sum WHERE sum < 42
// RETURN sum AS result
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
auto sum = SUM(PROPERTY_LOOKUP("n", prop));
@ -669,8 +669,8 @@ TEST(TestLogicalPlanner, MatchWithSumWhereReturn) {
TEST(TestLogicalPlanner, MatchReturnSum) {
// Test MATCH (n) RETURN SUM(n.prop1) AS sum, n.prop2 AS group
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop1 = dba.Property("prop1");
auto prop2 = dba.Property("prop2");
AstTreeStorage storage;
@ -684,8 +684,8 @@ TEST(TestLogicalPlanner, MatchReturnSum) {
TEST(TestLogicalPlanner, CreateWithSum) {
// Test CREATE (n) WITH SUM(n.prop) AS sum
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
auto n_prop = PROPERTY_LOOKUP("n", prop);
@ -709,8 +709,8 @@ TEST(TestLogicalPlanner, CreateWithSum) {
TEST(TestLogicalPlanner, MatchWithCreate) {
// Test MATCH (n) WITH n AS a CREATE (a) -[r :r]-> (b)
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto r_type = dba.EdgeType("r");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(
@ -738,8 +738,8 @@ TEST(TestLogicalPlanner, CreateWithSkipReturnLimit) {
RETURN("m", LIMIT(LITERAL(1)))));
auto symbol_table = MakeSymbolTable(*query);
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
auto query_parts = CollectQueryParts(symbol_table, storage);
ASSERT_TRUE(query_parts.query_parts.size() > 0);
@ -757,8 +757,8 @@ TEST(TestLogicalPlanner, CreateWithSkipReturnLimit) {
TEST(TestLogicalPlanner, CreateReturnSumSkipLimit) {
// Test CREATE (n) RETURN SUM(n.prop) AS s SKIP 2 LIMIT 1
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
auto n_prop = PROPERTY_LOOKUP("n", prop);
@ -781,8 +781,8 @@ TEST(TestLogicalPlanner, CreateReturnSumSkipLimit) {
TEST(TestLogicalPlanner, MatchReturnOrderBy) {
// Test MATCH (n) RETURN n ORDER BY n.prop
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
auto ret = RETURN("n", ORDER_BY(PROPERTY_LOOKUP("n", prop)));
@ -793,8 +793,8 @@ TEST(TestLogicalPlanner, MatchReturnOrderBy) {
TEST(TestLogicalPlanner, CreateWithOrderByWhere) {
// Test CREATE (n) -[r :r]-> (m)
// WITH n AS new ORDER BY new.prop, r.prop WHERE m.prop < 42
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
auto r_type = dba.EdgeType("r");
AstTreeStorage storage;
@ -839,8 +839,8 @@ TEST(TestLogicalPlanner, MatchMerge) {
// Test MATCH (n) MERGE (n) -[r :r]- (m)
// ON MATCH SET n.prop = 42 ON CREATE SET m = n
// RETURN n AS n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto r_type = dba.EdgeType("r");
auto prop = dba.Property("prop");
AstTreeStorage storage;
@ -874,8 +874,8 @@ TEST(TestLogicalPlanner, MatchMerge) {
TEST(TestLogicalPlanner, MatchOptionalMatchWhereReturn) {
// Test MATCH (n) OPTIONAL MATCH (n) -[r]- (m) WHERE m.prop < 42 RETURN r
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
@ -908,8 +908,8 @@ TEST(TestLogicalPlanner, ReturnDistinctOrderBySkipLimit) {
TEST(TestLogicalPlanner, CreateWithDistinctSumWhereReturn) {
// Test CREATE (n) WITH DISTINCT SUM(n.prop) AS s WHERE s < 42 RETURN s
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
auto node_n = NODE("n");
@ -932,8 +932,8 @@ TEST(TestLogicalPlanner, CreateWithDistinctSumWhereReturn) {
TEST(TestLogicalPlanner, MatchCrossReferenceVariable) {
// Test MATCH (n {prop: m.prop}), (m {prop: n.prop}) RETURN n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = PROPERTY_PAIR("prop");
AstTreeStorage storage;
auto node_n = NODE("n");
@ -951,8 +951,8 @@ TEST(TestLogicalPlanner, MatchCrossReferenceVariable) {
TEST(TestLogicalPlanner, MatchWhereBeforeExpand) {
// Test MATCH (n) -[r]- (m) WHERE n.prop < 42 RETURN n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
@ -965,8 +965,8 @@ TEST(TestLogicalPlanner, MatchWhereBeforeExpand) {
TEST(TestLogicalPlanner, MultiMatchWhere) {
// Test MATCH (n) -[r]- (m) MATCH (l) WHERE n.prop < 42 RETURN n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
@ -981,8 +981,8 @@ TEST(TestLogicalPlanner, MultiMatchWhere) {
TEST(TestLogicalPlanner, MatchOptionalMatchWhere) {
// Test MATCH (n) -[r]- (m) OPTIONAL MATCH (l) WHERE n.prop < 42 RETURN n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
@ -999,8 +999,8 @@ TEST(TestLogicalPlanner, MatchOptionalMatchWhere) {
TEST(TestLogicalPlanner, MatchReturnAsterisk) {
// Test MATCH (n) -[e]- (m) RETURN *, m.prop
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
auto ret = RETURN(PROPERTY_LOOKUP("m", prop), AS("m.prop"));
@ -1026,8 +1026,8 @@ TEST(TestLogicalPlanner, MatchReturnAsterisk) {
TEST(TestLogicalPlanner, MatchReturnAsteriskSum) {
// Test MATCH (n) RETURN *, SUM(n.prop) AS s
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = dba.Property("prop");
AstTreeStorage storage;
auto sum = SUM(PROPERTY_LOOKUP("n", prop));
@ -1060,8 +1060,8 @@ TEST(TestLogicalPlanner, MatchReturnAsteriskSum) {
TEST(TestLogicalPlanner, UnwindMergeNodeProperty) {
// Test UNWIND [1] AS i MERGE (n {prop: i})
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
auto node_n = NODE("n");
node_n->properties_[PROPERTY_PAIR("prop")] = IDENT("i");
@ -1116,8 +1116,8 @@ TEST(TestLogicalPlanner, ListLiteralAggregationReturn) {
TEST(TestLogicalPlanner, MapLiteralAggregationReturn) {
// Test RETURN {sum: SUM(2)} AS result, 42 AS group_by
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto sum = SUM(LITERAL(2));
auto group_by_literal = LITERAL(42);
QUERY(SINGLE_QUERY(RETURN(MAP({PROPERTY_PAIR("sum"), sum}), AS("result"),
@ -1180,7 +1180,7 @@ TEST(TestLogicalPlanner, AggregatonWithListWithAggregationAndGroupBy) {
TEST(TestLogicalPlanner, MapWithAggregationAndGroupBy) {
// Test RETURN {lit: 42, sum: sum(2)}
GraphDb db;
database::SingleNode db;
AstTreeStorage storage;
auto sum = SUM(LITERAL(2));
auto group_by_literal = LITERAL(42);
@ -1193,8 +1193,8 @@ TEST(TestLogicalPlanner, MapWithAggregationAndGroupBy) {
TEST(TestLogicalPlanner, CreateIndex) {
// Test CREATE INDEX ON :Label(property)
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto property = dba.Property("property");
AstTreeStorage storage;
@ -1205,8 +1205,8 @@ TEST(TestLogicalPlanner, CreateIndex) {
TEST(TestLogicalPlanner, AtomIndexedLabelProperty) {
// Test MATCH (n :label {property: 42, not_indexed: 0}) RETURN n
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto property = PROPERTY_PAIR("property");
auto not_indexed = PROPERTY_PAIR("not_indexed");
@ -1214,9 +1214,9 @@ TEST(TestLogicalPlanner, AtomIndexedLabelProperty) {
vertex.add_label(label);
vertex.PropsSet(property.second, 42);
dba.Commit();
GraphDbAccessor(db).BuildIndex(label, property.second);
database::GraphDbAccessor(db).BuildIndex(label, property.second);
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto node = NODE("n", label);
auto lit_42 = LITERAL(42);
node->properties_[property] = lit_42;
@ -1239,14 +1239,14 @@ TEST(TestLogicalPlanner, AtomIndexedLabelProperty) {
TEST(TestLogicalPlanner, AtomPropertyWhereLabelIndexing) {
// Test MATCH (n {property: 42}) WHERE n.not_indexed AND n:label RETURN n
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto property = PROPERTY_PAIR("property");
auto not_indexed = PROPERTY_PAIR("not_indexed");
dba.BuildIndex(label, property.second);
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto node = NODE("n");
auto lit_42 = LITERAL(42);
node->properties_[property] = lit_42;
@ -1254,7 +1254,7 @@ TEST(TestLogicalPlanner, AtomPropertyWhereLabelIndexing) {
MATCH(PATTERN(node)),
WHERE(AND(PROPERTY_LOOKUP("n", not_indexed),
storage.Create<query::LabelsTest>(
IDENT("n"), std::vector<GraphDbTypes::Label>{label}))),
IDENT("n"), std::vector<database::Label>{label}))),
RETURN("n")));
auto symbol_table = MakeSymbolTable(*storage.query());
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
@ -1273,13 +1273,13 @@ TEST(TestLogicalPlanner, AtomPropertyWhereLabelIndexing) {
TEST(TestLogicalPlanner, WhereIndexedLabelProperty) {
// Test MATCH (n :label) WHERE n.property = 42 RETURN n
AstTreeStorage storage;
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto property = PROPERTY_PAIR("property");
dba.BuildIndex(label, property.second);
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto lit_42 = LITERAL(42);
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
WHERE(EQ(PROPERTY_LOOKUP("n", property), lit_42)),
@ -1300,12 +1300,12 @@ TEST(TestLogicalPlanner, WhereIndexedLabelProperty) {
TEST(TestLogicalPlanner, BestPropertyIndexed) {
// Test MATCH (n :label) WHERE n.property = 1 AND n.better = 42 RETURN n
AstTreeStorage storage;
GraphDb db;
auto label = GraphDbAccessor(db).Label("label");
auto property = GraphDbAccessor(db).Property("property");
database::SingleNode db;
auto label = database::GraphDbAccessor(db).Label("label");
auto property = database::GraphDbAccessor(db).Property("property");
{
GraphDbAccessor(db).BuildIndex(label, property);
GraphDbAccessor dba(db);
database::GraphDbAccessor(db).BuildIndex(label, property);
database::GraphDbAccessor dba(db);
// Add a vertex with :label+property combination, so that the best
// :label+better remains empty and thus better choice.
auto vertex = dba.InsertVertex();
@ -1313,11 +1313,11 @@ TEST(TestLogicalPlanner, BestPropertyIndexed) {
vertex.PropsSet(property, 1);
dba.Commit();
}
ASSERT_EQ(GraphDbAccessor(db).VerticesCount(label, property), 1);
ASSERT_EQ(database::GraphDbAccessor(db).VerticesCount(label, property), 1);
auto better = PROPERTY_PAIR("better");
GraphDbAccessor(db).BuildIndex(label, better.second);
database::GraphDbAccessor(db).BuildIndex(label, better.second);
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
ASSERT_EQ(dba.VerticesCount(label, better.second), 0);
auto lit_42 = LITERAL(42);
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
@ -1340,14 +1340,14 @@ TEST(TestLogicalPlanner, BestPropertyIndexed) {
TEST(TestLogicalPlanner, MultiPropertyIndexScan) {
// Test MATCH (n :label1), (m :label2) WHERE n.prop1 = 1 AND m.prop2 = 2
// RETURN n, m
GraphDb db;
auto label1 = GraphDbAccessor(db).Label("label1");
auto label2 = GraphDbAccessor(db).Label("label2");
database::SingleNode db;
auto label1 = database::GraphDbAccessor(db).Label("label1");
auto label2 = database::GraphDbAccessor(db).Label("label2");
auto prop1 = PROPERTY_PAIR("prop1");
auto prop2 = PROPERTY_PAIR("prop2");
GraphDbAccessor(db).BuildIndex(label1, prop1.second);
GraphDbAccessor(db).BuildIndex(label2, prop2.second);
GraphDbAccessor dba(db);
database::GraphDbAccessor(db).BuildIndex(label1, prop1.second);
database::GraphDbAccessor(db).BuildIndex(label2, prop2.second);
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
auto lit_1 = LITERAL(1);
auto lit_2 = LITERAL(2);
@ -1372,11 +1372,11 @@ TEST(TestLogicalPlanner, MultiPropertyIndexScan) {
TEST(TestLogicalPlanner, WhereIndexedLabelPropertyRange) {
// Test MATCH (n :label) WHERE n.property REL_OP 42 RETURN n
// REL_OP is one of: `<`, `<=`, `>`, `>=`
GraphDb db;
auto label = GraphDbAccessor(db).Label("label");
auto property = GraphDbAccessor(db).Property("property");
GraphDbAccessor(db).BuildIndex(label, property);
GraphDbAccessor dba(db);
database::SingleNode db;
auto label = database::GraphDbAccessor(db).Label("label");
auto property = database::GraphDbAccessor(db).Property("property");
database::GraphDbAccessor(db).BuildIndex(label, property);
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
auto lit_42 = LITERAL(42);
auto n_prop = PROPERTY_LOOKUP("n", property);
@ -1426,13 +1426,13 @@ TEST(TestLogicalPlanner, WhereIndexedLabelPropertyRange) {
TEST(TestLogicalPlanner, UnableToUsePropertyIndex) {
// Test MATCH (n: label) WHERE n.property = n.property RETURN n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto property = dba.Property("property");
dba.BuildIndex(label, property);
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
WHERE(EQ(PROPERTY_LOOKUP("n", property),
@ -1454,13 +1454,13 @@ TEST(TestLogicalPlanner, UnableToUsePropertyIndex) {
TEST(TestLogicalPlanner, SecondPropertyIndex) {
// Test MATCH (n :label), (m :label) WHERE m.property = n.property RETURN n
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto property = PROPERTY_PAIR("property");
dba.BuildIndex(label, dba.Property("property"));
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
auto n_prop = PROPERTY_LOOKUP("n", property);
auto m_prop = PROPERTY_LOOKUP("m", property);
@ -1511,8 +1511,8 @@ TEST(TestLogicalPlanner, MatchExpandVariableNoBounds) {
TEST(TestLogicalPlanner, MatchExpandVariableInlinedFilter) {
// Test MATCH (n) -[r :type * {prop: 42}]-> (m) RETURN r
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto type = dba.EdgeType("type");
auto prop = PROPERTY_PAIR("prop");
AstTreeStorage storage;
@ -1526,8 +1526,8 @@ TEST(TestLogicalPlanner, MatchExpandVariableInlinedFilter) {
TEST(TestLogicalPlanner, MatchExpandVariableNotInlinedFilter) {
// Test MATCH (n) -[r :type * {prop: m.prop}]-> (m) RETURN r
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto type = dba.EdgeType("type");
auto prop = PROPERTY_PAIR("prop");
AstTreeStorage storage;
@ -1552,13 +1552,13 @@ TEST(TestLogicalPlanner, UnwindMatchVariable) {
TEST(TestLogicalPlanner, MatchBreadthFirst) {
// Test MATCH (n) -[r:type *..10 (r, n|n)]-> (m) RETURN r
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto edge_type = dba.EdgeType("type");
AstTreeStorage storage;
auto *bfs = storage.Create<query::EdgeAtom>(
IDENT("r"), query::EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
std::vector<GraphDbTypes::EdgeType>{edge_type});
std::vector<database::EdgeType>{edge_type});
bfs->inner_edge_ = IDENT("r");
bfs->inner_node_ = IDENT("n");
bfs->filter_expression_ = IDENT("n");
@ -1570,8 +1570,8 @@ TEST(TestLogicalPlanner, MatchBreadthFirst) {
TEST(TestLogicalPlanner, MatchDoubleScanToExpandExisting) {
// Test MATCH (n) -[r]- (m :label) RETURN r
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m", label))),
@ -1591,11 +1591,11 @@ TEST(TestLogicalPlanner, MatchDoubleScanToExpandExisting) {
TEST(TestLogicalPlanner, MatchScanToExpand) {
// Test MATCH (n) -[r]- (m :label {property: 1}) RETURN r
GraphDb db;
auto label = GraphDbAccessor(db).Label("label");
auto property = GraphDbAccessor(db).Property("property");
GraphDbAccessor(db).BuildIndex(label, property);
GraphDbAccessor dba(db);
database::SingleNode db;
auto label = database::GraphDbAccessor(db).Label("label");
auto property = database::GraphDbAccessor(db).Property("property");
database::GraphDbAccessor(db).BuildIndex(label, property);
database::GraphDbAccessor dba(db);
// Fill vertices to the max.
for (int64_t i = 0; i < FLAGS_query_vertex_count_to_expand_existing; ++i) {
auto vertex = dba.InsertVertex();
@ -1608,7 +1608,7 @@ TEST(TestLogicalPlanner, MatchScanToExpand) {
vertex.PropsSet(property, 1);
dba.Commit();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
auto node_m = NODE("m", label);
node_m->properties_[std::make_pair("property", property)] = LITERAL(1);
@ -1630,8 +1630,8 @@ TEST(TestLogicalPlanner, MatchScanToExpand) {
TEST(TestLogicalPlanner, MatchWhereAndSplit) {
// Test MATCH (n) -[r]- (m) WHERE n.prop AND r.prop RETURN m
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto prop = PROPERTY_PAIR("prop");
AstTreeStorage storage;
QUERY(SINGLE_QUERY(
@ -1645,8 +1645,8 @@ TEST(TestLogicalPlanner, MatchWhereAndSplit) {
TEST(TestLogicalPlanner, ReturnAsteriskOmitsLambdaSymbols) {
// Test MATCH (n) -[r* (ie, in | true)]- (m) RETURN *
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
AstTreeStorage storage;
auto edge = EDGE_VARIABLE("r", Direction::BOTH);
edge->inner_edge_ = IDENT("ie");

File diff suppressed because it is too large Load Diff

View File

@ -63,7 +63,8 @@ void AssertRows(const std::vector<std::vector<TypedValue>> &datum,
};
void CheckPlansProduce(
size_t expected_plan_count, AstTreeStorage &storage, GraphDbAccessor &dba,
size_t expected_plan_count, AstTreeStorage &storage,
database::GraphDbAccessor &dba,
std::function<void(const std::vector<std::vector<TypedValue>> &)> check) {
auto symbol_table = MakeSymbolTable(*storage.query());
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
@ -82,8 +83,8 @@ void CheckPlansProduce(
}
TEST(TestVariableStartPlanner, MatchReturn) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Make a graph (v1) -[:r]-> (v2)
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -102,8 +103,8 @@ TEST(TestVariableStartPlanner, MatchReturn) {
}
TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3)
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -138,8 +139,8 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
}
TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3)
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -164,8 +165,8 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
}
TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Graph (v1) -[:r]-> (v2)
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -189,8 +190,8 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
}
TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Graph (v1) -[:r]-> (v2)
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -212,8 +213,8 @@ TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
}
TEST(TestVariableStartPlanner, MatchVariableExpand) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
// Graph (v1) -[:r1]-> (v2) -[:r2]-> (v3)
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -235,8 +236,8 @@ TEST(TestVariableStartPlanner, MatchVariableExpand) {
}
TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto id = dba.Property("id");
// Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3})
auto v1 = dba.InsertVertex();
@ -262,8 +263,8 @@ TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
}
TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto id = dba.Property("id");
// Graph (v1 {id:1}) -[:r1]-> (v2) -[:r2]-> (v3)
auto v1 = dba.InsertVertex();
@ -288,8 +289,8 @@ TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
}
TEST(TestVariableStartPlanner, MatchBfs) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto id = dba.Property("id");
// Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3})
auto v1 = dba.InsertVertex();
@ -305,7 +306,7 @@ TEST(TestVariableStartPlanner, MatchBfs) {
AstTreeStorage storage;
auto *bfs = storage.Create<query::EdgeAtom>(
IDENT("r"), EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
std::vector<GraphDbTypes::EdgeType>{});
std::vector<database::EdgeType>{});
bfs->inner_edge_ = IDENT("r");
bfs->inner_node_ = IDENT("n");
bfs->filter_expression_ = NEQ(PROPERTY_LOOKUP("n", id), LITERAL(3));

View File

@ -11,8 +11,8 @@
#include "storage/vertex_accessor.hpp"
TEST(RecordAccessor, Properties) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto vertex = dba.InsertVertex();
auto &properties = vertex.Properties();
@ -33,8 +33,8 @@ TEST(RecordAccessor, Properties) {
}
TEST(RecordAccessor, DbAccessor) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto vertex = dba.InsertVertex();
const auto &const_vertex_dba = vertex.db_accessor();
@ -44,8 +44,8 @@ TEST(RecordAccessor, DbAccessor) {
}
TEST(RecordAccessor, RecordEquality) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
@ -59,11 +59,11 @@ TEST(RecordAccessor, RecordEquality) {
}
TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) {
GraphDb db;
database::SingleNode db;
// test both Switches work on new record
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
v1.SwitchOld();
v1.SwitchNew();
@ -72,7 +72,7 @@ TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) {
// test both Switches work on existing record
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto v1 = *dba.Vertices(false).begin();
v1.SwitchOld();
v1.SwitchNew();
@ -80,7 +80,7 @@ TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) {
// ensure switch exposes the right data
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto label = dba.Label("label");
auto v1 = *dba.Vertices(false).begin();
@ -95,20 +95,20 @@ TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) {
}
TEST(RecordAccessor, Reconstruct) {
GraphDb db;
auto label = GraphDbAccessor(db).Label("label");
database::SingleNode db;
auto label = database::GraphDbAccessor(db).Label("label");
{
// we must operate on an old vertex
// because otherwise we only have new
// so create a vertex and commit it
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
dba.InsertVertex();
dba.Commit();
}
// ensure we don't have label set
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto v1 = *dba.Vertices(false).begin();
v1.SwitchNew();
EXPECT_FALSE(v1.has_label(label));
@ -129,15 +129,15 @@ TEST(RecordAccessor, Reconstruct) {
}
TEST(RecordAccessor, VertexLabels) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto &labels = v1.labels();
EXPECT_EQ(v1.labels().size(), 0);
GraphDbTypes::Label l1 = dba.Label("label1");
GraphDbTypes::Label l2 = dba.Label("label2");
database::Label l1 = dba.Label("label1");
database::Label l2 = dba.Label("label2");
// adding labels
EXPECT_FALSE(v1.has_label(l1));
@ -157,7 +157,7 @@ TEST(RecordAccessor, VertexLabels) {
EXPECT_EQ(labels.size(), 2);
// removing labels
GraphDbTypes::Label l3 = dba.Label("label3");
database::Label l3 = dba.Label("label3");
EXPECT_EQ(v1.remove_label(l3), 0);
EXPECT_EQ(labels.size(), 2);
@ -170,13 +170,13 @@ TEST(RecordAccessor, VertexLabels) {
}
TEST(RecordAccessor, EdgeType) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
GraphDbTypes::EdgeType likes = dba.EdgeType("likes");
GraphDbTypes::EdgeType hates = dba.EdgeType("hates");
database::EdgeType likes = dba.EdgeType("likes");
database::EdgeType hates = dba.EdgeType("hates");
auto edge = dba.InsertEdge(v1, v2, likes);
EXPECT_EQ(edge.EdgeType(), likes);
@ -184,8 +184,8 @@ TEST(RecordAccessor, EdgeType) {
}
TEST(RecordAccessor, EdgeIsCycle) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
auto likes = dba.EdgeType("edge_type");
@ -197,8 +197,8 @@ TEST(RecordAccessor, EdgeIsCycle) {
}
TEST(RecordAccessor, VertexEdgeConnections) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
auto edge = dba.InsertEdge(v1, v2, dba.EdgeType("likes"));
@ -229,8 +229,8 @@ TEST(RecordAccessor, VertexEdgeConnections) {
}
TEST(RecordAccessor, VertexEdgeConnectionsWithExistingVertex) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
auto v3 = dba.InsertVertex();
@ -264,8 +264,8 @@ TEST(RecordAccessor, VertexEdgeConnectionsWithExistingVertex) {
}
TEST(RecordAccessor, VertexEdgeConnectionsWithEdgeType) {
GraphDb db;
GraphDbAccessor dba(db);
database::SingleNode db;
database::GraphDbAccessor dba(db);
auto v1 = dba.InsertVertex();
auto v2 = dba.InsertVertex();
auto a = dba.EdgeType("a");
@ -280,9 +280,9 @@ TEST(RecordAccessor, VertexEdgeConnectionsWithEdgeType) {
TEST_EDGE_ITERABLE(v1.in(), {eb_1, eb_2});
TEST_EDGE_ITERABLE(v2.in(), {ea, ec});
std::vector<GraphDbTypes::EdgeType> edges_a{a};
std::vector<GraphDbTypes::EdgeType> edges_b{b};
std::vector<GraphDbTypes::EdgeType> edges_ac{a, c};
std::vector<database::EdgeType> edges_a{a};
std::vector<database::EdgeType> edges_b{b};
std::vector<database::EdgeType> edges_ac{a, c};
TEST_EDGE_ITERABLE(v1.in(&edges_a));
TEST_EDGE_ITERABLE(v1.in(&edges_b), {eb_1, eb_2});
TEST_EDGE_ITERABLE(v1.out(&edges_a), {ea});

View File

@ -17,9 +17,9 @@ int Count(TIterable &collection) {
TEST(SkipListSuffix, EmptyRange) {
SkipList<int> V;
auto access = V.access();
auto r1 = IndexUtils::SkipListSuffix<typename SkipList<int>::Iterator, int,
SkipList<int>>(access.begin(),
std::move(access));
auto r1 = database::index::SkipListSuffix<typename SkipList<int>::Iterator,
int, SkipList<int>>(
access.begin(), std::move(access));
EXPECT_EQ(Count(r1), 0);
}
@ -29,9 +29,9 @@ TEST(SkipListSuffix, NonEmptyRange) {
access.insert(1);
access.insert(5);
access.insert(3);
auto r1 = IndexUtils::SkipListSuffix<typename SkipList<int>::Iterator, int,
SkipList<int>>(access.begin(),
std::move(access));
auto r1 = database::index::SkipListSuffix<typename SkipList<int>::Iterator,
int, SkipList<int>>(
access.begin(), std::move(access));
EXPECT_EQ(Count(r1), 3);
auto iter = r1.begin();
EXPECT_EQ(*iter, 1);

View File

@ -5,114 +5,114 @@
#include "database/state_delta.hpp"
TEST(StateDelta, CreateVertex) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta = database::StateDelta::CreateVertex(dba.transaction_id(), gid0);
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex = dba.FindVertex(gid0, false);
EXPECT_TRUE(vertex);
}
}
TEST(StateDelta, RemoveVertex) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
dba.InsertVertex(gid0);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta = database::StateDelta::RemoveVertex(dba.transaction_id(), gid0);
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex = dba.FindVertex(gid0, false);
EXPECT_FALSE(vertex);
}
}
TEST(StateDelta, CreateEdge) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
auto gid1 = generator.Next();
auto gid2 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
dba.InsertVertex(gid0);
dba.InsertVertex(gid1);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta = database::StateDelta::CreateEdge(dba.transaction_id(), gid2,
gid0, gid1, "edge");
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto edge = dba.FindEdge(gid2, false);
EXPECT_TRUE(edge);
}
}
TEST(StateDelta, RemoveEdge) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
auto gid1 = generator.Next();
auto gid2 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto v0 = dba.InsertVertex(gid0);
auto v1 = dba.InsertVertex(gid1);
dba.InsertEdge(v0, v1, dba.EdgeType("edge"), gid2);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta = database::StateDelta::RemoveEdge(dba.transaction_id(), gid2);
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto edge = dba.FindEdge(gid2, false);
EXPECT_FALSE(edge);
}
}
TEST(StateDelta, AddLabel) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
dba.InsertVertex(gid0);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta =
database::StateDelta::AddLabel(dba.transaction_id(), gid0, "label");
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex = dba.FindVertex(gid0, false);
EXPECT_TRUE(vertex);
auto labels = vertex->labels();
@ -122,24 +122,24 @@ TEST(StateDelta, AddLabel) {
}
TEST(StateDelta, RemoveLabel) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex = dba.InsertVertex(gid0);
vertex.add_label(dba.Label("label"));
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta =
database::StateDelta::RemoveLabel(dba.transaction_id(), gid0, "label");
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex = dba.FindVertex(gid0, false);
EXPECT_TRUE(vertex);
auto labels = vertex->labels();
@ -148,23 +148,23 @@ TEST(StateDelta, RemoveLabel) {
}
TEST(StateDelta, SetPropertyVertex) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
dba.InsertVertex(gid0);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta = database::StateDelta::PropsSetVertex(
dba.transaction_id(), gid0, "property", PropertyValue(2212));
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto vertex = dba.FindVertex(gid0, false);
EXPECT_TRUE(vertex);
auto prop = vertex->PropsAt(dba.Property("property"));
@ -173,27 +173,27 @@ TEST(StateDelta, SetPropertyVertex) {
}
TEST(StateDelta, SetPropertyEdge) {
GraphDb db;
database::SingleNode db;
gid::Generator generator(0);
auto gid0 = generator.Next();
auto gid1 = generator.Next();
auto gid2 = generator.Next();
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto v0 = dba.InsertVertex(gid0);
auto v1 = dba.InsertVertex(gid1);
dba.InsertEdge(v0, v1, dba.EdgeType("edge"), gid2);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto delta = database::StateDelta::PropsSetEdge(
dba.transaction_id(), gid2, "property", PropertyValue(2212));
delta.Apply(dba);
dba.Commit();
}
{
GraphDbAccessor dba(db);
database::GraphDbAccessor dba(db);
auto edge = dba.FindEdge(gid2, false);
EXPECT_TRUE(edge);
auto prop = edge->PropsAt(dba.Property("property"));

View File

@ -18,8 +18,8 @@ using query::TypedValueException;
class AllTypesFixture : public testing::Test {
protected:
std::vector<TypedValue> values_;
GraphDb db_;
GraphDbAccessor dba_{db_};
database::SingleNode db_;
database::GraphDbAccessor dba_{db_};
void SetUp() override {
values_.emplace_back(TypedValue::Null);

View File

@ -23,11 +23,11 @@ class RecoveryTest : public ::testing::Test {
durability::Recover(durability_dir, db_);
}
GraphDb db_;
database::SingleNode db_;
};
TEST_F(RecoveryTest, TestVerticesRecovered) {
GraphDbAccessor dba(db_);
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)) {
@ -37,7 +37,7 @@ TEST_F(RecoveryTest, TestVerticesRecovered) {
}
TEST_F(RecoveryTest, TestPropertyNull) {
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
bool found = false;
for (const auto &vertex : dba.Vertices(dba.Label("Comment"), false)) {
auto id_prop = query::TypedValue(vertex.PropsAt(dba.Property("id")));
@ -54,7 +54,7 @@ TEST_F(RecoveryTest, TestPropertyNull) {
}
TEST_F(RecoveryTest, TestEdgesRecovered) {
GraphDbAccessor dba(db_);
database::GraphDbAccessor dba(db_);
EXPECT_EQ(dba.EdgesCount(), 5);
for (const auto &edge : dba.Edges(false)) {
EXPECT_TRUE(edge.EdgeType() == dba.EdgeType("POSTED_ON"));