Compare commits

...

14 Commits

Author SHA1 Message Date
Josip Mrden
8e1d86eea1 Delete trigger address comments 2024-03-22 13:54:17 +01:00
Josip Mrden
e99c7dd45b Added edge type indices to be cleaned up inside drop graph 2024-03-22 13:47:27 +01:00
Josip Mrden
84df6dcd02 Fix 2024-03-22 12:29:35 +01:00
Josip Mrden
15e7f1fa02 Add creation of unique database transaction 2024-03-22 12:14:39 +01:00
Josip Mrden
1e7dad000f Merge branch 'master' into drop-graph 2024-03-13 14:12:50 +01:00
Josipmrden
dca71d3f7b
Merge branch 'master' into drop-graph 2024-03-13 13:20:49 +01:00
Josip Mrden
95bcb9c040 Clang tidy fix 2024-03-12 17:39:04 +01:00
Josip Mrden
4a6f062ef1 Add e2e tests and fix clearing of constriants 2024-03-12 14:07:14 +01:00
Josip Mrden
57794828e3 Merge branch 'master' into drop-graph 2024-03-12 12:45:45 +01:00
Josip Mrden
fe9f6f291b Add deletion of triggers and streams 2024-03-11 15:33:58 +01:00
Josip Mrden
d055495b44 Take just main lock over the storage 2024-02-27 13:22:47 +01:00
Josip Mrden
ffb5c751c8 Merge branch 'master' into drop-graph 2024-02-27 13:04:45 +01:00
Josip Mrden
6f47a860a4 Typo 2024-02-16 17:31:02 +01:00
Josip Mrden
d89193dd6d Drop graph functionality 2024-02-15 15:57:16 +01:00
46 changed files with 361 additions and 11 deletions

View File

@ -736,6 +736,8 @@ class DbAccessor final {
const std::set<storage::PropertyId> &properties) { const std::set<storage::PropertyId> &properties) {
return accessor_->DropUniqueConstraint(label, properties); return accessor_->DropUniqueConstraint(label, properties);
} }
void DropGraph() { return accessor_->DropGraph(); }
}; };
class SubgraphDbAccessor final { class SubgraphDbAccessor final {

View File

@ -433,4 +433,11 @@ class MultiDatabaseQueryInMulticommandTxException : public QueryException {
SPECIALIZE_GET_EXCEPTION_NAME(MultiDatabaseQueryInMulticommandTxException) SPECIALIZE_GET_EXCEPTION_NAME(MultiDatabaseQueryInMulticommandTxException)
}; };
class DropGraphInMulticommandTxException : public QueryException {
public:
DropGraphInMulticommandTxException()
: QueryException("Drop graph can not be executed in multicommand transactions.") {}
SPECIALIZE_GET_EXCEPTION_NAME(DropGraphInMulticommandTxException)
};
} // namespace memgraph::query } // namespace memgraph::query

View File

