Support removed vertices in the triggers (#140)

This commit is contained in:
antonio2368 2021-05-04 13:00:07 +02:00 committed by Antonio Andelic
parent 2f3fa656d9
commit 11c0dde11c
14 changed files with 245 additions and 103 deletions

View File

@ -103,7 +103,6 @@ ParsedQuery ParseQuery(const std::string &query_string, const std::map<std::stri
is_cacheable};
}
namespace {
std::unique_ptr<LogicalPlan> MakeLogicalPlan(AstStorage ast_storage, CypherQuery *query, const Parameters &parameters,
DbAccessor *db_accessor,
const std::vector<Identifier *> &predefined_identifiers) {
@ -114,26 +113,28 @@ std::unique_ptr<LogicalPlan> MakeLogicalPlan(AstStorage ast_storage, CypherQuery
return std::make_unique<SingleNodeLogicalPlan>(std::move(root), cost, std::move(ast_storage),
std::move(symbol_table));
}
} // namespace
std::shared_ptr<CachedPlan> CypherQueryToPlan(uint64_t hash, AstStorage ast_storage, CypherQuery *query,
const Parameters &parameters, utils::SkipList<PlanCacheEntry> *plan_cache,
DbAccessor *db_accessor, const bool is_cacheable,
DbAccessor *db_accessor,
const std::vector<Identifier *> &predefined_identifiers) {
auto plan_cache_access = plan_cache->access();
auto it = plan_cache_access.find(hash);
if (it != plan_cache_access.end()) {
if (it->second->IsExpired()) {
plan_cache_access.remove(hash);
} else {
return it->second;
std::optional<utils::SkipList<PlanCacheEntry>::Accessor> plan_cache_access;
if (plan_cache) {
plan_cache_access.emplace(plan_cache->access());
auto it = plan_cache_access->find(hash);
if (it != plan_cache_access->end()) {
if (it->second->IsExpired()) {
plan_cache_access->remove(hash);
} else {
return it->second;
}
}
}
auto plan = std::make_shared<CachedPlan>(
MakeLogicalPlan(std::move(ast_storage), query, parameters, db_accessor, predefined_identifiers));
if (is_cacheable) {
plan_cache_access.insert({hash, plan});
if (plan_cache_access) {
plan_cache_access->insert({hash, plan});
}
return plan;
}

View File

@ -129,6 +129,10 @@ class SingleNodeLogicalPlan final : public LogicalPlan {
SymbolTable symbol_table_;
};
std::unique_ptr<LogicalPlan> MakeLogicalPlan(AstStorage ast_storage, CypherQuery *query, const Parameters &parameters,
DbAccessor *db_accessor,
const std::vector<Identifier *> &predefined_identifiers);
/**
* Return the parsed *Cypher* query's AST cached logical plan, or create and
* cache a fresh one if it doesn't yet exist.
@ -139,7 +143,7 @@ class SingleNodeLogicalPlan final : public LogicalPlan {
*/
std::shared_ptr<CachedPlan> CypherQueryToPlan(uint64_t hash, AstStorage ast_storage, CypherQuery *query,
const Parameters &parameters, utils::SkipList<PlanCacheEntry> *plan_cache,
DbAccessor *db_accessor, bool is_cacheable = true,
DbAccessor *db_accessor,
const std::vector<Identifier *> &predefined_identifiers = {});
} // namespace query

View File

@ -246,8 +246,18 @@ class DbAccessor final {
return accessor_->DetachDeleteVertex(&vertex_accessor->impl_);
}
storage::Result<bool> RemoveVertex(VertexAccessor *vertex_accessor) {
return accessor_->DeleteVertex(&vertex_accessor->impl_);
storage::Result<std::optional<VertexAccessor>> RemoveVertex(VertexAccessor *vertex_accessor) {
auto res = accessor_->DeleteVertex(&vertex_accessor->impl_);
if (res.HasError()) {
return res.GetError();
}
const auto &value = res.GetValue();
if (!value) {
return std::optional<VertexAccessor>{};
}
return std::make_optional<VertexAccessor>(*value);
}
storage::PropertyId NameToProperty(const std::string_view &name) { return accessor_->NameToProperty(name); }

View File

@ -598,6 +598,35 @@ using RWType = plan::ReadWriteTypeChecker::RWType;
Interpreter::Interpreter(InterpreterContext *interpreter_context) : interpreter_context_(interpreter_context) {
MG_ASSERT(interpreter_context_, "Interpreter context must not be NULL");
// try {
// {
// auto storage_acc = interpreter_context_->db->Access();
// DbAccessor dba(&storage_acc);
// auto triggers_acc = interpreter_context_->before_commit_triggers.access();
// triggers_acc.insert(Trigger{"BeforeDelete", "UNWIND deletedVertices as u CREATE(:DELETED {id: u.id + 10})",
// &interpreter_context_->ast_cache, &dba, &interpreter_context_->antlr_lock});
// // triggers_acc.insert(Trigger{"BeforeDelete2", "UNWIND deletedVertices as u SET u.deleted = 0",
// // &interpreter_context_->ast_cache, &dba,
// // &interpreter_context_->antlr_lock});
// // triggers_acc.insert(Trigger{"BeforeDeleteProcedure", "CALL script.procedure(deletedVertices) YIELD * RETURN
// // *",
// // &interpreter_context_->ast_cache, &dba,
// // &interpreter_context_->antlr_lock});
// triggers_acc.insert(Trigger{"BeforeCreator", "UNWIND createdVertices as u SET u.before = u.id + 10",
// &interpreter_context_->ast_cache, &dba, &interpreter_context_->antlr_lock});
// }
// {
// auto storage_acc = interpreter_context->db->Access();
// DbAccessor dba(&storage_acc);
// auto triggers_acc = interpreter_context->after_commit_triggers.access();
// triggers_acc.insert(Trigger{"AfterDelete", "UNWIND deletedVertices as u CREATE(:DELETED {id: u.id + 100})",
// &interpreter_context_->ast_cache, &dba, &interpreter_context_->antlr_lock});
// triggers_acc.insert(Trigger{"AfterCreator", "UNWIND createdVertices as u SET u.after = u.id + 100",
// &interpreter_context_->ast_cache, &dba, &interpreter_context_->antlr_lock});
// }
//} catch (const utils::BasicException &e) {
// spdlog::critical("Failed to create a trigger because: {}", e.what());
//}
}
PreparedQuery Interpreter::PrepareTransactionQuery(std::string_view query_upper) {
@ -676,7 +705,8 @@ PreparedQuery PrepareCypherQuery(ParsedQuery parsed_query, std::map<std::string,
}
auto plan = CypherQueryToPlan(parsed_query.stripped_query.hash(), std::move(parsed_query.ast_storage), cypher_query,
parsed_query.parameters, &interpreter_context->plan_cache, dba);
parsed_query.parameters,
parsed_query.is_cacheable ? &interpreter_context->plan_cache : nullptr, dba);
summary->insert_or_assign("cost_estimate", plan->cost());
auto rw_type_checker = plan::ReadWriteTypeChecker();
@ -730,7 +760,7 @@ PreparedQuery PrepareExplainQuery(ParsedQuery parsed_query, std::map<std::string
auto cypher_query_plan = CypherQueryToPlan(
parsed_inner_query.stripped_query.hash(), std::move(parsed_inner_query.ast_storage), cypher_query,
parsed_inner_query.parameters, &interpreter_context->plan_cache, dba, parsed_inner_query.is_cacheable);
parsed_inner_query.parameters, parsed_inner_query.is_cacheable ? &interpreter_context->plan_cache : nullptr, dba);
std::stringstream printed_plan;
plan::PrettyPrint(*dba, &cypher_query_plan->plan(), &printed_plan);
@ -806,7 +836,7 @@ PreparedQuery PrepareProfileQuery(ParsedQuery parsed_query, bool in_explicit_tra
auto cypher_query_plan = CypherQueryToPlan(
parsed_inner_query.stripped_query.hash(), std::move(parsed_inner_query.ast_storage), cypher_query,
parsed_inner_query.parameters, &interpreter_context->plan_cache, dba, parsed_inner_query.is_cacheable);
parsed_inner_query.parameters, parsed_inner_query.is_cacheable ? &interpreter_context->plan_cache : nullptr, dba);
auto rw_type_checker = plan::ReadWriteTypeChecker();
rw_type_checker.InferRWType(const_cast<plan::LogicalOperator &>(cypher_query_plan->plan()));
@ -1404,7 +1434,7 @@ void Interpreter::Abort() {
in_explicit_transaction_ = false;
if (!db_accessor_) return;
db_accessor_->Abort();
execution_db_accessor_ = std::nullopt;
execution_db_accessor_.reset();
db_accessor_.reset();
trigger_context_.reset();
}
@ -1423,9 +1453,9 @@ void RunTriggersIndividually(const utils::SkipList<Trigger> &triggers, Interpret
trigger_context.AdaptForAccessor(&db_accessor);
try {
trigger.Execute(&interpreter_context->plan_cache, &db_accessor, &execution_memory,
*interpreter_context->tsc_frequency, interpreter_context->execution_timeout_sec,
&interpreter_context->is_shutting_down, trigger_context);
trigger.Execute(&db_accessor, &execution_memory, *interpreter_context->tsc_frequency,
interpreter_context->execution_timeout_sec, &interpreter_context->is_shutting_down,
trigger_context);
} catch (const utils::BasicException &exception) {
spdlog::warn("Trigger '{}' failed with exception:\n{}", trigger.name(), exception.what());
db_accessor.Abort();
@ -1472,9 +1502,15 @@ void Interpreter::Commit() {
for (const auto &trigger : interpreter_context_->before_commit_triggers.access()) {
spdlog::debug("Executing trigger '{}'", trigger.name());
utils::MonotonicBufferResource execution_memory{kExecutionMemoryBlockSize};
trigger.Execute(&interpreter_context_->plan_cache, &*execution_db_accessor_, &execution_memory,
*interpreter_context_->tsc_frequency, interpreter_context_->execution_timeout_sec,
&interpreter_context_->is_shutting_down, *trigger_context_);
AdvanceCommand();
try {
trigger.Execute(&*execution_db_accessor_, &execution_memory, *interpreter_context_->tsc_frequency,
interpreter_context_->execution_timeout_sec, &interpreter_context_->is_shutting_down,
*trigger_context_);
} catch (const utils::BasicException &e) {
throw utils::BasicException(
fmt::format("Trigger '{}' caused the transaction to fail.\nException: {}", trigger.name(), e.what()));
}
}
SPDLOG_DEBUG("Finished executing before commit triggers");
}

View File

@ -147,28 +147,7 @@ struct PreparedQuery {
* been passed to an `Interpreter` instance.
*/
struct InterpreterContext {
explicit InterpreterContext(storage::Storage *db) : db(db) {
// try {
// {
// auto storage_acc = db->Access();
// DbAccessor dba(&storage_acc);
// auto triggers_acc = before_commit_triggers.access();
// triggers_acc.insert(Trigger{"BeforeCreator", "UNWIND createdVertices as u SET u.before = u.id + 1",
// &ast_cache,
// &plan_cache, &dba, &antlr_lock});
// }
// {
// auto storage_acc = db->Access();
// DbAccessor dba(&storage_acc);
// auto triggers_acc = after_commit_triggers.access();
// triggers_acc.insert(Trigger{"AfterCreator", "UNWIND createdVertices as u SET u.after = u.id - 1",
// &ast_cache,
// &plan_cache, &dba, &antlr_lock});
// }
// } catch (const utils::BasicException &e) {
// spdlog::critical("Failed to create a trigger because: {}", e.what());
// }
}
explicit InterpreterContext(storage::Storage *db) : db(db) {}
storage::Storage *db;

View File

@ -1871,6 +1871,10 @@ bool Delete::DeleteCursor::Pull(Frame &frame, ExecutionContext &context) {
throw QueryRuntimeException("Unexpected error when deleting a node.");
}
}
if (context.trigger_context && res.GetValue()) {
context.trigger_context->RegisterDeletedVertex(*res.GetValue());
}
}
break;
}

View File

@ -10,24 +10,40 @@ namespace query {
namespace {
std::vector<std::pair<Identifier, trigger::IdentifierTag>> GetPredefinedIdentifiers() {
return {{{"createdVertices", false}, trigger::IdentifierTag::CREATED_VERTICES}};
return {{{"createdVertices", false}, trigger::IdentifierTag::CREATED_VERTICES},
{{"deletedVertices", false}, trigger::IdentifierTag::DELETED_VERTICES}};
}
template <typename T>
concept ConvertableToTypedValue = requires(T value) {
{TypedValue{value}};
};
template <ConvertableToTypedValue T>
TypedValue ToTypedValue(const std::vector<T> &values) {
std::vector<TypedValue> typed_values;
typed_values.reserve(values.size());
std::transform(std::begin(values), std::end(values), std::back_inserter(typed_values),
[](const auto &accessor) { return TypedValue(accessor); });
return TypedValue(typed_values);
}
} // namespace
void TriggerContext::RegisterCreatedVertex(const VertexAccessor created_vertex) {
created_vertices_.push_back(created_vertex);
}
void TriggerContext::RegisterDeletedVertex(const VertexAccessor deleted_vertex) {
deleted_vertices_.push_back(deleted_vertex);
}
TypedValue TriggerContext::GetTypedValue(const trigger::IdentifierTag tag) const {
switch (tag) {
case trigger::IdentifierTag::CREATED_VERTICES: {
std::vector<TypedValue> typed_created_vertices;
typed_created_vertices.reserve(created_vertices_.size());
std::transform(std::begin(created_vertices_), std::end(created_vertices_),
std::back_inserter(typed_created_vertices),
[](const auto &accessor) { return TypedValue(accessor); });
return TypedValue(typed_created_vertices);
}
case trigger::IdentifierTag::CREATED_VERTICES:
return ToTypedValue(created_vertices_);
case trigger::IdentifierTag::DELETED_VERTICES:
return ToTypedValue(deleted_vertices_);
}
}
@ -41,49 +57,63 @@ void TriggerContext::AdaptForAccessor(DbAccessor *accessor) {
}
}
created_vertices_.erase(it, created_vertices_.end());
// deleted_vertices_ should keep the transaction context of the transaction which deleted it
// because no other transaction can modify an object after it's deleted so it should be the
// latest state of the object
}
Trigger::Trigger(std::string name, const std::string &query, utils::SkipList<QueryCacheEntry> *query_cache,
utils::SkipList<PlanCacheEntry> *plan_cache, DbAccessor *db_accessor, utils::SpinLock *antlr_lock)
: name_(std::move(name)),
parsed_statements_{ParseQuery(query, {}, query_cache, antlr_lock)},
identifiers_{GetPredefinedIdentifiers()} {
DbAccessor *db_accessor, utils::SpinLock *antlr_lock)
: name_(std::move(name)), parsed_statements_{ParseQuery(query, {}, query_cache, antlr_lock)} {
// We check immediately if the query is valid by trying to create a plan.
GetPlan(plan_cache, db_accessor);
GetPlan(db_accessor);
}
std::shared_ptr<CachedPlan> Trigger::GetPlan(utils::SkipList<PlanCacheEntry> *plan_cache,
DbAccessor *db_accessor) const {
Trigger::TriggerPlan::TriggerPlan(std::unique_ptr<LogicalPlan> logical_plan, std::vector<IdentifierInfo> identifiers)
: cached_plan(std::move(logical_plan)), identifiers(std::move(identifiers)) {}
std::shared_ptr<Trigger::TriggerPlan> Trigger::GetPlan(DbAccessor *db_accessor) const {
std::lock_guard plan_guard{plan_lock_};
if (trigger_plan_ && !trigger_plan_->cached_plan.IsExpired()) {
return trigger_plan_;
}
auto identifiers = GetPredefinedIdentifiers();
AstStorage ast_storage;
ast_storage.properties_ = parsed_statements_.ast_storage.properties_;
ast_storage.labels_ = parsed_statements_.ast_storage.labels_;
ast_storage.edge_types_ = parsed_statements_.ast_storage.edge_types_;
std::vector<Identifier *> predefined_identifiers;
predefined_identifiers.reserve(identifiers_.size());
std::transform(identifiers_.begin(), identifiers_.end(), std::back_inserter(predefined_identifiers),
predefined_identifiers.reserve(identifiers.size());
std::transform(identifiers.begin(), identifiers.end(), std::back_inserter(predefined_identifiers),
[](auto &identifier) { return &identifier.first; });
return CypherQueryToPlan(parsed_statements_.stripped_query.hash(), std::move(ast_storage),
utils::Downcast<CypherQuery>(parsed_statements_.query), parsed_statements_.parameters,
plan_cache, db_accessor, parsed_statements_.is_cacheable, predefined_identifiers);
auto logical_plan = MakeLogicalPlan(std::move(ast_storage), utils::Downcast<CypherQuery>(parsed_statements_.query),
parsed_statements_.parameters, db_accessor, predefined_identifiers);
trigger_plan_ = std::make_shared<TriggerPlan>(std::move(logical_plan), std::move(identifiers));
return trigger_plan_;
}
void Trigger::Execute(utils::SkipList<PlanCacheEntry> *plan_cache, DbAccessor *dba,
utils::MonotonicBufferResource *execution_memory, const double tsc_frequency,
void Trigger::Execute(DbAccessor *dba, utils::MonotonicBufferResource *execution_memory, const double tsc_frequency,
const double max_execution_time_sec, std::atomic<bool> *is_shutting_down,
const TriggerContext &context) const {
auto plan = GetPlan(plan_cache, dba);
auto trigger_plan = GetPlan(dba);
MG_ASSERT(trigger_plan, "Invalid trigger plan received");
auto &[plan, identifiers] = *trigger_plan;
ExecutionContext ctx;
ctx.db_accessor = dba;
ctx.symbol_table = plan->symbol_table();
ctx.symbol_table = plan.symbol_table();
ctx.evaluation_context.timestamp =
std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now().time_since_epoch())
.count();
ctx.evaluation_context.parameters = parsed_statements_.parameters;
ctx.evaluation_context.properties = NamesToProperties(plan->ast_storage().properties_, dba);
ctx.evaluation_context.labels = NamesToLabels(plan->ast_storage().labels_, dba);
ctx.evaluation_context.properties = NamesToProperties(plan.ast_storage().properties_, dba);
ctx.evaluation_context.labels = NamesToLabels(plan.ast_storage().labels_, dba);
ctx.execution_tsc_timer = utils::TSCTimer(tsc_frequency);
ctx.max_execution_time_sec = max_execution_time_sec;
ctx.is_shutting_down = is_shutting_down;
@ -105,14 +135,14 @@ void Trigger::Execute(utils::SkipList<PlanCacheEntry> *plan_cache, DbAccessor *d
utils::PoolResource pool_memory(128, 1024, &monotonic_memory);
ctx.evaluation_context.memory = &pool_memory;
auto cursor = plan->plan().MakeCursor(execution_memory);
Frame frame{plan->symbol_table().max_position(), execution_memory};
for (const auto &[identifier, tag] : identifiers_) {
auto cursor = plan.plan().MakeCursor(execution_memory);
Frame frame{plan.symbol_table().max_position(), execution_memory};
for (const auto &[identifier, tag] : identifiers) {
if (identifier.symbol_pos_ == -1) {
continue;
}
frame[plan->symbol_table().at(identifier)] = context.GetTypedValue(tag);
frame[plan.symbol_table().at(identifier)] = context.GetTypedValue(tag);
}
while (cursor->Pull(frame, ctx))

View File

@ -1,17 +1,16 @@
#pragma once
#include "query/cypher_query_interpreter.hpp"
#include "query/db_accessor.hpp"
#include "query/frontend/ast/ast.hpp"
namespace query {
namespace trigger {
enum class IdentifierTag : uint8_t { CREATED_VERTICES };
enum class IdentifierTag : uint8_t { CREATED_VERTICES, DELETED_VERTICES };
} // namespace trigger
struct TriggerContext {
void RegisterCreatedVertex(VertexAccessor created_vertex);
void RegisterDeletedVertex(VertexAccessor deleted_vertex);
// Adapt the TriggerContext object inplace for a different DbAccessor
// (each derived accessor, e.g. VertexAccessor, gets adapted
@ -22,15 +21,15 @@ struct TriggerContext {
private:
std::vector<VertexAccessor> created_vertices_;
std::vector<VertexAccessor> deleted_vertices_;
};
struct Trigger {
explicit Trigger(std::string name, const std::string &query, utils::SkipList<QueryCacheEntry> *query_cache,
utils::SkipList<PlanCacheEntry> *plan_cache, DbAccessor *db_accessor, utils::SpinLock *antlr_lock);
DbAccessor *db_accessor, utils::SpinLock *antlr_lock);
void Execute(utils::SkipList<PlanCacheEntry> *plan_cache, DbAccessor *dba,
utils::MonotonicBufferResource *execution_memory, double tsc_frequency, double max_execution_time_sec,
std::atomic<bool> *is_shutting_down, const TriggerContext &context) const;
void Execute(DbAccessor *dba, utils::MonotonicBufferResource *execution_memory, double tsc_frequency,
double max_execution_time_sec, std::atomic<bool> *is_shutting_down, const TriggerContext &context) const;
bool operator==(const Trigger &other) const { return name_ == other.name_; }
// NOLINTNEXTLINE (modernize-use-nullptr)
@ -42,11 +41,20 @@ struct Trigger {
const auto &name() const noexcept { return name_; }
private:
std::shared_ptr<CachedPlan> GetPlan(utils::SkipList<PlanCacheEntry> *plan_cache, DbAccessor *db_accessor) const;
struct TriggerPlan {
using IdentifierInfo = std::pair<Identifier, trigger::IdentifierTag>;
explicit TriggerPlan(std::unique_ptr<LogicalPlan> logical_plan, std::vector<IdentifierInfo> identifiers);
CachedPlan cached_plan;
std::vector<IdentifierInfo> identifiers;
};
std::shared_ptr<TriggerPlan> GetPlan(DbAccessor *db_accessor) const;
std::string name_;
ParsedQuery parsed_statements_;
mutable std::vector<std::pair<Identifier, trigger::IdentifierTag>> identifiers_;
mutable utils::SpinLock plan_lock_;
mutable std::shared_ptr<TriggerPlan> trigger_plan_;
};
} // namespace query

View File

@ -1,6 +1,5 @@
#pragma once
#include "storage/v2/delta.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/view.hpp"
@ -15,6 +14,12 @@ namespace storage {
/// caller to apply the deltas.
template <typename TCallback>
inline void ApplyDeltasForRead(Transaction *transaction, const Delta *delta, View view, const TCallback &callback) {
// if the transaction is not committed, then its deltas have transaction_id for the timestamp, otherwise they have
// its commit timestamp set.
// This allows the transaction to see its changes even though it's committed.
const auto commit_timestamp = transaction->commit_timestamp
? transaction->commit_timestamp->load(std::memory_order_acquire)
: transaction->transaction_id;
while (delta != nullptr) {
auto ts = delta->timestamp->load(std::memory_order_acquire);
auto cid = delta->command_id;
@ -26,13 +31,13 @@ inline void ApplyDeltasForRead(Transaction *transaction, const Delta *delta, Vie
// We shouldn't undo our newest changes because the user requested a NEW
// view of the database.
if (view == View::NEW && ts == transaction->transaction_id && cid <= transaction->command_id) {
if (view == View::NEW && ts == commit_timestamp && cid <= transaction->command_id) {
break;
}
// We shouldn't undo our older changes because the user requested a OLD view
// of the database.
if (view == View::OLD && ts == transaction->transaction_id && cid < transaction->command_id) {
if (view == View::OLD && ts == commit_timestamp && cid < transaction->command_id) {
break;
}

View File

@ -13,6 +13,7 @@
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/replication/config.hpp"
@ -460,7 +461,7 @@ std::optional<VertexAccessor> Storage::Accessor::FindVertex(Gid gid, View view)
return VertexAccessor::Create(&*it, &transaction_, &storage_->indices_, &storage_->constraints_, config_, view);
}
Result<bool> Storage::Accessor::DeleteVertex(VertexAccessor *vertex) {
Result<std::optional<VertexAccessor>> Storage::Accessor::DeleteVertex(VertexAccessor *vertex) {
MG_ASSERT(vertex->transaction_ == &transaction_,
"VertexAccessor must be from the same transaction as the storage "
"accessor when deleting a vertex!");
@ -470,14 +471,17 @@ Result<bool> Storage::Accessor::DeleteVertex(VertexAccessor *vertex) {
if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
if (vertex_ptr->deleted) return false;
if (vertex_ptr->deleted) {
return std::optional<VertexAccessor>{};
}
if (!vertex_ptr->in_edges.empty() || !vertex_ptr->out_edges.empty()) return Error::VERTEX_HAS_EDGES;
CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag());
vertex_ptr->deleted = true;
return true;
return std::make_optional<VertexAccessor>(vertex_ptr, &transaction_, &storage_->indices_, &storage_->constraints_,
config_, true);
}
Result<bool> Storage::Accessor::DetachDeleteVertex(VertexAccessor *vertex) {
@ -862,7 +866,6 @@ utils::BasicResult<ConstraintViolation, void> Storage::Accessor::Commit(
// Release engine lock because we don't have to hold it anymore
// and emplace back could take a long time.
engine_guard.unlock();
committed_transactions.emplace_back(std::move(transaction_));
});
storage_->commit_log_->MarkFinished(start_timestamp);
@ -1046,6 +1049,8 @@ void Storage::Accessor::Abort() {
void Storage::Accessor::FinalizeTransaction() {
if (commit_timestamp_) {
storage_->commit_log_->MarkFinished(*commit_timestamp_);
storage_->committed_transactions_.WithLock(
[&](auto &committed_transactions) { committed_transactions.emplace_back(std::move(transaction_)); });
commit_timestamp_.reset();
}
}

View File

@ -250,7 +250,7 @@ class Storage final {
}
/// @throw std::bad_alloc
Result<bool> DeleteVertex(VertexAccessor *vertex);
Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex);
/// @throw std::bad_alloc
Result<bool> DetachDeleteVertex(VertexAccessor *vertex);

View File

@ -124,7 +124,7 @@ Result<bool> VertexAccessor::HasLabel(LabelId label, View view) const {
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return has_label;
}
@ -173,7 +173,7 @@ Result<std::vector<LabelId>> VertexAccessor::Labels(View view) const {
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(labels);
}
@ -257,7 +257,7 @@ Result<PropertyValue> VertexAccessor::GetProperty(PropertyId property, View view
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(value);
}
@ -307,7 +307,7 @@ Result<std::map<PropertyId, PropertyValue>> VertexAccessor::Properties(View view
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(properties);
}
@ -505,7 +505,7 @@ Result<size_t> VertexAccessor::InDegree(View view) const {
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}
@ -543,7 +543,7 @@ Result<size_t> VertexAccessor::OutDegree(View view) const {
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}

View File

@ -22,8 +22,13 @@ class VertexAccessor final {
public:
VertexAccessor(Vertex *vertex, Transaction *transaction, Indices *indices, Constraints *constraints,
Config::Items config)
: vertex_(vertex), transaction_(transaction), indices_(indices), constraints_(constraints), config_(config) {}
Config::Items config, bool for_deleted = false)
: vertex_(vertex),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config),
for_deleted_(for_deleted) {}
static std::optional<VertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view);
@ -90,6 +95,15 @@ class VertexAccessor final {
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
// if the accessor was created for a deleted vertex.
// Accessor behaves differently for some methods based on this
// flag.
// E.g. If this field is set to true, GetProperty will return the property of the node
// even though the node is deleted.
// All the write operations, and operators used for traversal (e.g. InEdges) will still
// return an error if it's called for a deleted vertex.
bool for_deleted_{false};
};
} // namespace storage

View File

@ -3,7 +3,9 @@
#include <limits>
#include "storage/v2/property_value.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/vertex_accessor.hpp"
using testing::UnorderedElementsAre;
@ -2530,3 +2532,47 @@ TEST(StorageV2, VertexVisibilityMultipleTransactions) {
acc.Abort();
}
}
// NOLINTNEXTLINE(hicpp-special-member-functions)
TEST(StorageV2, DeletedVertexAccessor) {
storage::Storage store;
const auto property = store.NameToProperty("property");
const storage::PropertyValue property_value{"property_value"};
std::optional<storage::Gid> gid;
// Create the vertex
{
auto acc = store.Access();
auto vertex = acc.CreateVertex();
gid = vertex.Gid();
ASSERT_FALSE(vertex.SetProperty(property, property_value).HasError());
ASSERT_FALSE(acc.Commit().HasError());
}
auto acc = store.Access();
auto vertex = acc.FindVertex(*gid, storage::View::OLD);
ASSERT_TRUE(vertex);
auto maybe_deleted_vertex = acc.DeleteVertex(&*vertex);
ASSERT_FALSE(maybe_deleted_vertex.HasError());
auto deleted_vertex = maybe_deleted_vertex.GetValue();
ASSERT_TRUE(deleted_vertex);
// you cannot modify deleted vertex
ASSERT_TRUE(deleted_vertex->ClearProperties().HasError());
// you can call read only methods
const auto maybe_property = deleted_vertex->GetProperty(property, storage::View::OLD);
ASSERT_FALSE(maybe_property.HasError());
ASSERT_EQ(property_value, *maybe_property);
ASSERT_FALSE(acc.Commit().HasError());
{
// you can call read only methods and get valid results even after the
// transaction which deleted the vertex committed, but only if the transaction
// accessor is still alive
const auto maybe_property = deleted_vertex->GetProperty(property, storage::View::OLD);
ASSERT_FALSE(maybe_property.HasError());
ASSERT_EQ(property_value, *maybe_property);
}
}