Add distributed Id Cypher function

Reviewers: msantl, vkasljevic, teon.banek, ipaljak

Reviewed By: msantl, teon.banek

Subscribers: pullbot, teon.banek

Differential Revision: https://phabricator.memgraph.io/D1477
This commit is contained in:
Marko Budiselic 2018-08-28 16:28:35 +02:00
parent e80c49f856
commit d9153921b8
40 changed files with 832 additions and 632 deletions

View File

@ -92,8 +92,7 @@ class DistributedRecordAccessor final {
if (address.is_local()) { if (address.is_local()) {
return address.local()->update(dba.transaction()); return address.local()->update(dba.transaction());
} }
return data_manager_->FindNew<TRecord>(dba.transaction_id(), return data_manager_->FindNew<TRecord>(dba.transaction_id(), address.gid());
address.gid());
} }
void ProcessDelta(const RecordAccessor<TRecord> &record_accessor, void ProcessDelta(const RecordAccessor<TRecord> &record_accessor,
@ -122,6 +121,25 @@ class DistributedRecordAccessor final {
throw utils::LockTimeoutException("Lock timeout on remote worker"); throw utils::LockTimeoutException("Lock timeout on remote worker");
} }
} }
int64_t CypherId(const RecordAccessor<TRecord> &record_accessor) {
auto &dba = record_accessor.db_accessor();
const auto &address = record_accessor.address();
if (record_accessor.is_local()) return address.local()->cypher_id();
// Fetch data from the cache.
//
// NOTE: This part is executed when we need to migrate
// a vertex and it has edges that don't belong to it. A machine that owns
// the vertex still need to figure out what is the cypher_id for each
// remote edge because the machine has to initiate remote edge creation
// and for that call it has to know the remote cypher_ids.
// TODO (buda): If we save cypher_id similar/next to edge_type we would save
// a network call.
return data_manager_
->Find<TRecord>(dba.transaction().id_, address.worker_id(),
address.gid())
.cypher_id;
}
}; };
class DistributedEdgeAccessor final : public ::RecordAccessor<Edge>::Impl { class DistributedEdgeAccessor final : public ::RecordAccessor<Edge>::Impl {
@ -150,6 +168,10 @@ class DistributedEdgeAccessor final : public ::RecordAccessor<Edge>::Impl {
const database::StateDelta &delta) override { const database::StateDelta &delta) override {
return distributed_accessor_.ProcessDelta(ra, delta); return distributed_accessor_.ProcessDelta(ra, delta);
} }
int64_t CypherId(const RecordAccessor<Edge> &ra) override {
return distributed_accessor_.CypherId(ra);
}
}; };
class DistributedVertexAccessor final : public ::VertexAccessor::Impl { class DistributedVertexAccessor final : public ::VertexAccessor::Impl {
@ -216,6 +238,10 @@ class DistributedVertexAccessor final : public ::VertexAccessor::Impl {
if (!va.is_local()) distributed_accessor_.SendDelta(va, delta); if (!va.is_local()) distributed_accessor_.SendDelta(va, delta);
} }
int64_t CypherId(const RecordAccessor<Vertex> &ra) override {
return distributed_accessor_.CypherId(ra);
}
}; };
////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////
@ -297,14 +323,17 @@ class DistributedAccessor : public GraphDbAccessor {
return GraphDbAccessor::InsertEdgeOnFrom(from, to, edge_type, return GraphDbAccessor::InsertEdgeOnFrom(from, to, edge_type,
requested_gid, cypher_id); requested_gid, cypher_id);
} }
auto edge_address = auto created_edge_info = updates_clients_->CreateEdge(
updates_clients_->CreateEdge(transaction_id(), *from, *to, edge_type); transaction_id(), *from, *to, edge_type, cypher_id);
auto edge_address = created_edge_info.edge_address;
auto *from_updated = auto *from_updated =
data_manager_->FindNew<Vertex>(transaction_id(), from->gid()); data_manager_->FindNew<Vertex>(transaction_id(), from->gid());
// Create an Edge and insert it into the Cache so we see it locally. // Create an Edge and insert it into the Cache so we see it locally.
data_manager_->Emplace<Edge>( data_manager_->Emplace<Edge>(
transaction_id(), edge_address.gid(), nullptr, transaction_id(), edge_address.gid(),
std::make_unique<Edge>(from->address(), to->address(), edge_type)); distributed::CachedRecordData<Edge>(
created_edge_info.cypher_id, nullptr,
std::make_unique<Edge>(from->address(), to->address(), edge_type)));
from_updated->out_.emplace( from_updated->out_.emplace(
db().storage().LocalizedAddressIfPossible(to->address()), edge_address, db().storage().LocalizedAddressIfPossible(to->address()), edge_address,
edge_type); edge_type);
@ -810,8 +839,8 @@ distributed::IndexRpcClients &Master::index_rpc_clients() {
VertexAccessor InsertVertexIntoRemote( VertexAccessor InsertVertexIntoRemote(
GraphDbAccessor *dba, int worker_id, GraphDbAccessor *dba, int worker_id,
const std::vector<storage::Label> &labels, const std::vector<storage::Label> &labels,
const std::unordered_map<storage::Property, query::TypedValue> const std::unordered_map<storage::Property, query::TypedValue> &properties,
&properties) { std::experimental::optional<int64_t> cypher_id) {
// TODO: Replace this with virtual call or some other mechanism. // TODO: Replace this with virtual call or some other mechanism.
auto *distributed_db = auto *distributed_db =
dynamic_cast<database::DistributedGraphDb *>(&dba->db()); dynamic_cast<database::DistributedGraphDb *>(&dba->db());
@ -821,13 +850,16 @@ VertexAccessor InsertVertexIntoRemote(
auto *updates_clients = &distributed_db->updates_clients(); auto *updates_clients = &distributed_db->updates_clients();
auto *data_manager = &distributed_db->data_manager(); auto *data_manager = &distributed_db->data_manager();
CHECK(updates_clients && data_manager); CHECK(updates_clients && data_manager);
gid::Gid gid = updates_clients->CreateVertex(worker_id, dba->transaction_id(), auto created_vertex_info = updates_clients->CreateVertex(
labels, properties); worker_id, dba->transaction_id(), labels, properties, cypher_id);
auto vertex = std::make_unique<Vertex>(); auto vertex = std::make_unique<Vertex>();
vertex->labels_ = labels; vertex->labels_ = labels;
for (auto &kv : properties) vertex->properties_.set(kv.first, kv.second); for (auto &kv : properties) vertex->properties_.set(kv.first, kv.second);
data_manager->Emplace<Vertex>(dba->transaction_id(), gid, nullptr, std::move(vertex)); data_manager->Emplace<Vertex>(
return VertexAccessor({gid, worker_id}, *dba); dba->transaction_id(), created_vertex_info.gid,
distributed::CachedRecordData<Vertex>(created_vertex_info.cypher_id,
nullptr, std::move(vertex)));
return VertexAccessor({created_vertex_info.gid, worker_id}, *dba);
} }
////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////

View File

@ -132,6 +132,7 @@ class Worker final : public DistributedGraphDb {
VertexAccessor InsertVertexIntoRemote( VertexAccessor InsertVertexIntoRemote(
GraphDbAccessor *dba, int worker_id, GraphDbAccessor *dba, int worker_id,
const std::vector<storage::Label> &labels, const std::vector<storage::Label> &labels,
const std::unordered_map<storage::Property, query::TypedValue> &properties); const std::unordered_map<storage::Property, query::TypedValue> &properties,
std::experimental::optional<int64_t> cypher_id);
} // namespace database } // namespace database

View File

@ -54,6 +54,10 @@ class SingleNodeRecordAccessor final {
CHECK(record_accessor.is_local()); CHECK(record_accessor.is_local());
record_accessor.db_accessor().wal().Emplace(delta); record_accessor.db_accessor().wal().Emplace(delta);
} }
int64_t CypherId(const RecordAccessor<TRecord> &record_accessor) {
return record_accessor.address().local()->cypher_id();
}
}; };
class VertexAccessorImpl final : public ::VertexAccessor::Impl { class VertexAccessorImpl final : public ::VertexAccessor::Impl {
@ -109,6 +113,10 @@ class VertexAccessorImpl final : public ::VertexAccessor::Impl {
dba.wal().Emplace(delta); dba.wal().Emplace(delta);
} }
} }
int64_t CypherId(const RecordAccessor<Vertex> &ra) override {
return accessor_.CypherId(ra);
}
}; };
class EdgeAccessorImpl final : public ::RecordAccessor<Edge>::Impl { class EdgeAccessorImpl final : public ::RecordAccessor<Edge>::Impl {
@ -133,6 +141,10 @@ class EdgeAccessorImpl final : public ::RecordAccessor<Edge>::Impl {
const database::StateDelta &delta) override { const database::StateDelta &delta) override {
return accessor_.ProcessDelta(ra, delta); return accessor_.ProcessDelta(ra, delta);
} }
int64_t CypherId(const RecordAccessor<Edge> &ra) override {
return accessor_.CypherId(ra);
}
}; };
class SingleNodeAccessor : public GraphDbAccessor { class SingleNodeAccessor : public GraphDbAccessor {

View File

@ -359,7 +359,7 @@ storage::EdgeAddress GraphDbAccessor::InsertEdgeOnFrom(
// `CREATE_EDGE`, but always have it split into 3 parts (edge insertion, // `CREATE_EDGE`, but always have it split into 3 parts (edge insertion,
// in/out modification). // in/out modification).
wal().Emplace(database::StateDelta::CreateEdge( wal().Emplace(database::StateDelta::CreateEdge(
transaction_.id_, edge_accessor.gid(), edge_accessor.cypher_id(), transaction_.id_, edge_accessor.gid(), edge_accessor.CypherId(),
from->gid(), to->gid(), edge_type, EdgeTypeName(edge_type))); from->gid(), to->gid(), edge_type, EdgeTypeName(edge_type)));
from_updated->out_.emplace( from_updated->out_.emplace(
@ -371,9 +371,9 @@ storage::EdgeAddress GraphDbAccessor::InsertEdgeOnFrom(
void GraphDbAccessor::InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to, void GraphDbAccessor::InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to,
const storage::EdgeType &edge_type, const storage::EdgeType &edge_type,
const storage::EdgeAddress &edge_address) { const storage::EdgeAddress &edge_address) {
// ensure that the "to" accessor has the latest version (Switch new) // Ensure that the "to" accessor has the latest version (switch new).
// WARNING: must do that after the above "from->update()" for cases when // WARNING: Must do that after the above "from->update()" for cases when
// we are creating a cycle and "from" and "to" are the same vlist // we are creating a cycle and "from" and "to" are the same vlist.
to->SwitchNew(); to->SwitchNew();
auto *to_updated = &to->update(); auto *to_updated = &to->update();
to_updated->in_.emplace( to_updated->in_.emplace(

View File

@ -1,12 +1,14 @@
/// @file
#pragma once #pragma once
#include <experimental/optional> #include <experimental/optional>
#include <string> #include <string>
#include <vector> #include <vector>
#include <glog/logging.h>
#include <cppitertools/filter.hpp> #include <cppitertools/filter.hpp>
#include <cppitertools/imap.hpp> #include <cppitertools/imap.hpp>
#include <glog/logging.h>
#include "database/graph_db.hpp" #include "database/graph_db.hpp"
#include "storage/address_types.hpp" #include "storage/address_types.hpp"

View File

@ -82,12 +82,14 @@ std::experimental::optional<VertexAccessor> BfsRpcClients::Pull(
CHECK(res) << "SubcursorPull RPC failed!"; CHECK(res) << "SubcursorPull RPC failed!";
if (!res->vertex) return std::experimental::nullopt; if (!res->vertex) return std::experimental::nullopt;
data_manager_->Emplace<Vertex>(dba->transaction_id(), data_manager_->Emplace<Vertex>(
res->vertex->global_address.gid(), dba->transaction_id(), res->vertex->global_address.gid(),
std::move(res->vertex->old_element_output), distributed::CachedRecordData<Vertex>(
std::move(res->vertex->new_element_output)); res->cypher_id, std::move(res->vertex->old_element_output),
std::move(res->vertex->new_element_output)));
return VertexAccessor(res->vertex->global_address, *dba); return VertexAccessor(res->vertex->global_address, *dba);
} }
bool BfsRpcClients::ExpandLevel( bool BfsRpcClients::ExpandLevel(
const std::unordered_map<int16_t, int64_t> &subcursor_ids) { const std::unordered_map<int16_t, int64_t> &subcursor_ids) {
auto futures = clients_->ExecuteOnWorkers<bool>( auto futures = clients_->ExecuteOnWorkers<bool>(
@ -140,8 +142,11 @@ PathSegment BuildPathSegment(ReconstructPathRes *res,
distributed::DataManager *data_manager) { distributed::DataManager *data_manager) {
std::vector<EdgeAccessor> edges; std::vector<EdgeAccessor> edges;
for (auto &edge : res->edges) { for (auto &edge : res->edges) {
data_manager->Emplace<Edge>(dba->transaction_id(), edge.global_address.gid(), std::move(edge.old_element_output), data_manager->Emplace<Edge>(
std::move(edge.new_element_output)); dba->transaction_id(), edge.global_address.gid(),
distributed::CachedRecordData<Edge>(
edge.cypher_id, std::move(edge.old_element_output),
std::move(edge.new_element_output)));
edges.emplace_back(edge.global_address, *dba); edges.emplace_back(edge.global_address, *dba);
} }

View File

@ -54,7 +54,8 @@ cpp<#
cpp<#)) cpp<#))
(lcp:define-struct (serialized-graph-element t-element) () (lcp:define-struct (serialized-graph-element t-element) ()
((global-address "storage::Address<mvcc::VersionList<TElement>>" ((cypher-id :int64_t)
(global-address "storage::Address<mvcc::VersionList<TElement>>"
:capnp-type "Storage.Address") :capnp-type "Storage.Address")
(old-element-input "TElement *" (old-element-input "TElement *"
:capnp-type '((null "Void") (vertex "Dis.Vertex") (edge "Dis.Edge")) :capnp-type '((null "Void") (vertex "Dis.Vertex") (edge "Dis.Edge"))
@ -67,10 +68,12 @@ cpp<#
(worker-id :int16_t :capnp-save :dont-save)) (worker-id :int16_t :capnp-save :dont-save))
(:public (:public
#>cpp #>cpp
SerializedGraphElement(storage::Address<mvcc::VersionList<TElement>> global_address, SerializedGraphElement(int64_t cypher_id,
storage::Address<mvcc::VersionList<TElement>> global_address,
TElement *old_element_input, TElement *new_element_input, TElement *old_element_input, TElement *new_element_input,
int16_t worker_id) int16_t worker_id)
: global_address(global_address), : cypher_id(cypher_id),
global_address(global_address),
old_element_input(old_element_input), old_element_input(old_element_input),
old_element_output(nullptr), old_element_output(nullptr),
new_element_input(new_element_input), new_element_input(new_element_input),
@ -81,8 +84,10 @@ cpp<#
} }
SerializedGraphElement(const RecordAccessor<TElement> &accessor, int16_t worker_id) SerializedGraphElement(const RecordAccessor<TElement> &accessor, int16_t worker_id)
: SerializedGraphElement(accessor.GlobalAddress(), accessor.GetOld(), : SerializedGraphElement(accessor.CypherId(),
accessor.GetNew(), worker_id) {} accessor.GlobalAddress(),
accessor.GetOld(), accessor.GetNew(),
worker_id) {}
SerializedGraphElement() {} SerializedGraphElement() {}
cpp<#) cpp<#)
@ -159,10 +164,13 @@ cpp<#
(lcp:define-rpc subcursor-pull (lcp:define-rpc subcursor-pull
(:request ((member :int64_t))) (:request ((member :int64_t)))
(:response ((vertex "std::experimental::optional<SerializedVertex>" :initarg :move (:response
:capnp-type "Utils.Optional(SerializedGraphElement)" ((cypher-id :int64_t)
:capnp-save (lcp:capnp-save-optional "capnp::SerializedGraphElement" "SerializedVertex") (vertex "std::experimental::optional<SerializedVertex>" :initarg :move
:capnp-load (lcp:capnp-load-optional "capnp::SerializedGraphElement" "SerializedVertex"))))) :capnp-type "Utils.Optional(SerializedGraphElement)"
:capnp-save (lcp:capnp-save-optional "capnp::SerializedGraphElement" "SerializedVertex")
:capnp-load (lcp:capnp-load-optional "capnp::SerializedGraphElement" "SerializedVertex")))))
(lcp:define-rpc set-source (lcp:define-rpc set-source
(:request (:request
((subcursor-id :int64_t) ((subcursor-id :int64_t)

View File

@ -1,4 +1,5 @@
/// @file /// @file
#pragma once #pragma once
#include <map> #include <map>
@ -84,7 +85,8 @@ class BfsRpcServer {
res.Save(res_builder); res.Save(res_builder);
return; return;
} }
SubcursorPullRes res(SerializedVertex(*vertex, db_->WorkerId())); SubcursorPullRes res(vertex->CypherId(),
SerializedVertex(*vertex, db_->WorkerId()));
res.Save(res_builder); res.Save(res_builder);
}); });
@ -111,8 +113,8 @@ class BfsRpcServer {
} else { } else {
LOG(FATAL) << "`edge` or `vertex` should be set in ReconstructPathReq"; LOG(FATAL) << "`edge` or `vertex` should be set in ReconstructPathReq";
} }
ReconstructPathRes res(result.edges, result.next_vertex, ReconstructPathRes res(result.edges, result.next_vertex, result.next_edge,
result.next_edge, db_->WorkerId()); db_->WorkerId());
res.Save(res_builder); res.Save(res_builder);
}); });

View File

@ -1,99 +0,0 @@
#include "glog/logging.h"
#include "database/storage.hpp"
#include "distributed/cache.hpp"
#include "storage/edge.hpp"
#include "storage/vertex.hpp"
namespace distributed {
template <typename TRecord>
TRecord *Cache<TRecord>::FindNew(gid::Gid gid) {
std::lock_guard<std::mutex> guard{lock_};
auto found = cache_.find(gid);
DCHECK(found != cache_.end())
<< "FindNew for uninitialized remote Vertex/Edge";
auto &pair = found->second;
if (!pair.second) {
pair.second = std::unique_ptr<TRecord>(pair.first->CloneData());
}
return pair.second.get();
}
template <typename TRecord>
void Cache<TRecord>::FindSetOldNew(tx::TransactionId tx_id, int worker_id,
gid::Gid gid, TRecord *&old_record,
TRecord *&new_record) {
{
std::lock_guard<std::mutex> guard(lock_);
auto found = cache_.find(gid);
if (found != cache_.end()) {
old_record = found->second.first.get();
new_record = found->second.second.get();
return;
}
}
auto remote = data_clients_.RemoteElement<TRecord>(worker_id, tx_id, gid);
LocalizeAddresses(*remote);
// This logic is a bit strange because we need to make sure that someone
// else didn't get a response and updated the cache before we did and we
// need a lock for that, but we also need to check if we can now return
// that result - otherwise we could get incosistent results for remote
// FindSetOldNew
std::lock_guard<std::mutex> guard(lock_);
auto it_pair = cache_.emplace(
gid, std::make_pair<rec_uptr, rec_uptr>(std::move(remote), nullptr));
old_record = it_pair.first->second.first.get();
new_record = it_pair.first->second.second.get();
}
template <typename TRecord>
void Cache<TRecord>::emplace(gid::Gid gid, rec_uptr old_record,
rec_uptr new_record) {
if (old_record) LocalizeAddresses(*old_record);
if (new_record) LocalizeAddresses(*new_record);
std::lock_guard<std::mutex> guard{lock_};
// We can't replace existing data because some accessors might be using
// it.
// TODO - consider if it's necessary and OK to copy just the data content.
auto found = cache_.find(gid);
if (found != cache_.end())
return;
else
cache_[gid] = std::make_pair(std::move(old_record), std::move(new_record));
}
template <typename TRecord>
void Cache<TRecord>::ClearCache() {
std::lock_guard<std::mutex> guard{lock_};
cache_.clear();
}
template <>
void Cache<Vertex>::LocalizeAddresses(Vertex &vertex) {
auto localize_edges = [this](auto &edges) {
for (auto &element : edges) {
element.vertex = storage_.LocalizedAddressIfPossible(element.vertex);
element.edge = storage_.LocalizedAddressIfPossible(element.edge);
}
};
localize_edges(vertex.in_.storage());
localize_edges(vertex.out_.storage());
}
template <>
void Cache<Edge>::LocalizeAddresses(Edge &edge) {
edge.from_ = storage_.LocalizedAddressIfPossible(edge.from_);
edge.to_ = storage_.LocalizedAddressIfPossible(edge.to_);
}
template class Cache<Vertex>;
template class Cache<Edge>;
} // namespace distributed

View File

@ -1,3 +1,5 @@
/// @file
#pragma once #pragma once
#include <mutex> #include <mutex>
@ -40,21 +42,13 @@ class Cache {
return cache_.emplace(std::forward<TKey>(key), std::forward<TValue>(value)); return cache_.emplace(std::forward<TKey>(key), std::forward<TValue>(value));
} }
void erase(const TKey &key) { void erase(const TKey &key) { cache_.erase(key); }
cache_.erase(key);
}
Iterator end() {
return cache_.end();
}
bool contains(const TKey &key) { Iterator end() { return cache_.end(); }
return find(key) != end();
}
void clear() { bool contains(const TKey &key) { return find(key) != end(); }
cache_.clear();
} void clear() { cache_.clear(); }
private: private:
std::unordered_map<TKey, TValue> cache_; std::unordered_map<TKey, TValue> cache_;

View File

@ -1,3 +1,5 @@
/// @file
#pragma once #pragma once
#include "data_structures/concurrent/concurrent_map.hpp" #include "data_structures/concurrent/concurrent_map.hpp"
@ -11,12 +13,26 @@ class Edge;
namespace distributed { namespace distributed {
/// A wrapper for cached vertex/edge from other machines in the distributed
/// system.
///
/// @tparam TRecord Vertex or Edge
template <typename TRecord>
struct CachedRecordData {
CachedRecordData(int64_t cypher_id, std::unique_ptr<TRecord> old_record,
std::unique_ptr<TRecord> new_record)
: cypher_id(cypher_id),
old_record(std::move(old_record)),
new_record(std::move(new_record)) {}
int64_t cypher_id;
std::unique_ptr<TRecord> old_record;
std::unique_ptr<TRecord> new_record;
};
/// Handles remote data caches for edges and vertices, per transaction. /// Handles remote data caches for edges and vertices, per transaction.
class DataManager { class DataManager {
template <typename TRecord> template <typename TRecord>
using CacheG = using CacheG = Cache<gid::Gid, CachedRecordData<TRecord>>;
Cache<gid::Gid,
std::pair<std::unique_ptr<TRecord>, std::unique_ptr<TRecord>>>;
template <typename TRecord> template <typename TRecord>
using CacheT = ConcurrentMap<tx::TransactionId, CacheG<TRecord>>; using CacheT = ConcurrentMap<tx::TransactionId, CacheG<TRecord>>;
@ -35,12 +51,11 @@ class DataManager {
DCHECK(found != cache.end()) DCHECK(found != cache.end())
<< "FindNew is called on uninitialized remote Vertex/Edge"; << "FindNew is called on uninitialized remote Vertex/Edge";
auto &pair = found->second; auto &data = found->second;
if (!pair.second) { if (!data.new_record) {
pair.second = std::unique_ptr<TRecord>(pair.first->CloneData()); data.new_record = std::unique_ptr<TRecord>(data.old_record->CloneData());
} }
return data.new_record.get();
return pair.second.get();
} }
/// For the Vertex/Edge with the given global ID, looks for the data visible /// For the Vertex/Edge with the given global ID, looks for the data visible
@ -57,14 +72,14 @@ class DataManager {
std::lock_guard<std::mutex> guard(lock); std::lock_guard<std::mutex> guard(lock);
auto found = cache.find(gid); auto found = cache.find(gid);
if (found != cache.end()) { if (found != cache.end()) {
*old_record = found->second.first.get(); *old_record = found->second.old_record.get();
*new_record = found->second.second.get(); *new_record = found->second.new_record.get();
return; return;
} }
} }
auto remote = data_clients_.RemoteElement<TRecord>(worker_id, tx_id, gid); auto remote = data_clients_.RemoteElement<TRecord>(worker_id, tx_id, gid);
LocalizeAddresses(*remote); LocalizeAddresses(*remote.record_ptr);
// This logic is a bit strange because we need to make sure that someone // This logic is a bit strange because we need to make sure that someone
// else didn't get a response and updated the cache before we did and we // else didn't get a response and updated the cache before we did and we
@ -72,21 +87,45 @@ class DataManager {
// that result - otherwise we could get incosistent results for remote // that result - otherwise we could get incosistent results for remote
// FindSetOldNew // FindSetOldNew
std::lock_guard<std::mutex> guard(lock); std::lock_guard<std::mutex> guard(lock);
auto it_pair = cache.emplace(std::move(gid), auto it_pair = cache.emplace(
std::make_pair(std::move(remote), nullptr)); std::move(gid),
CachedRecordData<TRecord>(remote.cypher_id,
std::move(remote.record_ptr), nullptr));
*old_record = it_pair.first->second.first.get(); *old_record = it_pair.first->second.old_record.get();
*new_record = it_pair.first->second.second.get(); *new_record = it_pair.first->second.new_record.get();
}
/// Finds cached element for the given transaction, worker and gid.
///
/// @tparam TRecord Vertex or Edge
template <typename TRecord>
const CachedRecordData<TRecord> &Find(tx::TransactionId tx_id, int worker_id,
gid::Gid gid) {
auto &cache = GetCache<TRecord>(tx_id);
std::unique_lock<std::mutex> guard(GetLock(tx_id));
auto found = cache.find(gid);
if (found != cache.end()) {
return found->second;
} else {
guard.unlock();
auto remote = data_clients_.RemoteElement<TRecord>(worker_id, tx_id, gid);
LocalizeAddresses(*remote.record_ptr);
guard.lock();
return cache
.emplace(std::move(gid),
CachedRecordData<TRecord>(
remote.cypher_id, std::move(remote.record_ptr), nullptr))
.first->second;
}
} }
/// Sets the given records as (new, old) data for the given gid. /// Sets the given records as (new, old) data for the given gid.
template <typename TRecord> template <typename TRecord>
void Emplace(tx::TransactionId tx_id, gid::Gid gid, void Emplace(tx::TransactionId tx_id, gid::Gid gid,
std::unique_ptr<TRecord> old_record, CachedRecordData<TRecord> data) {
std::unique_ptr<TRecord> new_record) { if (data.old_record) LocalizeAddresses(*data.old_record);
if (data.new_record) LocalizeAddresses(*data.new_record);
if (old_record) LocalizeAddresses(*old_record);
if (new_record) LocalizeAddresses(*new_record);
std::lock_guard<std::mutex> guard(GetLock(tx_id)); std::lock_guard<std::mutex> guard(GetLock(tx_id));
// We can't replace existing data because some accessors might be using // We can't replace existing data because some accessors might be using
@ -94,9 +133,7 @@ class DataManager {
// TODO - consider if it's necessary and OK to copy just the data content. // TODO - consider if it's necessary and OK to copy just the data content.
auto &cache = GetCache<TRecord>(tx_id); auto &cache = GetCache<TRecord>(tx_id);
auto found = cache.find(gid); auto found = cache.find(gid);
if (found == cache.end()) if (found == cache.end()) cache.emplace(std::move(gid), std::move(data));
cache.emplace(std::move(gid), std::make_pair(std::move(old_record),
std::move(new_record)));
} }
/// Removes all the caches for a single transaction. /// Removes all the caches for a single transaction.

View File

@ -8,23 +8,25 @@
namespace distributed { namespace distributed {
template <> template <>
std::unique_ptr<Edge> DataRpcClients::RemoteElement(int worker_id, RemoteElementInfo<Edge> DataRpcClients::RemoteElement(int worker_id,
tx::TransactionId tx_id,
gid::Gid gid) {
auto response =
clients_.GetClientPool(worker_id).Call<EdgeRpc>(TxGidPair{tx_id, gid});
CHECK(response) << "EdgeRpc failed";
return std::move(response->edge_output);
}
template <>
std::unique_ptr<Vertex> DataRpcClients::RemoteElement(int worker_id,
tx::TransactionId tx_id, tx::TransactionId tx_id,
gid::Gid gid) { gid::Gid gid) {
auto response =
clients_.GetClientPool(worker_id).Call<EdgeRpc>(TxGidPair{tx_id, gid});
CHECK(response) << "EdgeRpc failed";
return RemoteElementInfo<Edge>(response->cypher_id,
std::move(response->edge_output));
}
template <>
RemoteElementInfo<Vertex> DataRpcClients::RemoteElement(int worker_id,
tx::TransactionId tx_id,
gid::Gid gid) {
auto response = auto response =
clients_.GetClientPool(worker_id).Call<VertexRpc>(TxGidPair{tx_id, gid}); clients_.GetClientPool(worker_id).Call<VertexRpc>(TxGidPair{tx_id, gid});
CHECK(response) << "VertexRpc failed"; CHECK(response) << "VertexRpc failed";
return std::move(response->vertex_output); return RemoteElementInfo<Vertex>(response->cypher_id,
std::move(response->vertex_output));
} }
std::unordered_map<int, int64_t> DataRpcClients::VertexCounts( std::unordered_map<int, int64_t> DataRpcClients::VertexCounts(

View File

@ -1,3 +1,5 @@
/// @file
#pragma once #pragma once
#include <mutex> #include <mutex>
@ -9,16 +11,35 @@
namespace distributed { namespace distributed {
template <typename TRecord>
struct RemoteElementInfo {
RemoteElementInfo() = delete;
RemoteElementInfo(const RemoteElementInfo &) = delete;
// TODO (buda): The default move constructor should be deleted but it seems
// that clang-3.9 doesn't know how to do RVO when this struct is used.
RemoteElementInfo(RemoteElementInfo &&) = default;
RemoteElementInfo &operator=(const RemoteElementInfo &) = delete;
RemoteElementInfo &operator=(RemoteElementInfo &&) = delete;
RemoteElementInfo(int64_t cypher_id, std::unique_ptr<TRecord> record_ptr)
: cypher_id(cypher_id), record_ptr(std::move(record_ptr)) {}
int64_t cypher_id;
std::unique_ptr<TRecord> record_ptr;
};
/// Provides access to other worker's data. /// Provides access to other worker's data.
class DataRpcClients { class DataRpcClients {
public: public:
DataRpcClients(RpcWorkerClients &clients) : clients_(clients) {} DataRpcClients(RpcWorkerClients &clients) : clients_(clients) {}
/// Returns a remote worker's record (vertex/edge) data for the given params. /// Returns a remote worker's record (vertex/edge) data for the given params.
/// That worker must own the vertex/edge for the given id, and that vertex /// That worker must own the vertex/edge for the given id, and that vertex
/// must be visible in given transaction. /// must be visible in given transaction.
template <typename TRecord> template <typename TRecord>
std::unique_ptr<TRecord> RemoteElement(int worker_id, tx::TransactionId tx_id, RemoteElementInfo<TRecord> RemoteElement(int worker_id,
gid::Gid gid); tx::TransactionId tx_id,
gid::Gid gid);
/// Returns (worker_id, vertex_count) for each worker and the number of /// Returns (worker_id, vertex_count) for each worker and the number of
/// vertices on it from the perspective of transaction `tx_id`. /// vertices on it from the perspective of transaction `tx_id`.

View File

@ -28,7 +28,8 @@ cpp<#
(lcp:define-rpc vertex (lcp:define-rpc vertex
(:request ((member "TxGidPair"))) (:request ((member "TxGidPair")))
(:response (:response
((vertex-input "const Vertex *" ((cypher-id :int64_t)
(vertex-input "const Vertex *"
:capnp-type "Dist.Vertex" :capnp-type "Dist.Vertex"
:capnp-save :capnp-save
(lambda (builder member) (lambda (builder member)
@ -48,7 +49,8 @@ cpp<#
(lcp:define-rpc edge (lcp:define-rpc edge
(:request ((member "TxGidPair"))) (:request ((member "TxGidPair")))
(:response (:response
((edge-input "const Edge *" ((cypher-id :int64_t)
(edge-input "const Edge *"
:capnp-type "Dist.Edge" :capnp-type "Dist.Edge"
:capnp-save :capnp-save
(lambda (builder member) (lambda (builder member)

View File

@ -17,7 +17,7 @@ DataRpcServer::DataRpcServer(database::DistributedGraphDb *db,
auto vertex = dba->FindVertex(req_reader.getMember().getGid(), false); auto vertex = dba->FindVertex(req_reader.getMember().getGid(), false);
CHECK(vertex.GetOld()) CHECK(vertex.GetOld())
<< "Old record must exist when sending vertex by RPC"; << "Old record must exist when sending vertex by RPC";
VertexRes response(vertex.GetOld(), db_->WorkerId()); VertexRes response(vertex.CypherId(), vertex.GetOld(), db_->WorkerId());
response.Save(res_builder); response.Save(res_builder);
}); });
@ -26,7 +26,7 @@ DataRpcServer::DataRpcServer(database::DistributedGraphDb *db,
auto dba = db_->Access(req_reader.getMember().getTxId()); auto dba = db_->Access(req_reader.getMember().getTxId());
auto edge = dba->FindEdge(req_reader.getMember().getGid(), false); auto edge = dba->FindEdge(req_reader.getMember().getGid(), false);
CHECK(edge.GetOld()) << "Old record must exist when sending edge by RPC"; CHECK(edge.GetOld()) << "Old record must exist when sending edge by RPC";
EdgeRes response(edge.GetOld(), db_->WorkerId()); EdgeRes response(edge.CypherId(), edge.GetOld(), db_->WorkerId());
response.Save(res_builder); response.Save(res_builder);
}); });

View File

@ -132,7 +132,8 @@ frame (potentially embedded in lists/maps) is too error-prone.")
private: private:
cpp<# cpp<#
(lcp:define-struct (graph-element-data t-record) () (lcp:define-struct (graph-element-data t-record) ()
((global-address "storage::Address<mvcc::VersionList<TRecord>>") ((cypher-id :int64_t)
(global-address "storage::Address<mvcc::VersionList<TRecord>>")
(old-record "std::unique_ptr<TRecord>") (old-record "std::unique_ptr<TRecord>")
(new-record "std::unique_ptr<TRecord>") (new-record "std::unique_ptr<TRecord>")
(element-in-frame (element-in-frame
@ -149,10 +150,11 @@ and the `element_in_frame` reference is used to set the appropriate accessor
to the appropriate value. Not used on side that generates the response.") to the appropriate value. Not used on side that generates the response.")
(:public (:public
#>cpp #>cpp
GraphElementData(storage::Address<mvcc::VersionList<TRecord>> address, GraphElementData(int64_t cypher_id, storage::Address<mvcc::VersionList<TRecord>> address,
std::unique_ptr<TRecord> old_record, std::unique_ptr<TRecord> new_record, std::unique_ptr<TRecord> old_record, std::unique_ptr<TRecord> new_record,
query::TypedValue *element_in_frame) query::TypedValue *element_in_frame)
: global_address(address), : cypher_id(cypher_id),
global_address(address),
old_record(std::move(old_record)), old_record(std::move(old_record)),
new_record(std::move(new_record)), new_record(std::move(new_record)),
element_in_frame(element_in_frame) {} element_in_frame(element_in_frame) {}
@ -194,6 +196,7 @@ to the appropriate value. Not used on side that generates the response.")
const query::TypedValue &value, const query::TypedValue &value,
distributed::capnp::TypedValue::Builder *builder) const { distributed::capnp::TypedValue::Builder *builder) const {
auto save_element = [this](auto accessor, auto *builder) { auto save_element = [this](auto accessor, auto *builder) {
builder->setCypherId(accessor.CypherId());
builder->setAddress(accessor.GlobalAddress().raw()); builder->setAddress(accessor.GlobalAddress().raw());
// If both old and new are null, we need to reconstruct // If both old and new are null, we need to reconstruct
if (!(accessor.GetOld() || accessor.GetNew())) { if (!(accessor.GetOld() || accessor.GetNew())) {
@ -253,6 +256,7 @@ void PullResData::LoadGraphElement(
const distributed::capnp::TypedValue::Reader &reader, const distributed::capnp::TypedValue::Reader &reader,
query::TypedValue *value, distributed::DataManager *data_manager) { query::TypedValue *value, distributed::DataManager *data_manager) {
auto load_vertex = [dba, data_manager](const auto &vertex_reader) { auto load_vertex = [dba, data_manager](const auto &vertex_reader) {
int64_t cypher_id = vertex_reader.getCypherId();
storage::VertexAddress global_address(vertex_reader.getAddress()); storage::VertexAddress global_address(vertex_reader.getAddress());
auto old_record = auto old_record =
vertex_reader.hasOld() vertex_reader.hasOld()
@ -262,11 +266,17 @@ void PullResData::LoadGraphElement(
vertex_reader.hasNew() vertex_reader.hasNew()
? distributed::LoadVertex(vertex_reader.getNew()) ? distributed::LoadVertex(vertex_reader.getNew())
: nullptr; : nullptr;
data_manager->Emplace<Vertex>(dba->transaction_id(), global_address.gid(), data_manager->Emplace<Vertex>(
std::move(old_record), std::move(new_record)); dba->transaction_id(), global_address.gid(),
distributed::CachedRecordData<Vertex>(cypher_id,
std::move(old_record),
std::move(new_record)));
// We don't need to pass cypher_id here because cypher_id is going to be
// fetched from the cache.
return VertexAccessor(global_address, *dba); return VertexAccessor(global_address, *dba);
}; };
auto load_edge = [dba, data_manager](const auto &edge_reader) { auto load_edge = [dba, data_manager](const auto &edge_reader) {
int64_t cypher_id = edge_reader.getCypherId();
storage::EdgeAddress global_address(edge_reader.getAddress()); storage::EdgeAddress global_address(edge_reader.getAddress());
auto old_record = auto old_record =
edge_reader.hasOld() edge_reader.hasOld()
@ -276,8 +286,11 @@ void PullResData::LoadGraphElement(
edge_reader.hasNew() edge_reader.hasNew()
? distributed::LoadEdge(edge_reader.getNew()) ? distributed::LoadEdge(edge_reader.getNew())
: nullptr; : nullptr;
data_manager->Emplace<Edge>(dba->transaction_id(), global_address.gid(), data_manager->Emplace<Edge>(
std::move(old_record), std::move(new_record)); dba->transaction_id(), global_address.gid(),
distributed::CachedRecordData<Edge>(cypher_id,
std::move(old_record),
std::move(new_record)));
return EdgeAccessor(global_address, *dba); return EdgeAccessor(global_address, *dba);
}; };
switch (reader.which()) { switch (reader.which()) {

View File

@ -53,15 +53,17 @@ struct TypedValue {
} }
struct VertexAccessor { struct VertexAccessor {
address @0 :UInt64; cypherId @0 :Int64;
old @1 :Vertex; address @1 :UInt64;
new @2: Vertex; old @2 :Vertex;
new @3 :Vertex;
} }
struct EdgeAccessor { struct EdgeAccessor {
address @0 :UInt64; cypherId @0 :Int64;
old @1 :Edge; address @1 :UInt64;
new @2: Edge; old @2 :Edge;
new @3 :Edge;
} }
struct Path { struct Path {

View File

@ -1,4 +1,3 @@
#include <unordered_map> #include <unordered_map>
#include <vector> #include <vector>
@ -24,7 +23,7 @@ void RaiseIfRemoteError(UpdateResult result) {
break; break;
} }
} }
} } // namespace
UpdateResult UpdatesRpcClients::Update(int worker_id, UpdateResult UpdatesRpcClients::Update(int worker_id,
const database::StateDelta &delta) { const database::StateDelta &delta) {
@ -33,34 +32,36 @@ UpdateResult UpdatesRpcClients::Update(int worker_id,
return res->member; return res->member;
} }
gid::Gid UpdatesRpcClients::CreateVertex( CreatedVertexInfo UpdatesRpcClients::CreateVertex(
int worker_id, tx::TransactionId tx_id, int worker_id, tx::TransactionId tx_id,
const std::vector<storage::Label> &labels, const std::vector<storage::Label> &labels,
const std::unordered_map<storage::Property, query::TypedValue> const std::unordered_map<storage::Property, query::TypedValue> &properties,
&properties) { std::experimental::optional<int64_t> cypher_id) {
auto res = worker_clients_.GetClientPool(worker_id).Call<CreateVertexRpc>( auto res = worker_clients_.GetClientPool(worker_id).Call<CreateVertexRpc>(
CreateVertexReqData{tx_id, labels, properties}); CreateVertexReqData{tx_id, labels, properties, cypher_id});
CHECK(res) << "CreateVertexRpc failed on worker: " << worker_id; CHECK(res) << "CreateVertexRpc failed on worker: " << worker_id;
CHECK(res->member.result == UpdateResult::DONE) CHECK(res->member.result == UpdateResult::DONE)
<< "Remote Vertex creation result not UpdateResult::DONE"; << "Remote Vertex creation result not UpdateResult::DONE";
return res->member.gid; return CreatedVertexInfo(res->member.cypher_id, res->member.gid);
} }
storage::EdgeAddress UpdatesRpcClients::CreateEdge( CreatedEdgeInfo UpdatesRpcClients::CreateEdge(
tx::TransactionId tx_id, VertexAccessor &from, VertexAccessor &to, tx::TransactionId tx_id, VertexAccessor &from, VertexAccessor &to,
storage::EdgeType edge_type) { storage::EdgeType edge_type,
std::experimental::optional<int64_t> cypher_id) {
CHECK(from.address().is_remote()) << "In CreateEdge `from` must be remote"; CHECK(from.address().is_remote()) << "In CreateEdge `from` must be remote";
int from_worker = from.address().worker_id(); int from_worker = from.address().worker_id();
auto res = worker_clients_.GetClientPool(from_worker) auto res =
.Call<CreateEdgeRpc>(CreateEdgeReqData{ worker_clients_.GetClientPool(from_worker)
from.gid(), to.GlobalAddress(), edge_type, tx_id}); .Call<CreateEdgeRpc>(CreateEdgeReqData{from.gid(), to.GlobalAddress(),
edge_type, tx_id, cypher_id});
CHECK(res) << "CreateEdge RPC failed on worker: " << from_worker; CHECK(res) << "CreateEdge RPC failed on worker: " << from_worker;
RaiseIfRemoteError(res->member.result); RaiseIfRemoteError(res->member.result);
return {res->member.gid, from_worker}; return CreatedEdgeInfo(res->member.cypher_id,
storage::EdgeAddress{res->member.gid, from_worker});
} }
void UpdatesRpcClients::AddInEdge(tx::TransactionId tx_id, void UpdatesRpcClients::AddInEdge(tx::TransactionId tx_id, VertexAccessor &from,
VertexAccessor &from,
storage::EdgeAddress edge_address, storage::EdgeAddress edge_address,
VertexAccessor &to, VertexAccessor &to,
storage::EdgeType edge_type) { storage::EdgeType edge_type) {

View File

@ -27,20 +27,26 @@ class UpdatesRpcClients {
UpdateResult Update(int worker_id, const database::StateDelta &delta); UpdateResult Update(int worker_id, const database::StateDelta &delta);
/// Creates a vertex on the given worker and returns it's id. /// Creates a vertex on the given worker and returns it's id.
gid::Gid CreateVertex( CreatedVertexInfo CreateVertex(
int worker_id, tx::TransactionId tx_id, int worker_id, tx::TransactionId tx_id,
const std::vector<storage::Label> &labels, const std::vector<storage::Label> &labels,
const std::unordered_map<storage::Property, query::TypedValue> const std::unordered_map<storage::Property, query::TypedValue>
&properties); &properties,
std::experimental::optional<int64_t> cypher_id =
std::experimental::nullopt);
/// Creates an edge on the given worker and returns it's address. If the `to` /// Creates an edge on the given worker and returns it's address. If the `to`
/// vertex is on the same worker as `from`, then all remote CRUD will be /// vertex is on the same worker as `from`, then all remote CRUD will be
/// handled by a call to this function. Otherwise a separate call to /// handled by a call to this function. Otherwise a separate call to
/// `AddInEdge` might be necessary. Throws all the exceptions that can /// `AddInEdge` might be necessary. Throws all the exceptions that can
/// occur remotely as a result of updating a vertex. /// occur remotely as a result of updating a vertex.
storage::EdgeAddress CreateEdge(tx::TransactionId tx_id, CreatedEdgeInfo CreateEdge(tx::TransactionId tx_id, VertexAccessor &from,
VertexAccessor &from, VertexAccessor &to, VertexAccessor &to, storage::EdgeType edge_type,
storage::EdgeType edge_type); std::experimental::optional<int64_t> cypher_id =
std::experimental::nullopt);
// TODO (buda): Another machine in the cluster is asked to create an edge.
// cypher_id should be generated in that process. It probably doesn't make
// sense to have optional cypher id here. Maybe for the recovery purposes.
/// Adds the edge with the given address to the `to` vertex as an incoming /// Adds the edge with the given address to the `to` vertex as an incoming
/// edge. Only used when `to` is remote and not on the same worker as `from`. /// edge. Only used when `to` is remote and not on the same worker as `from`.
@ -61,8 +67,8 @@ class UpdatesRpcClients {
gid::Gid vertex_from_id, gid::Gid vertex_from_id,
storage::VertexAddress vertex_to_addr); storage::VertexAddress vertex_to_addr);
void RemoveInEdge(tx::TransactionId tx_id, int worker_id, void RemoveInEdge(tx::TransactionId tx_id, int worker_id, gid::Gid vertex_id,
gid::Gid vertex_id, storage::EdgeAddress edge_address); storage::EdgeAddress edge_address);
/// Calls for all the workers (except the given one) to apply their updates /// Calls for all the workers (except the given one) to apply their updates
/// and returns the future results. /// and returns the future results.

View File

@ -48,6 +48,7 @@ cpp<#
(lcp:define-struct create-result () (lcp:define-struct create-result ()
((result "UpdateResult") ((result "UpdateResult")
(cypher-id :int64_t :documentation "Only valid if creation was successful.")
(gid "gid::Gid" :documentation "Only valid if creation was successful.")) (gid "gid::Gid" :documentation "Only valid if creation was successful."))
(:serialize :capnp)) (:serialize :capnp))
@ -82,7 +83,25 @@ cpp<#
distributed::LoadCapnpTypedValue(reader.getValue(), &value); distributed::LoadCapnpTypedValue(reader.getValue(), &value);
return std::make_pair(prop, value); return std::make_pair(prop, value);
}); });
cpp<#))) cpp<#))
(cypher-id "std::experimental::optional<int64_t>"
:capnp-type "Utils.Optional(Utils.BoxInt64)"
:capnp-save
(lambda (builder member)
#>cpp
utils::SaveOptional<utils::capnp::BoxInt64, int64_t>(
${member}, &${builder}, [](auto *builder, const auto &value) {
builder->setValue(value);
});
cpp<#)
:capnp-load
(lambda (reader member)
#>cpp
cypher_id = utils::LoadOptional<utils::capnp::BoxInt64, int64_t>(
${reader}, [](const auto &reader) {
return reader.getValue();
});
cpp<#)))
(:serialize :capnp)) (:serialize :capnp))
(lcp:define-rpc create-vertex (lcp:define-rpc create-vertex
@ -93,7 +112,25 @@ cpp<#
((from "gid::Gid") ((from "gid::Gid")
(to "storage::VertexAddress") (to "storage::VertexAddress")
(edge-type "storage::EdgeType") (edge-type "storage::EdgeType")
(tx-id "tx::TransactionId")) (tx-id "tx::TransactionId")
(cypher-id "std::experimental::optional<int64_t>"
:capnp-type "Utils.Optional(Utils.BoxInt64)"
:capnp-save
(lambda (builder member)
#>cpp
utils::SaveOptional<utils::capnp::BoxInt64, int64_t>(
${member}, &${builder}, [](auto *builder, const auto &value) {
builder->setValue(value);
});
cpp<#)
:capnp-load
(lambda (reader member)
#>cpp
cypher_id = utils::LoadOptional<utils::capnp::BoxInt64, int64_t>(
${reader}, [](const auto &reader) {
return reader.getValue();
});
cpp<#)))
(:serialize :capnp)) (:serialize :capnp))
(lcp:define-rpc create-edge (lcp:define-rpc create-edge
@ -143,4 +180,28 @@ cpp<#
(:request ((member "RemoveInEdgeData"))) (:request ((member "RemoveInEdgeData")))
(:response ((member "UpdateResult")))) (:response ((member "UpdateResult"))))
(lcp:define-struct created-info ()
((cypher-id "int64_t")
(gid "gid::Gid"))
(:public #>cpp
CreatedInfo(int64_t cypher_id, gid::Gid gid)
: cypher_id(cypher_id), gid(gid) {}
cpp<#))
(lcp:define-struct created-vertex-info ()
((cypher-id "int64_t")
(gid "gid::Gid"))
(:public #>cpp
CreatedVertexInfo(int64_t cypher_id, gid::Gid gid)
: cypher_id(cypher_id), gid(gid) {}
cpp<#))
(lcp:define-struct created-edge-info ()
((cypher-id "int64_t")
(edge-address "storage::EdgeAddress"))
(:public #>cpp
CreatedEdgeInfo(int64_t cypher_id, storage::EdgeAddress edge_address)
: cypher_id(cypher_id), edge_address(edge_address) {}
cpp<#))
(lcp:pop-namespace) ;; distributed (lcp:pop-namespace) ;; distributed

View File

@ -60,32 +60,34 @@ UpdateResult UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::Emplace(
} }
template <typename TRecordAccessor> template <typename TRecordAccessor>
gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateVertex( CreatedInfo UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateVertex(
const std::vector<storage::Label> &labels, const std::vector<storage::Label> &labels,
const std::unordered_map<storage::Property, query::TypedValue> const std::unordered_map<storage::Property, query::TypedValue> &properties,
&properties) { std::experimental::optional<int64_t> cypher_id) {
auto result = db_accessor_->InsertVertex(); auto result =
db_accessor_->InsertVertex(std::experimental::nullopt, cypher_id);
for (auto &label : labels) result.add_label(label); for (auto &label : labels) result.add_label(label);
for (auto &kv : properties) result.PropsSet(kv.first, kv.second); for (auto &kv : properties) result.PropsSet(kv.first, kv.second);
std::lock_guard<utils::SpinLock> guard{lock_}; std::lock_guard<utils::SpinLock> guard{lock_};
deltas_.emplace(result.gid(), deltas_.emplace(result.gid(),
std::make_pair(result, std::vector<database::StateDelta>{})); std::make_pair(result, std::vector<database::StateDelta>{}));
return result.gid(); return CreatedInfo(result.CypherId(), result.gid());
} }
template <typename TRecordAccessor> template <typename TRecordAccessor>
gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateEdge( CreatedInfo UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateEdge(
gid::Gid from, storage::VertexAddress to, storage::EdgeType edge_type, gid::Gid from, storage::VertexAddress to, storage::EdgeType edge_type,
int worker_id) { int worker_id, std::experimental::optional<int64_t> cypher_id) {
auto &db = db_accessor_->db(); auto &db = db_accessor_->db();
auto from_addr = db.storage().LocalizedAddressIfPossible( auto from_addr = db.storage().LocalizedAddressIfPossible(
storage::VertexAddress(from, worker_id)); storage::VertexAddress(from, worker_id));
auto to_addr = db.storage().LocalizedAddressIfPossible(to); auto to_addr = db.storage().LocalizedAddressIfPossible(to);
auto edge = db_accessor_->InsertOnlyEdge(from_addr, to_addr, edge_type); auto edge = db_accessor_->InsertOnlyEdge(
from_addr, to_addr, edge_type, std::experimental::nullopt, cypher_id);
std::lock_guard<utils::SpinLock> guard{lock_}; std::lock_guard<utils::SpinLock> guard{lock_};
deltas_.emplace(edge.gid(), deltas_.emplace(edge.gid(),
std::make_pair(edge, std::vector<database::StateDelta>{})); std::make_pair(edge, std::vector<database::StateDelta>{}));
return edge.gid(); return CreatedInfo(edge.CypherId(), edge.gid());
} }
template <typename TRecordAccessor> template <typename TRecordAccessor>
@ -218,9 +220,11 @@ UpdatesRpcServer::UpdatesRpcServer(database::DistributedGraphDb *db,
auto *res_builder) { auto *res_builder) {
CreateVertexReq req; CreateVertexReq req;
req.Load(req_reader); req.Load(req_reader);
gid::Gid gid = GetUpdates(vertex_updates_, req.member.tx_id) auto result = GetUpdates(vertex_updates_, req.member.tx_id)
.CreateVertex(req.member.labels, req.member.properties); .CreateVertex(req.member.labels, req.member.properties,
CreateVertexRes res(CreateResult{UpdateResult::DONE, gid}); req.member.cypher_id);
CreateVertexRes res(
CreateResult{UpdateResult::DONE, result.cypher_id, result.gid});
res.Save(res_builder); res.Save(res_builder);
}); });
@ -337,14 +341,17 @@ UpdatesRpcServer::TransactionUpdates<TAccessor> &UpdatesRpcServer::GetUpdates(
} }
CreateResult UpdatesRpcServer::CreateEdge(const CreateEdgeReqData &req) { CreateResult UpdatesRpcServer::CreateEdge(const CreateEdgeReqData &req) {
auto gid = GetUpdates(edge_updates_, req.tx_id) auto ids = GetUpdates(edge_updates_, req.tx_id)
.CreateEdge(req.from, req.to, req.edge_type, db_->WorkerId()); .CreateEdge(req.from, req.to, req.edge_type, db_->WorkerId(),
req.cypher_id);
// cypher_id doesn't have to be inserted because edge is stored
// somewhere else in the cluster. Here is only vertex update.
auto from_delta = database::StateDelta::AddOutEdge( auto from_delta = database::StateDelta::AddOutEdge(
req.tx_id, req.from, req.to, {gid, db_->WorkerId()}, req.edge_type); req.tx_id, req.from, req.to, {ids.gid, db_->WorkerId()}, req.edge_type);
auto result = GetUpdates(vertex_updates_, req.tx_id).Emplace(from_delta); auto result = GetUpdates(vertex_updates_, req.tx_id).Emplace(from_delta);
return {result, gid}; return {result, ids.cypher_id, ids.gid};
} }
UpdateResult UpdatesRpcServer::RemoveEdge(const RemoveEdgeData &data) { UpdateResult UpdatesRpcServer::RemoveEdge(const RemoveEdgeData &data) {

View File

@ -1,3 +1,5 @@
/// @file
#pragma once #pragma once
#include <unordered_map> #include <unordered_map>
@ -36,21 +38,25 @@ class UpdatesRpcServer {
tx::TransactionId tx_id) tx::TransactionId tx_id)
: db_accessor_(db->Access(tx_id)) {} : db_accessor_(db->Access(tx_id)) {}
/// Adds a delta and returns the result. Does not modify the state (data) of /// Adds a delta and returns the result. Does not modify the state (data)
/// the graph element the update is for, but calls the `update` method to /// of the graph element the update is for, but calls the `update` method
/// fail-fast on serialization and update-after-delete errors. /// to fail-fast on serialization and update-after-delete errors.
UpdateResult Emplace(const database::StateDelta &delta); UpdateResult Emplace(const database::StateDelta &delta);
/// Creates a new vertex and returns it's gid. /// Creates a new vertex and returns it's cypher_id and gid.
gid::Gid CreateVertex( CreatedInfo CreateVertex(
const std::vector<storage::Label> &labels, const std::vector<storage::Label> &labels,
const std::unordered_map<storage::Property, query::TypedValue> const std::unordered_map<storage::Property, query::TypedValue>
&properties); &properties,
std::experimental::optional<int64_t> cypher_id =
std::experimental::nullopt);
/// Creates a new edge and returns it's gid. Does not update vertices at the /// Creates a new edge and returns it's cypher_id and gid. Does not update
/// end of the edge. /// vertices at the end of the edge.
gid::Gid CreateEdge(gid::Gid from, storage::VertexAddress to, CreatedInfo CreateEdge(gid::Gid from, storage::VertexAddress to,
storage::EdgeType edge_type, int worker_id); storage::EdgeType edge_type, int worker_id,
std::experimental::optional<int64_t> cypher_id =
std::experimental::nullopt);
/// Applies all the deltas on the record. /// Applies all the deltas on the record.
UpdateResult Apply(); UpdateResult Apply();
@ -74,8 +80,8 @@ class UpdatesRpcServer {
communication::rpc::Server *server); communication::rpc::Server *server);
/// Applies all existsing updates for the given transaction ID. If there are /// Applies all existsing updates for the given transaction ID. If there are
/// no updates for that transaction, nothing happens. Clears the updates cache /// no updates for that transaction, nothing happens. Clears the updates
/// after applying them, regardless of the result. /// cache after applying them, regardless of the result.
UpdateResult Apply(tx::TransactionId tx_id); UpdateResult Apply(tx::TransactionId tx_id);
/// Clears the cache of local transactions that are completed. The signature /// Clears the cache of local transactions that are completed. The signature

View File

@ -68,7 +68,7 @@ bool Encode(const fs::path &snapshot_file, database::GraphDb &db,
} }
for (const auto &edge : dba.Edges(false)) { for (const auto &edge : dba.Edges(false)) {
encoder.WriteEdge(glue::ToBoltEdge(edge)); encoder.WriteEdge(glue::ToBoltEdge(edge));
encoder.WriteInt(edge.cypher_id()); encoder.WriteInt(edge.CypherId());
edge_num++; edge_num++;
} }
buffer.WriteValue(vertex_num); buffer.WriteValue(vertex_num);

View File

@ -17,7 +17,7 @@ class SnapshotEncoder : public communication::bolt::BaseEncoder<Buffer> {
glue::ToBoltVertex(vertex)); glue::ToBoltVertex(vertex));
// Write cypher_id // Write cypher_id
this->WriteInt(vertex.cypher_id()); this->WriteInt(vertex.CypherId());
// Write in edges without properties // Write in edges without properties
this->WriteUInt(vertex.in_degree()); this->WriteUInt(vertex.in_degree());

View File

@ -224,9 +224,6 @@ class VersionList {
record->mark_expired(t); record->mark_expired(t);
} }
/**
* TODO (buda): Try to move git_ to storage::Address.
*/
const gid::Gid gid_; const gid::Gid gid_;
auto cypher_id() { return cypher_id_; } auto cypher_id() { return cypher_id_; }

View File

@ -616,10 +616,10 @@ TypedValue Id(TypedValue *args, int64_t nargs, Context *ctx) {
auto &arg = args[0]; auto &arg = args[0];
switch (arg.type()) { switch (arg.type()) {
case TypedValue::Type::Vertex: { case TypedValue::Type::Vertex: {
return TypedValue(arg.ValueVertex().cypher_id()); return TypedValue(arg.ValueVertex().CypherId());
} }
case TypedValue::Type::Edge: { case TypedValue::Type::Edge: {
return TypedValue(arg.ValueEdge().cypher_id()); return TypedValue(arg.ValueEdge().CypherId());
} }
default: default:
throw QueryRuntimeException("'id' argument must be a node or an edge."); throw QueryRuntimeException("'id' argument must be a node or an edge.");

View File

@ -1191,8 +1191,9 @@ VertexAccessor &CreateVertexOnWorker(int worker_id, NodeAtom *node_atom,
properties.emplace(kv.first.second, std::move(value)); properties.emplace(kv.first.second, std::move(value));
} }
auto new_node = database::InsertVertexIntoRemote( auto new_node =
&dba, worker_id, node_atom->labels_, properties); database::InsertVertexIntoRemote(&dba, worker_id, node_atom->labels_,
properties, std::experimental::nullopt);
frame[context.symbol_table_.at(*node_atom->identifier_)] = new_node; frame[context.symbol_table_.at(*node_atom->identifier_)] = new_node;
return frame[context.symbol_table_.at(*node_atom->identifier_)].ValueVertex(); return frame[context.symbol_table_.at(*node_atom->identifier_)].ValueVertex();
} }

View File

@ -2010,8 +2010,7 @@ bool RemoveLabels::RemoveLabelsCursor::Pull(Frame &frame, Context &context) {
try { try {
for (auto label : self_.labels_) vertex.remove_label(label); for (auto label : self_.labels_) vertex.remove_label(label);
} catch (const RecordDeletedError &) { } catch (const RecordDeletedError &) {
throw QueryRuntimeException( throw QueryRuntimeException("Trying to remove labels from a deleted node.");
"Trying to remove labels from a deleted node.");
} }
return true; return true;

View File

@ -22,32 +22,38 @@ void VertexMigrator::MigrateVertex(VertexAccessor &vertex, int destination) {
}; };
auto relocated_vertex = database::InsertVertexIntoRemote( auto relocated_vertex = database::InsertVertexIntoRemote(
dba_, destination, vertex.labels(), get_props(vertex)); dba_, destination, vertex.labels(), get_props(vertex), vertex.CypherId());
vertex_migrated_to_[vertex.gid()] = relocated_vertex.address(); vertex_migrated_to_[vertex.gid()] = relocated_vertex.address();
for (auto in_edge : vertex.in()) {
auto from = in_edge.from();
update_if_moved(from);
auto new_in_edge =
dba_->InsertEdge(from, relocated_vertex, in_edge.EdgeType());
for (auto prop : get_props(in_edge)) {
new_in_edge.PropsSet(prop.first, prop.second);
}
}
for (auto out_edge : vertex.out()) { for (auto out_edge : vertex.out()) {
auto to = out_edge.to(); auto to = out_edge.to();
// Continue on self-loops since those edges have already been added
// while iterating over in edges
if (to == vertex) continue;
update_if_moved(to); update_if_moved(to);
// Here cypher_id has to be passed to the other machine because this
// machine owns the edge.
auto new_out_edge = auto new_out_edge =
dba_->InsertEdge(relocated_vertex, to, out_edge.EdgeType()); dba_->InsertEdge(relocated_vertex, to, out_edge.EdgeType(),
std::experimental::nullopt, out_edge.CypherId());
for (auto prop : get_props(out_edge)) { for (auto prop : get_props(out_edge)) {
new_out_edge.PropsSet(prop.first, prop.second); new_out_edge.PropsSet(prop.first, prop.second);
} }
} }
for (auto in_edge : vertex.in()) {
auto from = in_edge.from();
// Continue on self-loops since those edges have already been added
// while iterating over out edges.
if (from == vertex) continue;
update_if_moved(from);
// Both gid and cypher_id should be without value because this machine
// doesn't own the edge.
auto new_in_edge =
dba_->InsertEdge(from, relocated_vertex, in_edge.EdgeType(),
std::experimental::nullopt, in_edge.CypherId());
for (auto prop : get_props(in_edge)) {
new_in_edge.PropsSet(prop.first, prop.second);
}
}
dba_->DetachRemoveVertex(vertex); dba_->DetachRemoveVertex(vertex);
} }

View File

@ -165,11 +165,18 @@ TRecord &RecordAccessor<TRecord>::update() const {
return *new_; return *new_;
} }
template <typename TRecord>
int64_t RecordAccessor<TRecord>::CypherId() const {
return impl_->CypherId(*this);
}
template <typename TRecord> template <typename TRecord>
const TRecord &RecordAccessor<TRecord>::current() const { const TRecord &RecordAccessor<TRecord>::current() const {
// Edges have lazily initialize mutable, versioned data (properties). // Edges have lazily initialize mutable, versioned data (properties).
if (std::is_same<TRecord, Edge>::value && current_ == nullptr) if (std::is_same<TRecord, Edge>::value && current_ == nullptr) {
RecordAccessor::Reconstruct(); bool reconstructed = Reconstruct();
DCHECK(reconstructed) << "Unable to initialize record";
}
DCHECK(current_ != nullptr) << "RecordAccessor.current_ pointer is nullptr"; DCHECK(current_ != nullptr) << "RecordAccessor.current_ pointer is nullptr";
return *current_; return *current_;
} }

View File

@ -53,6 +53,7 @@ class RecordAccessor : public utils::TotalOrdering<RecordAccessor<TRecord>> {
/** Process a change delta, e.g. by writing WAL. */ /** Process a change delta, e.g. by writing WAL. */
virtual void ProcessDelta(const RecordAccessor<TRecord> &ra, virtual void ProcessDelta(const RecordAccessor<TRecord> &ra,
const database::StateDelta &delta) = 0; const database::StateDelta &delta) = 0;
virtual int64_t CypherId(const RecordAccessor<TRecord> &ra) = 0;
}; };
// this class is default copyable, movable and assignable // this class is default copyable, movable and assignable
@ -176,12 +177,10 @@ class RecordAccessor : public utils::TotalOrdering<RecordAccessor<TRecord>> {
* owner is some other worker in a distributed system. */ * owner is some other worker in a distributed system. */
bool is_local() const { return address_.is_local(); } bool is_local() const { return address_.is_local(); }
int64_t cypher_id() const { /**
if (address_.is_local()) * Returns Cypher Id of this record.
return address_.local()->cypher_id(); */
else int64_t CypherId() const;
throw utils::NotYetImplemented("Fetch remote cypher_id");
}
protected: protected:
/** /**

View File

@ -67,12 +67,12 @@ target_link_libraries(${test_prefix}distributed_coordination memgraph_lib kvstor
add_unit_test(distributed_data_exchange.cpp) add_unit_test(distributed_data_exchange.cpp)
target_link_libraries(${test_prefix}distributed_data_exchange memgraph_lib kvstore_dummy_lib) target_link_libraries(${test_prefix}distributed_data_exchange memgraph_lib kvstore_dummy_lib)
add_unit_test(distributed_dgp_vertex_migrator.cpp)
target_link_libraries(${test_prefix}distributed_dgp_vertex_migrator memgraph_lib kvstore_dummy_lib)
add_unit_test(distributed_durability.cpp) add_unit_test(distributed_durability.cpp)
target_link_libraries(${test_prefix}distributed_durability memgraph_lib kvstore_dummy_lib) target_link_libraries(${test_prefix}distributed_durability memgraph_lib kvstore_dummy_lib)
add_unit_test(distributed_dynamic_graph_partitioner.cpp)
target_link_libraries(${test_prefix}distributed_dynamic_graph_partitioner memgraph_lib kvstore_dummy_lib)
add_unit_test(distributed_gc.cpp) add_unit_test(distributed_gc.cpp)
target_link_libraries(${test_prefix}distributed_gc memgraph_lib kvstore_dummy_lib) target_link_libraries(${test_prefix}distributed_gc memgraph_lib kvstore_dummy_lib)
@ -94,9 +94,6 @@ target_link_libraries(${test_prefix}distributed_serialization memgraph_lib kvsto
add_unit_test(distributed_updates.cpp) add_unit_test(distributed_updates.cpp)
target_link_libraries(${test_prefix}distributed_updates memgraph_lib kvstore_dummy_lib) target_link_libraries(${test_prefix}distributed_updates memgraph_lib kvstore_dummy_lib)
# add_unit_test(distributed_vertex_migrator.cpp)
# target_link_libraries(${test_prefix}distributed_vertex_migrator memgraph_lib kvstore_dummy_lib)
add_unit_test(durability.cpp) add_unit_test(durability.cpp)
target_link_libraries(${test_prefix}durability memgraph_lib kvstore_dummy_lib) target_link_libraries(${test_prefix}durability memgraph_lib kvstore_dummy_lib)

View File

@ -50,8 +50,8 @@ class DistributedGraphDbTest : public ::testing::Test {
master_config.durability_directory = tmp_dir_; master_config.durability_directory = tmp_dir_;
// This is semantically wrong since this is not a cluster of size 1 but of // This is semantically wrong since this is not a cluster of size 1 but of
// size kWorkerCount+1, but it's hard to wait here for workers to recover // size kWorkerCount+1, but it's hard to wait here for workers to recover
// and simultaneously assign the port to which the workers must connect // and simultaneously assign the port to which the workers must connect.
// TODO(dgleich): Fix sometime in the future - not mission critical // TODO (buda): Fix sometime in the future - not mission critical.
master_config.recovering_cluster_size = 1; master_config.recovering_cluster_size = 1;
master_ = std::make_unique<database::Master>(modify_config(master_config)); master_ = std::make_unique<database::Master>(modify_config(master_config));
@ -121,8 +121,8 @@ class DistributedGraphDbTest : public ::testing::Test {
auto dba = master().Access(); auto dba = master().Access();
VertexAccessor from{from_addr, *dba}; VertexAccessor from{from_addr, *dba};
VertexAccessor to{to_addr, *dba}; VertexAccessor to{to_addr, *dba};
auto r_val = auto r_val = dba->InsertEdge(from, to, dba->EdgeType(edge_type_name))
dba->InsertEdge(from, to, dba->EdgeType(edge_type_name)).GlobalAddress(); .GlobalAddress();
master().updates_server().Apply(dba->transaction_id()); master().updates_server().Apply(dba->transaction_id());
worker(1).updates_server().Apply(dba->transaction_id()); worker(1).updates_server().Apply(dba->transaction_id());
worker(2).updates_server().Apply(dba->transaction_id()); worker(2).updates_server().Apply(dba->transaction_id());

View File

@ -0,0 +1,401 @@
#include "distributed_common.hpp"
#include <memory>
#include <thread>
#include <unordered_set>
#include "gtest/gtest.h"
#include "distributed/updates_rpc_clients.hpp"
#include "storage/dynamic_graph_partitioner/vertex_migrator.hpp"
using namespace distributed;
using namespace database;
/// Check if the following data is migrated correctly accross the cluster:
/// * cypher_id
/// * labels
/// * edge_types
/// * properties
class DistributedVertexMigratorTest : public DistributedGraphDbTest {
private:
struct GraphSize {
int worker_id;
int vertex_no;
int edge_no;
};
public:
DistributedVertexMigratorTest() : DistributedGraphDbTest("vertex_migrator") {}
/**
* Prefill the cluster with vertices and edges so that the ids are not the
* same across the cluster.
*/
void FillOutCluster(const std::vector<int> graph_sizes) {
for (int i = 0; i < graph_sizes[0]; ++i) {
auto vaddr = InsertVertex(master());
InsertEdge(vaddr, vaddr, "edge");
}
for (int i = 0; i < graph_sizes[1]; ++i) {
auto vaddr = InsertVertex(worker(1));
InsertEdge(vaddr, vaddr, "edge");
}
for (int i = 0; i < graph_sizes[2]; ++i) {
auto vaddr = InsertVertex(worker(2));
InsertEdge(vaddr, vaddr, "edge");
}
}
/**
* Wait for all futures and commit the transaction.
*/
void MasterApplyUpdatesAndCommit(database::GraphDbAccessor *dba) {
{
auto apply_futures = master().updates_clients().UpdateApplyAll(
master().WorkerId(), dba->transaction().id_);
// Destructor waits on application
}
dba->Commit();
}
/**
* Migrate vertex with a given cypher_id from a given database to a given
* machine.
*/
void MigrateVertexAndCommit(database::GraphDbAccessor *from_dba,
int64_t cypher_id, int to_worker_id) {
auto vacc = FindVertex(from_dba, cypher_id);
VertexMigrator migrator(from_dba);
migrator.MigrateVertex(*vacc, to_worker_id);
MasterApplyUpdatesAndCommit(from_dba);
}
/**
* Assert number of vertices and edges on each worker.
*
* @param sizes An array of structs that hold information about graph size
* on each worker.
*/
void CheckGraphSizes(const std::vector<GraphSize> &graph_sizes) {
for (auto &graph_size : graph_sizes) {
if (graph_size.worker_id == 0) { // on master
ASSERT_EQ(VertexCount(master()), graph_size.vertex_no);
ASSERT_EQ(EdgeCount(master()), graph_size.edge_no);
} else { // on workers
ASSERT_EQ(VertexCount(worker(graph_size.worker_id)),
graph_size.vertex_no);
ASSERT_EQ(EdgeCount(worker(graph_size.worker_id)), graph_size.edge_no);
}
}
}
/**
* Collect all visible cypher_ids into an unordered_map for easier
* checking.
*/
auto CollectVertexCypherIds(database::GraphDbAccessor *dba) {
std::unordered_set<int64_t> cypher_ids;
for (auto &vertex : dba->Vertices(false)) {
cypher_ids.emplace(vertex.CypherId());
}
return cypher_ids;
}
/**
* Collect all visible cypher_ids into an unordered_map for easier
* checking.
*/
auto CollectEdgeCypherIds(database::GraphDbAccessor *dba) {
std::unordered_set<int64_t> cypher_ids;
for (auto &edge : dba->Edges(false)) {
cypher_ids.emplace(edge.CypherId());
}
return cypher_ids;
}
/**
* Check that container contains all containees.
*
* @tparam type of elements in the sets.
*/
template <typename T>
auto ContainsAll(const std::unordered_set<T> &container,
const std::unordered_set<T> &containees) {
// TODO (C++20): container.contains(item);
return std::all_of(containees.begin(), containees.end(),
[&container](T item) {
return container.find(item) != container.end();
});
}
/**
* Find vertex with a given cypher_id within a given database.
*/
std::experimental::optional<VertexAccessor> FindVertex(
database::GraphDbAccessor *dba, int64_t cypher_id) {
for (auto &vertex : dba->Vertices(false)) {
if (vertex.CypherId() == cypher_id)
return std::experimental::optional<VertexAccessor>(vertex);
}
return std::experimental::nullopt;
}
/**
* Find edge with a given cypher_id within a given database.
*/
std::experimental::optional<EdgeAccessor> FindEdge(
database::GraphDbAccessor *dba, int64_t cypher_id) {
for (auto &edge : dba->Edges(false)) {
if (edge.CypherId() == cypher_id)
return std::experimental::optional<EdgeAccessor>(edge);
}
return std::experimental::nullopt;
}
};
TEST_F(DistributedVertexMigratorTest, MigrationofLabelsEdgeTypesAndProperties) {
{
auto dba = master().Access();
auto va = dba->InsertVertex();
auto vb = dba->InsertVertex();
va.add_label(dba->Label("l"));
va.add_label(dba->Label("k"));
vb.add_label(dba->Label("l"));
vb.add_label(dba->Label("k"));
va.PropsSet(dba->Property("p"), 42);
vb.PropsSet(dba->Property("p"), 42);
auto ea = dba->InsertEdge(va, vb, dba->EdgeType("edge"));
ea.PropsSet(dba->Property("pe"), 43);
auto eb = dba->InsertEdge(vb, va, dba->EdgeType("edge"));
eb.PropsSet(dba->Property("pe"), 43);
dba->Commit();
}
{
auto dba = master().Access();
VertexMigrator migrator(dba.get());
for (auto &vertex : dba->Vertices(false)) {
migrator.MigrateVertex(vertex, worker(1).WorkerId());
}
MasterApplyUpdatesAndCommit(dba.get());
}
{
auto dba = worker(1).Access();
EXPECT_EQ(VertexCount(master()), 0);
ASSERT_EQ(VertexCount(worker(1)), 2);
for (auto vertex : dba->Vertices(false)) {
ASSERT_EQ(vertex.labels().size(), 2);
EXPECT_EQ(vertex.labels()[0], dba->Label("l"));
EXPECT_EQ(vertex.labels()[1], dba->Label("k"));
EXPECT_EQ(vertex.PropsAt(dba->Property("p")).Value<int64_t>(), 42);
}
ASSERT_EQ(EdgeCount(worker(1)), 2);
auto edge = *dba->Edges(false).begin();
EXPECT_EQ(edge.PropsAt(dba->Property("pe")).Value<int64_t>(), 43);
EXPECT_EQ(edge.EdgeType(), dba->EdgeType("edge"));
}
}
TEST_F(DistributedVertexMigratorTest, MigrationOfSelfLoopEdge) {
FillOutCluster({10, 0, 0});
// Create additional node on master and migrate that node to worker1.
auto vaddr = InsertVertex(master());
auto eaddr = InsertEdge(vaddr, vaddr, "edge");
auto dba = master().Access();
VertexAccessor vacc(vaddr, *dba);
EdgeAccessor eacc(eaddr, *dba);
auto initial_vcypher_id = vacc.CypherId();
auto initial_ecypher_id = eacc.CypherId();
{
auto dba = master().Access();
MigrateVertexAndCommit(dba.get(), initial_vcypher_id, worker(1).WorkerId());
}
// Check grpah size and cypher_ids.
CheckGraphSizes({{0, 10, 10}, {1, 1, 1}, {2, 0, 0}});
{
auto dba = worker(1).Access();
auto vaccessor = *dba->Vertices(false).begin();
auto eaccessor = *dba->Edges(false).begin();
ASSERT_EQ(vaccessor.CypherId(), initial_vcypher_id);
ASSERT_EQ(eaccessor.CypherId(), initial_ecypher_id);
ASSERT_TRUE(eaccessor.from_addr().is_local());
ASSERT_TRUE(eaccessor.to_addr().is_local());
}
}
TEST_F(DistributedVertexMigratorTest, MigrationOfSimpleVertex) {
FillOutCluster({1, 100, 200});
auto v1addr = InsertVertex(master());
auto v2addr = InsertVertex(master());
auto e1addr = InsertEdge(v1addr, v2addr, "edge");
auto dba = master().Access();
auto original_v1_cypher_id = VertexAccessor(v1addr, *dba).CypherId();
auto original_v2_cypher_id = VertexAccessor(v2addr, *dba).CypherId();
std::unordered_set<int64_t> original_v_cypher_ids = {original_v1_cypher_id,
original_v2_cypher_id};
auto original_e1_cypher_id = EdgeAccessor(e1addr, *dba).CypherId();
std::unordered_set<int64_t> original_e_cypher_ids = {original_e1_cypher_id};
CheckGraphSizes({{0, 3, 2}, {1, 100, 100}, {2, 200, 200}});
// Migrate v2 from master to worker1.
{
auto dba = master().Access();
MigrateVertexAndCommit(dba.get(), original_v2_cypher_id,
worker(1).WorkerId());
}
CheckGraphSizes({{0, 2, 2}, {1, 101, 100}, {2, 200, 200}});
{
auto dba = worker(1).Access();
auto v2acc = FindVertex(dba.get(), original_v2_cypher_id);
ASSERT_TRUE(v2acc);
}
// Migrate v1 from master to worker1.
{
auto dba = master().Access();
MigrateVertexAndCommit(dba.get(), original_v1_cypher_id,
worker(1).WorkerId());
}
CheckGraphSizes({{0, 1, 1}, {1, 102, 101}, {2, 200, 200}});
{
auto dba = worker(1).Access();
auto worker1_v_cypher_ids = CollectVertexCypherIds(dba.get());
auto worker1_e_cypher_ids = CollectEdgeCypherIds(dba.get());
ASSERT_TRUE(ContainsAll(worker1_v_cypher_ids, original_v_cypher_ids));
ASSERT_TRUE(ContainsAll(worker1_e_cypher_ids, original_e_cypher_ids));
}
// Migrate v1 from worker1 to worker2.
{
auto dba = worker(1).Access();
MigrateVertexAndCommit(dba.get(), original_v1_cypher_id,
worker(2).WorkerId());
}
CheckGraphSizes({{0, 1, 1}, {1, 101, 100}, {2, 201, 201}});
{
auto dba = worker(2).Access();
auto worker2_v_cypher_ids = CollectVertexCypherIds(dba.get());
auto worker2_e_cypher_ids = CollectEdgeCypherIds(dba.get());
ASSERT_TRUE(ContainsAll(worker2_v_cypher_ids, {original_v1_cypher_id}));
ASSERT_TRUE(ContainsAll(worker2_e_cypher_ids, {original_e1_cypher_id}));
}
// Migrate v2 from worker1 to master.
{
auto dba = worker(1).Access();
MigrateVertexAndCommit(dba.get(), original_v2_cypher_id,
master().WorkerId());
}
CheckGraphSizes({{0, 2, 1}, {1, 100, 100}, {2, 201, 201}});
{
auto master_dba = master().Access();
auto worker2_dba = worker(2).Access();
auto master_v_cypher_ids = CollectVertexCypherIds(master_dba.get());
auto worker2_v_cypher_ids = CollectVertexCypherIds(worker2_dba.get());
auto worker2_e_cypher_ids = CollectEdgeCypherIds(worker2_dba.get());
ASSERT_TRUE(ContainsAll(master_v_cypher_ids, {original_v2_cypher_id}));
ASSERT_TRUE(ContainsAll(worker2_v_cypher_ids, {original_v1_cypher_id}));
ASSERT_TRUE(ContainsAll(worker2_e_cypher_ids, {original_e1_cypher_id}));
}
// Migrate v2 from master wo worker2.
{
auto dba = master().Access();
MigrateVertexAndCommit(dba.get(), original_v2_cypher_id,
worker(2).WorkerId());
}
CheckGraphSizes({{0, 1, 1}, {1, 100, 100}, {2, 202, 201}});
{
auto dba = worker(2).Access();
auto worker2_v_cypher_ids = CollectVertexCypherIds(dba.get());
auto worker2_e_cypher_ids = CollectEdgeCypherIds(dba.get());
ASSERT_TRUE(ContainsAll(worker2_v_cypher_ids, original_v_cypher_ids));
ASSERT_TRUE(ContainsAll(worker2_e_cypher_ids, original_e_cypher_ids));
}
}
TEST_F(DistributedVertexMigratorTest, MigrationOfVertexWithMultipleEdges) {
FillOutCluster({1, 100, 200});
auto m_v1addr = InsertVertex(master());
auto m_v2addr = InsertVertex(master());
auto v3addr = InsertVertex(master());
auto dba = master().Access();
auto original_v3_cypher_id = VertexAccessor(v3addr, *dba).CypherId();
auto w1_v1addr = InsertVertex(worker(1));
auto w1_v2addr = InsertVertex(worker(1));
auto w2_v1addr = InsertVertex(worker(2));
auto w2_v2addr = InsertVertex(worker(2));
auto e1addr = InsertEdge(v3addr, m_v1addr, "edge");
auto e2addr = InsertEdge(v3addr, m_v2addr, "edge");
auto e3addr = InsertEdge(v3addr, w1_v1addr, "edge");
auto e4addr = InsertEdge(v3addr, w1_v2addr, "edge");
auto e5addr = InsertEdge(v3addr, w2_v1addr, "edge");
auto e6addr = InsertEdge(v3addr, w2_v2addr, "edge");
std::unordered_set<int64_t> original_e_cypher_ids = {
EdgeAccessor(e1addr, *dba).CypherId(),
EdgeAccessor(e2addr, *dba).CypherId(),
EdgeAccessor(e3addr, *dba).CypherId(),
EdgeAccessor(e4addr, *dba).CypherId(),
EdgeAccessor(e5addr, *dba).CypherId(),
EdgeAccessor(e6addr, *dba).CypherId()};
CheckGraphSizes({{0, 4, 7}, {1, 102, 100}, {2, 202, 200}});
// Migrate v3 from master to worker1.
{
auto dba = master().Access();
MigrateVertexAndCommit(dba.get(), original_v3_cypher_id,
worker(1).WorkerId());
}
CheckGraphSizes({{0, 3, 1}, {1, 103, 106}, {2, 202, 200}});
{
auto dba = worker(1).Access();
auto worker1_v_cypher_ids = CollectVertexCypherIds(dba.get());
auto worker1_e_cypher_ids = CollectEdgeCypherIds(dba.get());
ASSERT_TRUE(ContainsAll(worker1_v_cypher_ids, {original_v3_cypher_id}));
ASSERT_TRUE(ContainsAll(worker1_e_cypher_ids, original_e_cypher_ids));
}
// Migrate v3 from worker1 to worker2.
{
auto dba = worker(1).Access();
MigrateVertexAndCommit(dba.get(), original_v3_cypher_id,
worker(2).WorkerId());
}
CheckGraphSizes({{0, 3, 1}, {1, 102, 100}, {2, 203, 206}});
{
auto dba = worker(2).Access();
auto worker2_v_cypher_ids = CollectVertexCypherIds(dba.get());
auto worker2_e_cypher_ids = CollectEdgeCypherIds(dba.get());
ASSERT_TRUE(ContainsAll(worker2_v_cypher_ids, {original_v3_cypher_id}));
ASSERT_TRUE(ContainsAll(worker2_e_cypher_ids, original_e_cypher_ids));
}
// Migrate v3 from worker2 back to master.
{
auto dba = worker(2).Access();
MigrateVertexAndCommit(dba.get(), original_v3_cypher_id,
master().WorkerId());
}
CheckGraphSizes({{0, 4, 7}, {1, 102, 100}, {2, 202, 200}});
{
auto dba = master().Access();
auto master_v_cypher_ids = CollectVertexCypherIds(dba.get());
auto master_e_cypher_ids = CollectEdgeCypherIds(dba.get());
ASSERT_TRUE(ContainsAll(master_v_cypher_ids, {original_v3_cypher_id}));
ASSERT_TRUE(ContainsAll(master_e_cypher_ids, original_e_cypher_ids));
}
}

View File

@ -1,158 +0,0 @@
#include "distributed_common.hpp"
#include <memory>
#include <thread>
#include <unordered_set>
#include <vector>
#include "gtest/gtest.h"
#include "distributed/updates_rpc_clients.hpp"
#include "storage/dynamic_graph_partitioner/dgp.hpp"
using namespace distributed;
using namespace database;
DECLARE_int32(dgp_max_batch_size);
class DistributedDynamicGraphPartitionerTest : public DistributedGraphDbTest {
public:
DistributedDynamicGraphPartitionerTest()
: DistributedGraphDbTest("dynamic_graph_partitioner") {}
};
TEST_F(DistributedDynamicGraphPartitionerTest, CountLabels) {
auto va = InsertVertex(master());
auto vb = InsertVertex(worker(1));
auto vc = InsertVertex(worker(2));
for (int i = 0; i < 2; ++i) InsertEdge(va, va, "edge");
for (int i = 0; i < 3; ++i) InsertEdge(va, vb, "edge");
for (int i = 0; i < 4; ++i) InsertEdge(va, vc, "edge");
for (int i = 0; i < 5; ++i) InsertEdge(vb, va, "edge");
for (int i = 0; i < 6; ++i) InsertEdge(vc, va, "edge");
DynamicGraphPartitioner dgp(&master());
auto dba = master().Access();
VertexAccessor v(va, *dba);
auto count_labels = dgp.CountLabels(v);
// Self loops counted twice
EXPECT_EQ(count_labels[master().WorkerId()], 2 * 2);
EXPECT_EQ(count_labels[worker(1).WorkerId()], 3 + 5);
EXPECT_EQ(count_labels[worker(2).WorkerId()], 4 + 6);
}
TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMoveVertex) {
auto va = InsertVertex(master());
auto vb = InsertVertex(worker(1));
// Balance the number of nodes on workers a bit
InsertVertex(worker(2));
InsertVertex(worker(2));
for (int i = 0; i < 100; ++i) InsertEdge(va, vb, "edge");
DynamicGraphPartitioner dgp(&master());
auto dba = master().Access();
auto migrations = dgp.FindMigrations(*dba);
// Expect `va` to try to move to another worker, the one connected to it
ASSERT_EQ(migrations.size(), 1);
EXPECT_EQ(migrations[0].second, worker(1).WorkerId());
}
TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsNoChange) {
InsertVertex(master());
InsertVertex(worker(1));
InsertVertex(worker(2));
// Everything is balanced, there should be no movement
DynamicGraphPartitioner dgp(&master());
auto dba = master().Access();
auto migrations = dgp.FindMigrations(*dba);
EXPECT_EQ(migrations.size(), 0);
}
TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMultipleAndLimit) {
auto va = InsertVertex(master());
auto vb = InsertVertex(master());
auto vc = InsertVertex(worker(1));
// Balance the number of nodes on workers a bit
InsertVertex(worker(1));
InsertVertex(worker(2));
InsertVertex(worker(2));
for (int i = 0; i < 100; ++i) InsertEdge(va, vc, "edge");
for (int i = 0; i < 100; ++i) InsertEdge(vb, vc, "edge");
DynamicGraphPartitioner dgp(&master());
auto dba = master().Access();
{
auto migrations = dgp.FindMigrations(*dba);
// Expect vertices to try to move to another worker
ASSERT_EQ(migrations.size(), 2);
}
// See if flag affects number of returned results
{
FLAGS_dgp_max_batch_size = 1;
auto migrations = dgp.FindMigrations(*dba);
// Expect vertices to try to move to another worker
ASSERT_EQ(migrations.size(), 1);
}
}
TEST_F(DistributedDynamicGraphPartitionerTest, Run) {
// Emulate a bipartite graph with lots of connections on the left, and right
// side, and some connections between the halfs
std::vector<storage::VertexAddress> left;
for (int i = 0; i < 10; ++i) {
left.push_back(InsertVertex(master()));
}
std::vector<storage::VertexAddress> right;
for (int i = 0; i < 10; ++i) {
right.push_back(InsertVertex(master()));
}
// Force the nodes of both sides to stay on one worker by inserting a lot of
// edges in between them
for (int i = 0; i < 1000; ++i) {
InsertEdge(left[rand() % 10], left[rand() % 10], "edge");
InsertEdge(right[rand() % 10], right[rand() % 10], "edge");
}
// Insert edges between left and right side
for (int i = 0; i < 50; ++i)
InsertEdge(left[rand() % 10], right[rand() % 10], "edge");
// Balance it out so that the vertices count on workers don't influence the
// partitioning too much
for (int i = 0; i < 10; ++i) InsertVertex(worker(2));
DynamicGraphPartitioner dgp(&master());
// Transfer one by one to actually converge
FLAGS_dgp_max_batch_size = 1;
// Try a bit more transfers to see if we reached a steady state
for (int i = 0; i < 15; ++i) {
dgp.Run();
}
EXPECT_EQ(VertexCount(master()), 10);
EXPECT_EQ(VertexCount(worker(1)), 10);
auto CountRemotes = [](GraphDbAccessor &dba) {
int64_t cnt = 0;
for (auto vertex : dba.Vertices(false)) {
for (auto edge : vertex.in())
if (edge.from_addr().is_remote()) ++cnt;
for (auto edge : vertex.out())
if (edge.to_addr().is_remote()) ++cnt;
}
return cnt;
};
auto dba_m = master().Access();
auto dba_w1 = worker(1).Access();
EXPECT_EQ(CountRemotes(*dba_m), 50);
EXPECT_EQ(CountRemotes(*dba_w1), 50);
}

View File

@ -77,7 +77,8 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertex) {
gid::Gid gid; gid::Gid gid;
{ {
auto dba = worker(1).Access(); auto dba = worker(1).Access();
auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {}); auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {},
std::experimental::nullopt);
gid = v.gid(); gid = v.gid();
dba->Commit(); dba->Commit();
} }
@ -93,7 +94,8 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithUpdate) {
storage::Property prop; storage::Property prop;
{ {
auto dba = worker(1).Access(); auto dba = worker(1).Access();
auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {}); auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {},
std::experimental::nullopt);
gid = v.gid(); gid = v.gid();
prop = dba->Property("prop"); prop = dba->Property("prop");
v.PropsSet(prop, 42); v.PropsSet(prop, 42);
@ -118,8 +120,8 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithData) {
l1 = dba->Label("l1"); l1 = dba->Label("l1");
l2 = dba->Label("l2"); l2 = dba->Label("l2");
prop = dba->Property("prop"); prop = dba->Property("prop");
auto v = auto v = database::InsertVertexIntoRemote(
database::InsertVertexIntoRemote(dba.get(), 2, {l1, l2}, {{prop, 42}}); dba.get(), 2, {l1, l2}, {{prop, 42}}, std::experimental::nullopt);
gid = v.gid(); gid = v.gid();
// Check local visibility before commit. // Check local visibility before commit.

View File

@ -1,166 +0,0 @@
#include "distributed_common.hpp"
#include <memory>
#include <thread>
#include <unordered_set>
#include "gtest/gtest.h"
#include "distributed/updates_rpc_clients.hpp"
#include "storage/dynamic_graph_partitioner/vertex_migrator.hpp"
using namespace distributed;
using namespace database;
class DistributedVertexMigratorTest : public DistributedGraphDbTest {
public:
DistributedVertexMigratorTest() : DistributedGraphDbTest("vertex_migrator") {}
};
// Check if the auto-generated gid property is unchanged after migration
TEST_F(DistributedVertexMigratorTest, VertexEdgeGidSaved) {
// Fill master so that the ids are not the same on master and worker 1
for (int i = 0; i < 10; ++i) {
auto va = InsertVertex(master());
InsertEdge(va, va, "edge");
}
auto va = InsertVertex(master());
auto ea = InsertEdge(va, va, "edge");
database::GraphDbAccessor dba(master());
VertexAccessor vacc(va, dba);
EdgeAccessor eacc(ea, dba);
auto old_vgid_id = vacc.cypher_id();
auto old_egid_id = eacc.cypher_id();
{
database::GraphDbAccessor dba(master());
VertexAccessor accessor(va, dba);
VertexMigrator migrator(&dba);
migrator.MigrateVertex(accessor, worker(1).WorkerId());
{
auto apply_futures = master().updates_clients().UpdateApplyAll(
master().WorkerId(), dba.transaction().id_);
// Destructor waits on application
}
dba.Commit();
}
ASSERT_EQ(VertexCount(worker(1)), 1);
{
database::GraphDbAccessor dba(worker(1));
auto vaccessor = *dba.Vertices(false).begin();
auto eaccessor = *dba.Edges(false).begin();
EXPECT_EQ(vaccessor.cypher_id(), old_vgid_id);
EXPECT_EQ(eaccessor.cypher_id(), old_egid_id);
}
}
// Checks if two connected nodes from master will be transfered to worker 1 and
// if edge from vertex on the worker 2 will now point to worker 1 after transfer
TEST_F(DistributedVertexMigratorTest, SomeTransfer) {
auto va = InsertVertex(master());
auto vb = InsertVertex(master());
auto vc = InsertVertex(worker(2));
InsertEdge(va, vb, "edge");
InsertEdge(vc, va, "edge");
{
database::GraphDbAccessor dba(master());
VertexMigrator migrator(&dba);
for (auto &vertex : dba.Vertices(false)) {
migrator.MigrateVertex(vertex, worker(1).WorkerId());
}
{
auto apply_futures = master().updates_clients().UpdateApplyAll(
master().WorkerId(), dba.transaction().id_);
// Destructor waits on application
}
dba.Commit();
}
EXPECT_EQ(VertexCount(master()), 0);
EXPECT_EQ(EdgeCount(master()), 0);
EXPECT_EQ(VertexCount(worker(1)), 2);
EXPECT_EQ(EdgeCount(worker(1)), 1);
EXPECT_EQ(VertexCount(worker(2)), 1);
ASSERT_EQ(EdgeCount(worker(2)), 1);
{
database::GraphDbAccessor dba(worker(2));
auto edge = *dba.Edges(false).begin();
// Updated remote edge on another worker
EXPECT_EQ(edge.to_addr().worker_id(), worker(1).WorkerId());
}
}
// Check if cycle edge is transfered only once since it's contained in both in
// and out edges of a vertex and if not handled correctly could cause problems
TEST_F(DistributedVertexMigratorTest, EdgeCycle) {
auto va = InsertVertex(master());
InsertEdge(va, va, "edge");
{
database::GraphDbAccessor dba(master());
VertexMigrator migrator(&dba);
for (auto &vertex : dba.Vertices(false)) {
migrator.MigrateVertex(vertex, worker(1).WorkerId());
}
{
auto apply_futures = master().updates_clients().UpdateApplyAll(
master().WorkerId(), dba.transaction().id_);
// Destructor waits on application
}
dba.Commit();
}
EXPECT_EQ(VertexCount(master()), 0);
EXPECT_EQ(EdgeCount(master()), 0);
EXPECT_EQ(VertexCount(worker(1)), 1);
EXPECT_EQ(EdgeCount(worker(1)), 1);
}
TEST_F(DistributedVertexMigratorTest, TransferLabelsAndProperties) {
{
database::GraphDbAccessor dba(master());
auto va = dba.InsertVertex();
auto vb = dba.InsertVertex();
va.add_label(dba.Label("l"));
vb.add_label(dba.Label("l"));
va.PropsSet(dba.Property("p"), 42);
vb.PropsSet(dba.Property("p"), 42);
auto ea = dba.InsertEdge(va, vb, dba.EdgeType("edge"));
ea.PropsSet(dba.Property("pe"), 43);
auto eb = dba.InsertEdge(vb, va, dba.EdgeType("edge"));
eb.PropsSet(dba.Property("pe"), 43);
dba.Commit();
}
{
database::GraphDbAccessor dba(master());
VertexMigrator migrator(&dba);
for (auto &vertex : dba.Vertices(false)) {
migrator.MigrateVertex(vertex, worker(1).WorkerId());
}
{
auto apply_futures = master().updates_clients().UpdateApplyAll(
master().WorkerId(), dba.transaction().id_);
// Destructor waits on application
}
dba.Commit();
}
{
database::GraphDbAccessor dba(worker(1));
EXPECT_EQ(VertexCount(master()), 0);
ASSERT_EQ(VertexCount(worker(1)), 2);
for (auto vertex : dba.Vertices(false)) {
ASSERT_EQ(vertex.labels().size(), 1);
EXPECT_EQ(vertex.labels()[0], dba.Label("l"));
EXPECT_EQ(vertex.PropsAt(dba.Property("p")).Value<int64_t>(), 42);
}
ASSERT_EQ(EdgeCount(worker(1)), 2);
auto edge = *dba.Edges(false).begin();
EXPECT_EQ(edge.PropsAt(dba.Property("pe")).Value<int64_t>(), 43);
EXPECT_EQ(edge.EdgeType(), dba.EdgeType("edge"));
}
}

View File

@ -1148,8 +1148,8 @@ class QueryPlanExpandBfs
vertex.PropsSet(prop.second, id); vertex.PropsSet(prop.second, id);
v.push_back(vertex.GlobalAddress()); v.push_back(vertex.GlobalAddress());
} else { } else {
auto vertex = database::InsertVertexIntoRemote(&dba, worker, {}, auto vertex = database::InsertVertexIntoRemote(
{{prop.second, id}}); &dba, worker, {}, {{prop.second, id}}, std::experimental::nullopt);
v.push_back(vertex.GlobalAddress()); v.push_back(vertex.GlobalAddress());
} }
} }

View File

@ -19,7 +19,7 @@ TEST(StateDelta, CreateVertex) {
auto dba = db.Access(); auto dba = db.Access();
auto vertex = dba->FindVertexOptional(gid0, false); auto vertex = dba->FindVertexOptional(gid0, false);
EXPECT_TRUE(vertex); EXPECT_TRUE(vertex);
EXPECT_EQ(vertex->cypher_id(), 0); EXPECT_EQ(vertex->CypherId(), 0);
} }
} }
@ -137,8 +137,8 @@ TEST(StateDelta, RemoveLabel) {
} }
{ {
auto dba = db.Access(); auto dba = db.Access();
auto delta = database::StateDelta::RemoveLabel(dba->transaction_id(), gid0, auto delta = database::StateDelta::RemoveLabel(
dba->Label("label"), "label"); dba->transaction_id(), gid0, dba->Label("label"), "label");
delta.Apply(*dba); delta.Apply(*dba);
dba->Commit(); dba->Commit();
} }

View File

@ -6,7 +6,7 @@
#include "stats/stats_rpc_messages.hpp" #include "stats/stats_rpc_messages.hpp"
#include "utils/string.hpp" #include "utils/string.hpp"
// TODO (buda): move this logic to a unit test // TODO (buda): Move this logic to a unit test.
bool parse_input(const std::string &s, std::string &metric_path, bool parse_input(const std::string &s, std::string &metric_path,
std::vector<std::pair<std::string, std::string>> &tags, std::vector<std::pair<std::string, std::string>> &tags,