@ -245,6 +245,9 @@ constexpr utils::TypeInfo query::ReplicationQuery::kType{utils::TypeId::AST_REPL
constexpr utils::TypeInfo query::CoordinatorQuery::kType{utils::TypeId::AST_COORDINATOR_QUERY, "CoordinatorQuery", constexpr utils::TypeInfo query::CoordinatorQuery::kType{utils::TypeId::AST_COORDINATOR_QUERY, "CoordinatorQuery",
&query::Query::kType}; &query::Query::kType};
constexpr utils::TypeInfo query::DropGraphQuery::kType{utils::TypeId::AST_DROP_GRAPH_QUERY, "DropGraphQuery",
&query::Query::kType};
constexpr utils::TypeInfo query::LockPathQuery::kType{utils::TypeId::AST_LOCK_PATH_QUERY, "LockPathQuery", constexpr utils::TypeInfo query::LockPathQuery::kType{utils::TypeId::AST_LOCK_PATH_QUERY, "LockPathQuery",
&query::Query::kType}; &query::Query::kType};

View File

@ -3168,6 +3168,24 @@ class CoordinatorQuery : public memgraph::query::Query {
friend class AstStorage; friend class AstStorage;
}; };
class DropGraphQuery : public memgraph::query::Query {
public:
static const utils::TypeInfo kType;
const utils::TypeInfo &GetTypeInfo() const override { return kType; }
DropGraphQuery() = default;
DEFVISITABLE(QueryVisitor<void>);
DropGraphQuery *Clone(AstStorage *storage) const override {
auto *object = storage->Create<DropGraphQuery>();
return object;
}
private:
friend class AstStorage;
};
class EdgeImportModeQuery : public memgraph::query::Query { class EdgeImportModeQuery : public memgraph::query::Query {
public: public:
static const utils::TypeInfo kType; static const utils::TypeInfo kType;

View File

@ -110,6 +110,7 @@ class ShowDatabasesQuery;
class EdgeImportModeQuery; class EdgeImportModeQuery;
class PatternComprehension; class PatternComprehension;
class CoordinatorQuery; class CoordinatorQuery;
class DropGraphQuery;
using TreeCompositeVisitor = utils::CompositeVisitor< using TreeCompositeVisitor = utils::CompositeVisitor<
SingleQuery, CypherUnion, NamedExpression, OrOperator, XorOperator, AndOperator, NotOperator, AdditionOperator, SingleQuery, CypherUnion, NamedExpression, OrOperator, XorOperator, AndOperator, NotOperator, AdditionOperator,
@ -149,6 +150,6 @@ class QueryVisitor
LockPathQuery, FreeMemoryQuery, TriggerQuery, IsolationLevelQuery, CreateSnapshotQuery, LockPathQuery, FreeMemoryQuery, TriggerQuery, IsolationLevelQuery, CreateSnapshotQuery,
StreamQuery, SettingQuery, VersionQuery, ShowConfigQuery, TransactionQueueQuery, StreamQuery, SettingQuery, VersionQuery, ShowConfigQuery, TransactionQueueQuery,
StorageModeQuery, AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery, StorageModeQuery, AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery,
EdgeImportModeQuery, CoordinatorQuery> {}; EdgeImportModeQuery, CoordinatorQuery, DropGraphQuery> {};
} // namespace memgraph::query } // namespace memgraph::query

View File

@ -341,6 +341,12 @@ antlrcpp::Any CypherMainVisitor::visitEdgeImportModeQuery(MemgraphCypher::EdgeIm
return edge_import_mode_query; return edge_import_mode_query;
} }
antlrcpp::Any CypherMainVisitor::visitDropGraphQuery(MemgraphCypher::DropGraphQueryContext * /*ctx*/) {
auto *drop_graph_query = storage_->Create<DropGraphQuery>();
query_ = drop_graph_query;
return drop_graph_query;
}
antlrcpp::Any CypherMainVisitor::visitSetReplicationRole(MemgraphCypher::SetReplicationRoleContext *ctx) { antlrcpp::Any CypherMainVisitor::visitSetReplicationRole(MemgraphCypher::SetReplicationRoleContext *ctx) {
auto *replication_query = storage_->Create<ReplicationQuery>(); auto *replication_query = storage_->Create<ReplicationQuery>();
replication_query->action_ = ReplicationQuery::Action::SET_REPLICATION_ROLE; replication_query->action_ = ReplicationQuery::Action::SET_REPLICATION_ROLE;

View File

@ -243,6 +243,11 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
*/ */
antlrcpp::Any visitCoordinatorQuery(MemgraphCypher::CoordinatorQueryContext *ctx) override; antlrcpp::Any visitCoordinatorQuery(MemgraphCypher::CoordinatorQueryContext *ctx) override;
/**
* @return DropGraphQuery*
*/
antlrcpp::Any visitDropGraphQuery(MemgraphCypher::DropGraphQueryContext *ctx) override;
/** /**
* @return CoordinatorQuery* * @return CoordinatorQuery*
*/ */

View File

@ -158,6 +158,7 @@ query : cypherQuery
| showDatabases | showDatabases
| edgeImportModeQuery | edgeImportModeQuery
| coordinatorQuery | coordinatorQuery
| dropGraphQuery
; ;
cypherQuery : ( indexHints )? singleQuery ( cypherUnion )* ( queryMemoryLimit )? ; cypherQuery : ( indexHints )? singleQuery ( cypherUnion )* ( queryMemoryLimit )? ;
@ -530,3 +531,5 @@ createEdgeIndex : CREATE EDGE INDEX ON ':' labelName ;
dropEdgeIndex : DROP EDGE INDEX ON ':' labelName ; dropEdgeIndex : DROP EDGE INDEX ON ':' labelName ;
edgeIndexQuery : createEdgeIndex | dropEdgeIndex ; edgeIndexQuery : createEdgeIndex | dropEdgeIndex ;
dropGraphQuery : DROP GRAPH ;

View File

@ -99,6 +99,8 @@ class PrivilegeExtractor : public QueryVisitor<void>, public HierarchicalTreeVis
void Visit(EdgeImportModeQuery & /*edge_import_mode_query*/) override {} void Visit(EdgeImportModeQuery & /*edge_import_mode_query*/) override {}
void Visit(DropGraphQuery & /*drop_graph_query*/) override {}
void Visit(VersionQuery & /*version_query*/) override { AddPrivilege(AuthQuery::Privilege::STATS); } void Visit(VersionQuery & /*version_query*/) override { AddPrivilege(AuthQuery::Privilege::STATS); }
void Visit(MultiDatabaseQuery &query) override { void Visit(MultiDatabaseQuery &query) override {

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source

View File

@ -3321,6 +3321,27 @@ Callback SwitchMemoryDevice(storage::StorageMode current_mode, storage::StorageM
return callback; return callback;
} }
Callback DropGraph(memgraph::dbms::DatabaseAccess &db, DbAccessor *dba) {
Callback callback;
callback.fn = [&db, dba]() mutable {
auto storage_mode = db->GetStorageMode();
if (storage_mode != storage::StorageMode::IN_MEMORY_ANALYTICAL) {
throw utils::BasicException("Drop graph can not be used without IN_MEMORY_ANALYTICAL storage mode!");
}
dba->DropGraph();
auto *trigger_store = db->trigger_store();
trigger_store->DropAll();
auto *streams = db->streams();
streams->DropAll();
return std::vector<std::vector<TypedValue>>();
};
return callback;
}
bool ActiveTransactionsExist(InterpreterContext *interpreter_context) { bool ActiveTransactionsExist(InterpreterContext *interpreter_context) {
bool exists_active_transaction = interpreter_context->interpreters.WithLock([](const auto &interpreters_) { bool exists_active_transaction = interpreter_context->interpreters.WithLock([](const auto &interpreters_) {
return std::any_of(interpreters_.begin(), interpreters_.end(), [](const auto &interpreter) { return std::any_of(interpreters_.begin(), interpreters_.end(), [](const auto &interpreter) {
@ -3373,6 +3394,28 @@ PreparedQuery PrepareStorageModeQuery(ParsedQuery parsed_query, const bool in_ex
RWType::NONE}; RWType::NONE};
} }
PreparedQuery PrepareDropGraphQuery(ParsedQuery parsed_query, CurrentDB &current_db) {
MG_ASSERT(current_db.db_acc_, "Drop graph query expects a current DB");
memgraph::dbms::DatabaseAccess &db_acc = *current_db.db_acc_;
MG_ASSERT(current_db.db_transactional_accessor_, "Drop graph query expects a current DB transaction");
auto *dba = &*current_db.execution_db_accessor_;
auto *drop_graph_query = utils::Downcast<DropGraphQuery>(parsed_query.query);
MG_ASSERT(drop_graph_query);
std::function<void()> callback = DropGraph(db_acc, dba).fn;
return PreparedQuery{{},
std::move(parsed_query.required_privileges),
[callback = std::move(callback)](AnyStream * /*stream*/,
std::optional<int> /*n*/) -> std::optional<QueryHandlerResult> {
callback();
return QueryHandlerResult::COMMIT;
},
RWType::NONE};
}
PreparedQuery PrepareEdgeImportModeQuery(ParsedQuery parsed_query, CurrentDB &current_db) { PreparedQuery PrepareEdgeImportModeQuery(ParsedQuery parsed_query, CurrentDB &current_db) {
MG_ASSERT(current_db.db_acc_, "Edge Import query expects a current DB"); MG_ASSERT(current_db.db_acc_, "Edge Import query expects a current DB");
storage::Storage *storage = current_db.db_acc_->get()->storage(); storage::Storage *storage = current_db.db_acc_->get()->storage();
@ -4421,7 +4464,8 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
utils::Downcast<ProfileQuery>(parsed_query.query) || utils::Downcast<DumpQuery>(parsed_query.query) || utils::Downcast<ProfileQuery>(parsed_query.query) || utils::Downcast<DumpQuery>(parsed_query.query) ||
utils::Downcast<TriggerQuery>(parsed_query.query) || utils::Downcast<AnalyzeGraphQuery>(parsed_query.query) || utils::Downcast<TriggerQuery>(parsed_query.query) || utils::Downcast<AnalyzeGraphQuery>(parsed_query.query) ||
utils::Downcast<IndexQuery>(parsed_query.query) || utils::Downcast<EdgeIndexQuery>(parsed_query.query) || utils::Downcast<IndexQuery>(parsed_query.query) || utils::Downcast<EdgeIndexQuery>(parsed_query.query) ||
utils::Downcast<DatabaseInfoQuery>(parsed_query.query) || utils::Downcast<ConstraintQuery>(parsed_query.query); utils::Downcast<DatabaseInfoQuery>(parsed_query.query) ||
utils::Downcast<ConstraintQuery>(parsed_query.query) || utils::Downcast<DropGraphQuery>(parsed_query.query);
if (!in_explicit_transaction_ && requires_db_transaction) { if (!in_explicit_transaction_ && requires_db_transaction) {
// TODO: ATM only a single database, will change when we have multiple database transactions // TODO: ATM only a single database, will change when we have multiple database transactions
@ -4429,6 +4473,7 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
bool unique = utils::Downcast<IndexQuery>(parsed_query.query) != nullptr || bool unique = utils::Downcast<IndexQuery>(parsed_query.query) != nullptr ||
utils::Downcast<EdgeIndexQuery>(parsed_query.query) != nullptr || utils::Downcast<EdgeIndexQuery>(parsed_query.query) != nullptr ||
utils::Downcast<ConstraintQuery>(parsed_query.query) != nullptr || utils::Downcast<ConstraintQuery>(parsed_query.query) != nullptr ||
utils::Downcast<DropGraphQuery>(parsed_query.query) != nullptr ||
upper_case_query.find(kSchemaAssert) != std::string::npos; upper_case_query.find(kSchemaAssert) != std::string::npos;
SetupDatabaseTransaction(could_commit, unique); SetupDatabaseTransaction(could_commit, unique);
} }
@ -4545,6 +4590,11 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
throw EdgeImportModeModificationInMulticommandTxException(); throw EdgeImportModeModificationInMulticommandTxException();
} }
prepared_query = PrepareEdgeImportModeQuery(std::move(parsed_query), current_db_); prepared_query = PrepareEdgeImportModeQuery(std::move(parsed_query), current_db_);
} else if (utils::Downcast<DropGraphQuery>(parsed_query.query)) {
if (in_explicit_transaction_) {
throw DropGraphInMulticommandTxException();
}
prepared_query = PrepareDropGraphQuery(std::move(parsed_query), current_db_);
} else { } else {
LOG_FATAL("Should not get here -- unknown query type!"); LOG_FATAL("Should not get here -- unknown query type!");
} }

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source

View File

@ -432,6 +432,14 @@ void TriggerStore::DropTrigger(const std::string &name) {
storage_.Delete(name); storage_.Delete(name);
} }
void TriggerStore::DropAll() {
std::unique_lock store_guard{store_lock_};
storage_.DeletePrefix("");
before_commit_triggers_.clear();
after_commit_triggers_.clear();
}
std::vector<TriggerStore::TriggerInfo> TriggerStore::GetTriggerInfo() const { std::vector<TriggerStore::TriggerInfo> TriggerStore::GetTriggerInfo() const {
std::vector<TriggerInfo> info; std::vector<TriggerInfo> info;
info.reserve(before_commit_triggers_.size() + after_commit_triggers_.size()); info.reserve(before_commit_triggers_.size() + after_commit_triggers_.size());

View File

@ -90,6 +90,7 @@ struct TriggerStore {
const InterpreterConfig::Query &query_config, std::shared_ptr<QueryUserOrRole> owner); const InterpreterConfig::Query &query_config, std::shared_ptr<QueryUserOrRole> owner);
void DropTrigger(const std::string &name); void DropTrigger(const std::string &name);
void DropAll();
struct TriggerInfo { struct TriggerInfo {
std::string name; std::string name;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -33,4 +33,10 @@ Constraints::Constraints(const Config &config, StorageMode storage_mode) {
void Constraints::AbortEntries(std::span<Vertex const *const> vertices, uint64_t exact_start_timestamp) const { void Constraints::AbortEntries(std::span<Vertex const *const> vertices, uint64_t exact_start_timestamp) const {
static_cast<InMemoryUniqueConstraints *>(unique_constraints_.get())->AbortEntries(vertices, exact_start_timestamp); static_cast<InMemoryUniqueConstraints *>(unique_constraints_.get())->AbortEntries(vertices, exact_start_timestamp);
} }
void Constraints::DropGraphClearConstraints() const {
static_cast<InMemoryUniqueConstraints *>(unique_constraints_.get())->DropGraphClearConstraints();
existence_constraints_->DropGraphClearConstraints();
}
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -30,6 +30,7 @@ struct Constraints {
~Constraints() = default; ~Constraints() = default;
void AbortEntries(std::span<Vertex const *const> vertices, uint64_t exact_start_timestamp) const; void AbortEntries(std::span<Vertex const *const> vertices, uint64_t exact_start_timestamp) const;
void DropGraphClearConstraints() const;
std::unique_ptr<ExistenceConstraints> existence_constraints_; std::unique_ptr<ExistenceConstraints> existence_constraints_;
std::unique_ptr<UniqueConstraints> unique_constraints_; std::unique_ptr<UniqueConstraints> unique_constraints_;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -122,4 +122,6 @@ std::optional<ConstraintViolation> ExistenceConstraints::SingleThreadConstraintV
return std::nullopt; return std::nullopt;
} }
void ExistenceConstraints::DropGraphClearConstraints() { constraints_.clear(); }
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -67,6 +67,8 @@ class ExistenceConstraints {
std::vector<std::pair<LabelId, PropertyId>> ListConstraints() const; std::vector<std::pair<LabelId, PropertyId>> ListConstraints() const;
void LoadExistenceConstraints(const std::vector<std::string> &keys); void LoadExistenceConstraints(const std::vector<std::string> &keys);
void DropGraphClearConstraints();
}; };
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -46,4 +46,8 @@ void DiskEdgeTypeIndex::UpdateOnEdgeModification(Vertex * /*old_from*/, Vertex *
spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode."); spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode.");
} }
void DiskEdgeTypeIndex::DropGraphClearIndices() {
spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode.");
}
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -30,6 +30,8 @@ class DiskEdgeTypeIndex : public storage::EdgeTypeIndex {
void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, EdgeRef edge_ref, void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, EdgeRef edge_ref,
EdgeTypeId edge_type, const Transaction &tx) override; EdgeTypeId edge_type, const Transaction &tx) override;
void DropGraphClearIndices() override;
}; };
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -58,6 +58,8 @@ class DiskLabelIndex : public storage::LabelIndex {
std::unordered_set<LabelId> GetInfo() const; std::unordered_set<LabelId> GetInfo() const;
void DropGraphClearIndices() override{};
private: private:
utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<LabelId>>>> entries_for_deletion; utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<LabelId>>>> entries_for_deletion;
std::unordered_set<LabelId> index_; std::unordered_set<LabelId> index_;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -62,6 +62,8 @@ class DiskLabelPropertyIndex : public storage::LabelPropertyIndex {
std::set<std::pair<LabelId, PropertyId>> GetInfo() const; std::set<std::pair<LabelId, PropertyId>> GetInfo() const;
void DropGraphClearIndices() override{};
private: private:
utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<std::pair<LabelId, PropertyId>>>>> utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<std::pair<LabelId, PropertyId>>>>>
entries_for_deletion; entries_for_deletion;

View File

@ -183,6 +183,8 @@ class DiskStorage final : public Storage {
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) override; const std::set<PropertyId> &properties) override;
void DropGraph() override{};
}; };
using Storage::Access; using Storage::Access;

View File

@ -41,6 +41,8 @@ class EdgeTypeIndex {
virtual void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, virtual void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to,
EdgeRef edge_ref, EdgeTypeId edge_type, const Transaction &tx) = 0; EdgeRef edge_ref, EdgeTypeId edge_type, const Transaction &tx) = 0;
virtual void DropGraphClearIndices() = 0;
}; };
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -42,6 +42,12 @@ void Indices::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp, std:
->RemoveObsoleteEntries(oldest_active_start_timestamp, std::move(token)); ->RemoveObsoleteEntries(oldest_active_start_timestamp, std::move(token));
} }
void Indices::DropGraphClearIndices() const {
static_cast<InMemoryLabelIndex *>(label_index_.get())->DropGraphClearIndices();
static_cast<InMemoryLabelPropertyIndex *>(label_property_index_.get())->DropGraphClearIndices();
static_cast<InMemoryEdgeTypeIndex *>(edge_type_index_.get())->DropGraphClearIndices();
}
void Indices::UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx) const { void Indices::UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx) const {
label_index_->UpdateOnAddLabel(label, vertex, tx); label_index_->UpdateOnAddLabel(label, vertex, tx);
label_property_index_->UpdateOnAddLabel(label, vertex, tx); label_property_index_->UpdateOnAddLabel(label, vertex, tx);

View File

@ -44,6 +44,8 @@ struct Indices {
void AbortEntries(LabelId label, std::span<std::pair<PropertyValue, Vertex *> const> vertices, void AbortEntries(LabelId label, std::span<std::pair<PropertyValue, Vertex *> const> vertices,
uint64_t exact_start_timestamp) const; uint64_t exact_start_timestamp) const;
void DropGraphClearIndices() const;
struct IndexStats { struct IndexStats {
std::vector<LabelId> label; std::vector<LabelId> label;
LabelPropertyIndex::IndexStats property_label; LabelPropertyIndex::IndexStats property_label;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -40,6 +40,8 @@ class LabelIndex {
virtual std::vector<LabelId> ListIndices() const = 0; virtual std::vector<LabelId> ListIndices() const = 0;
virtual uint64_t ApproximateVertexCount(LabelId label) const = 0; virtual uint64_t ApproximateVertexCount(LabelId label) const = 0;
virtual void DropGraphClearIndices() = 0;
}; };
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -53,6 +53,8 @@ class LabelPropertyIndex {
virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property, virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower, const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const = 0; const std::optional<utils::Bound<PropertyValue>> &upper) const = 0;
virtual void DropGraphClearIndices() = 0;
}; };
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -215,6 +215,8 @@ void InMemoryEdgeTypeIndex::UpdateOnEdgeModification(Vertex *old_from, Vertex *o
acc.insert(Entry{new_from, new_to, edge_ref.ptr, tx.start_timestamp}); acc.insert(Entry{new_from, new_to, edge_ref.ptr, tx.start_timestamp});
} }
void InMemoryEdgeTypeIndex::DropGraphClearIndices() { index_.clear(); }
InMemoryEdgeTypeIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, EdgeTypeId edge_type, InMemoryEdgeTypeIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, EdgeTypeId edge_type,
View view, Storage *storage, Transaction *transaction) View view, Storage *storage, Transaction *transaction)
: index_accessor_(std::move(index_accessor)), : index_accessor_(std::move(index_accessor)),

View File

@ -61,6 +61,8 @@ class InMemoryEdgeTypeIndex : public storage::EdgeTypeIndex {
void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, EdgeRef edge_ref, void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, EdgeRef edge_ref,
EdgeTypeId edge_type, const Transaction &tx) override; EdgeTypeId edge_type, const Transaction &tx) override;
void DropGraphClearIndices() override;
static constexpr std::size_t kEdgeTypeIdPos = 0U; static constexpr std::size_t kEdgeTypeIdPos = 0U;
static constexpr std::size_t kVertexPos = 1U; static constexpr std::size_t kVertexPos = 1U;
static constexpr std::size_t kEdgeRefPos = 2U; static constexpr std::size_t kEdgeRefPos = 2U;

View File

@ -235,4 +235,10 @@ std::vector<LabelId> InMemoryLabelIndex::Analysis() const {
} }
return res; return res;
} }
void InMemoryLabelIndex::DropGraphClearIndices() {
index_.clear();
stats_->clear();
}
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -115,6 +115,8 @@ class InMemoryLabelIndex : public storage::LabelIndex {
bool DeleteIndexStats(const storage::LabelId &label); bool DeleteIndexStats(const storage::LabelId &label);
void DropGraphClearIndices() override;
private: private:
std::map<LabelId, utils::SkipList<Entry>> index_; std::map<LabelId, utils::SkipList<Entry>> index_;
utils::Synchronized<std::map<LabelId, storage::LabelIndexStats>, utils::ReadPrioritizedRWLock> stats_; utils::Synchronized<std::map<LabelId, storage::LabelIndexStats>, utils::ReadPrioritizedRWLock> stats_;

View File

@ -498,4 +498,11 @@ void InMemoryLabelPropertyIndex::AbortEntries(LabelId label,
} }
} }
} }
void InMemoryLabelPropertyIndex::DropGraphClearIndices() {
index_.clear();
indices_by_property_.clear();
stats_->clear();
}
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -154,6 +154,8 @@ class InMemoryLabelPropertyIndex : public storage::LabelPropertyIndex {
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Storage *storage, const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Storage *storage,
Transaction *transaction); Transaction *transaction);
void DropGraphClearIndices() override;
private: private:
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_; std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_;
std::unordered_map<PropertyId, std::unordered_map<LabelId, utils::SkipList<Entry> *>> indices_by_property_; std::unordered_map<PropertyId, std::unordered_map<LabelId, utils::SkipList<Entry> *>> indices_by_property_;

View File

@ -2214,6 +2214,66 @@ void InMemoryStorage::FreeMemory(std::unique_lock<utils::ResourceLock> main_guar
edges_.run_gc(); edges_.run_gc();
} }
void InMemoryStorage::DropGraph() {
auto const unlink_remove_clear = [&](std::deque<Delta> &deltas) {
for (auto &delta : deltas) {
auto prev = delta.prev.Get();
switch (prev.type) {
case PreviousPtr::Type::NULLPTR:
case PreviousPtr::Type::DELTA:
break;
case PreviousPtr::Type::VERTEX: {
// safe because no other txn can be reading this while we have engine lock
auto &vertex = *prev.vertex;
vertex.delta = nullptr;
if (vertex.deleted) {
DMG_ASSERT(delta.action == Delta::Action::RECREATE_OBJECT);
}
break;
}
case PreviousPtr::Type::EDGE: {
// safe because no other txn can be reading this while we have engine lock
auto &edge = *prev.edge;
edge.delta = nullptr;
if (edge.deleted) {
DMG_ASSERT(delta.action == Delta::Action::RECREATE_OBJECT);
}
break;
}
}
}
// delete deltas
deltas.clear();
};
// STEP 1) ensure everything in GC is gone
// 1.a) old garbage_undo_buffers are safe to remove
// we are the only transaction, no one is reading those unlinked deltas
garbage_undo_buffers_.WithLock([&](auto &garbage_undo_buffers) { garbage_undo_buffers.clear(); });
// 1.b.0) old committed_transactions_ need mininal unlinking + remove + clear
// must be done before this transactions delta unlinking
auto linked_undo_buffers = std::list<GCDeltas>{};
committed_transactions_.WithLock(
[&](auto &committed_transactions) { committed_transactions.swap(linked_undo_buffers); });
// 1.b.1) unlink, gathering the removals
for (auto &gc_deltas : linked_undo_buffers) {
unlink_remove_clear(gc_deltas.deltas_);
}
// 1.b.2) clear the list of deltas deques
linked_undo_buffers.clear();
indices_.DropGraphClearIndices();
constraints_.DropGraphClearConstraints();
vertices_.clear();
edges_.clear();
memory::PurgeUnusedMemory();
}
uint64_t InMemoryStorage::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) { uint64_t InMemoryStorage::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) {
if (!desired_commit_timestamp) { if (!desired_commit_timestamp) {
return timestamp_++; return timestamp_++;
@ -2337,4 +2397,9 @@ std::vector<std::pair<LabelId, PropertyId>> InMemoryStorage::InMemoryAccessor::D
return res; return res;
} }
void InMemoryStorage::InMemoryAccessor::DropGraph() {
auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
mem_storage->DropGraph();
}
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -322,6 +322,8 @@ class InMemoryStorage final : public Storage {
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) override; const std::set<PropertyId> &properties) override;
void DropGraph() override;
protected: protected:
// TODO Better naming // TODO Better naming
/// @throw std::bad_alloc /// @throw std::bad_alloc
@ -377,6 +379,8 @@ class InMemoryStorage final : public Storage {
const durability::Recovery &GetRecovery() const noexcept { return recovery_; } const durability::Recovery &GetRecovery() const noexcept { return recovery_; }
void DropGraph();
private: private:
/// The force parameter determines the behaviour of the garbage collector. /// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't /// If it's set to true, it will behave as a global operation, i.e. it can't

View File

@ -524,4 +524,8 @@ void InMemoryUniqueConstraints::Clear() {
} }
bool InMemoryUniqueConstraints::empty() const { return constraints_.empty() && constraints_by_label_.empty(); } bool InMemoryUniqueConstraints::empty() const { return constraints_.empty() && constraints_by_label_.empty(); }
void InMemoryUniqueConstraints::DropGraphClearConstraints() {
constraints_.clear();
constraints_by_label_.clear();
}
} // namespace memgraph::storage } // namespace memgraph::storage

View File

@ -129,6 +129,8 @@ class InMemoryUniqueConstraints : public UniqueConstraints {
void Clear() override; void Clear() override;
void DropGraphClearConstraints();
static std::variant<MultipleThreadsConstraintValidation, SingleThreadConstraintValidation> GetCreationFunction( static std::variant<MultipleThreadsConstraintValidation, SingleThreadConstraintValidation> GetCreationFunction(
const std::optional<durability::ParallelizedSchemaCreationInfo> &); const std::optional<durability::ParallelizedSchemaCreationInfo> &);

View File

@ -296,6 +296,7 @@ class Storage {
virtual UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, virtual UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) = 0; const std::set<PropertyId> &properties) = 0;
virtual void DropGraph() = 0;
auto GetTransaction() -> Transaction * { return std::addressof(transaction_); } auto GetTransaction() -> Transaction * { return std::addressof(transaction_); }
protected: protected:

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd. // Copyright 2024 Memgraph Ltd.
// //
// Use of this software is governed by the Business Source License // Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source

View File

@ -230,6 +230,7 @@ enum class TypeId : uint64_t {
AST_EDGE_IMPORT_MODE_QUERY, AST_EDGE_IMPORT_MODE_QUERY,
AST_PATTERN_COMPREHENSION, AST_PATTERN_COMPREHENSION,
AST_COORDINATOR_QUERY, AST_COORDINATOR_QUERY,
AST_DROP_GRAPH_QUERY,
// Symbol // Symbol
SYMBOL = 4000, SYMBOL = 4000,

View File

@ -77,6 +77,7 @@ add_subdirectory(garbage_collection)
add_subdirectory(query_planning) add_subdirectory(query_planning)
add_subdirectory(awesome_functions) add_subdirectory(awesome_functions)
add_subdirectory(high_availability) add_subdirectory(high_availability)
add_subdirectory(drop_graph)
add_subdirectory(replication_experimental) add_subdirectory(replication_experimental)

View File

@ -0,0 +1,6 @@
function(copy_drop_graph_e2e_python_files FILE_NAME)
copy_e2e_python_files(drop_graph ${FILE_NAME})
endfunction()
copy_drop_graph_e2e_python_files(common.py)
copy_drop_graph_e2e_python_files(drop_graph.py)

View File

@ -0,0 +1,40 @@
# Copyright 2023 Memgraph Ltd.
#
# Use of this software is governed by the Business Source License
# included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
# License, and you may not use this file except in compliance with the Business Source License.
#
# As of the Change Date specified in that file, in accordance with
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0, included in the file
# licenses/APL.txt.
import pytest
from gqlalchemy import Memgraph
def get_results_length(memgraph, query):
return len(list(memgraph.execute_and_fetch(query)))
@pytest.fixture
def memgraph(**kwargs) -> Memgraph:
memgraph = Memgraph()
memgraph.execute("STORAGE MODE IN_MEMORY_TRANSACTIONAL")
memgraph.drop_indexes()
memgraph.ensure_constraints([])
trigger_names = [x["trigger name"] for x in list(memgraph.execute_and_fetch("SHOW TRIGGERS"))]
for trigger_name in trigger_names:
memgraph.execute(f"DROP TRIGGER {trigger_name}")
yield memgraph
memgraph.execute("STORAGE MODE IN_MEMORY_TRANSACTIONAL")
memgraph.drop_indexes()
memgraph.ensure_constraints([])
trigger_names = [x["trigger name"] for x in list(memgraph.execute_and_fetch("SHOW TRIGGERS"))]
for trigger_name in trigger_names:
memgraph.execute(f"DROP TRIGGER {trigger_name}")
memgraph.drop_database()

View File

@ -0,0 +1,48 @@
# Copyright 2023 Memgraph Ltd.
#
# Use of this software is governed by the Business Source License
# included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
# License, and you may not use this file except in compliance with the Business Source License.
#
# As of the Change Date specified in that file, in accordance with
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0, included in the file
# licenses/APL.txt.
import sys
import pytest
from common import get_results_length, memgraph
from gqlalchemy import GQLAlchemyError
def test_create_everything_then_drop_graph(memgraph):
memgraph.execute("CREATE (:Node {id:1})-[:TYPE {id:2}]->(:Node {id:3})")
memgraph.execute("CREATE INDEX ON :Node")
memgraph.execute("CREATE INDEX ON :Node(id)")
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.id IS UNIQUE;")
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.id);")
memgraph.execute("CREATE TRIGGER t1 ON () UPDATE BEFORE COMMIT EXECUTE RETURN 1")
memgraph.execute("CREATE TRIGGER t2 ON () UPDATE AFTER COMMIT EXECUTE RETURN 1")
assert get_results_length(memgraph, "MATCH (n) RETURN n") == 2
assert get_results_length(memgraph, "MATCH (n)-[r]->(m) RETURN r") == 1
assert get_results_length(memgraph, "SHOW INDEX INFO") == 2
assert get_results_length(memgraph, "SHOW CONSTRAINT INFO") == 2
assert get_results_length(memgraph, "SHOW TRIGGERS") == 2
with pytest.raises(GQLAlchemyError):
memgraph.execute("DROP GRAPH")
memgraph.execute("STORAGE MODE IN_MEMORY_ANALYTICAL")
memgraph.execute("DROP GRAPH")
assert get_results_length(memgraph, "MATCH (n) RETURN n") == 0
assert get_results_length(memgraph, "MATCH (n)-[r]->(m) RETURN r") == 0
assert get_results_length(memgraph, "SHOW INDEX INFO") == 0
assert get_results_length(memgraph, "SHOW CONSTRAINT INFO") == 0
assert get_results_length(memgraph, "SHOW TRIGGERS") == 0
if __name__ == "__main__":
sys.exit(pytest.main([__file__, "-rA"]))

View File

@ -0,0 +1,14 @@
drop_graph_cluster: &drop_graph_cluster
cluster:
main:
args: ["--bolt-port", "7687", "--log-level=TRACE"]
log_file: "drop_graph.log"
setup_queries: []
validation_queries: []
workloads:
- name: "Drop graph"
binary: "tests/e2e/pytest_runner.sh"
args: ["drop_graph/drop_graph.py"]
<<: *drop_graph_cluster