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) {
return accessor_->DropUniqueConstraint(label, properties);
}
void DropGraph() { return accessor_->DropGraph(); }
};
class SubgraphDbAccessor final {

View File

@ -433,4 +433,11 @@ class MultiDatabaseQueryInMulticommandTxException : public QueryException {
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

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",
&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",
&query::Query::kType};

View File

@ -3168,6 +3168,24 @@ class CoordinatorQuery : public memgraph::query::Query {
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 {
public:
static const utils::TypeInfo kType;

View File

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

View File

@ -341,6 +341,12 @@ antlrcpp::Any CypherMainVisitor::visitEdgeImportModeQuery(MemgraphCypher::EdgeIm
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) {
auto *replication_query = storage_->Create<ReplicationQuery>();
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;
/**
* @return DropGraphQuery*
*/
antlrcpp::Any visitDropGraphQuery(MemgraphCypher::DropGraphQueryContext *ctx) override;
/**
* @return CoordinatorQuery*
*/

View File

@ -158,6 +158,7 @@ query : cypherQuery
| showDatabases
| edgeImportModeQuery
| coordinatorQuery
| dropGraphQuery
;
cypherQuery : ( indexHints )? singleQuery ( cypherUnion )* ( queryMemoryLimit )? ;
@ -530,3 +531,5 @@ createEdgeIndex : CREATE EDGE INDEX ON ':' labelName ;
dropEdgeIndex : DROP EDGE INDEX ON ':' labelName ;
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(DropGraphQuery & /*drop_graph_query*/) override {}
void Visit(VersionQuery & /*version_query*/) override { AddPrivilege(AuthQuery::Privilege::STATS); }
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
// 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;
}
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 exists_active_transaction = interpreter_context->interpreters.WithLock([](const auto &interpreters_) {
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};
}
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) {
MG_ASSERT(current_db.db_acc_, "Edge Import query expects a current DB");
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<TriggerQuery>(parsed_query.query) || utils::Downcast<AnalyzeGraphQuery>(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) {
// 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 ||
utils::Downcast<EdgeIndexQuery>(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;
SetupDatabaseTransaction(could_commit, unique);
}
@ -4545,6 +4590,11 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
throw EdgeImportModeModificationInMulticommandTxException();
}
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 {
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
// 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);
}
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<TriggerInfo> info;
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);
void DropTrigger(const std::string &name);
void DropAll();
struct TriggerInfo {
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
// 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 {
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

View File

@ -30,6 +30,7 @@ struct Constraints {
~Constraints() = default;
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<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
// 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;
}
void ExistenceConstraints::DropGraphClearConstraints() { constraints_.clear(); }
} // namespace memgraph::storage

View File

@ -67,6 +67,8 @@ class ExistenceConstraints {
std::vector<std::pair<LabelId, PropertyId>> ListConstraints() const;
void LoadExistenceConstraints(const std::vector<std::string> &keys);
void DropGraphClearConstraints();
};
} // 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.");
}
void DiskEdgeTypeIndex::DropGraphClearIndices() {
spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode.");
}
} // 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,
EdgeTypeId edge_type, const Transaction &tx) override;
void DropGraphClearIndices() override;
};
} // 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
// 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;
void DropGraphClearIndices() override{};
private:
utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<LabelId>>>> entries_for_deletion;
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
// 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;
void DropGraphClearIndices() override{};
private:
utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<std::pair<LabelId, PropertyId>>>>>
entries_for_deletion;

View File

@ -183,6 +183,8 @@ class DiskStorage final : public Storage {
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) override;
void DropGraph() override{};
};
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,
EdgeRef edge_ref, EdgeTypeId edge_type, const Transaction &tx) = 0;
virtual void DropGraphClearIndices() = 0;
};
} // 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));
}
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 {
label_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,
uint64_t exact_start_timestamp) const;
void DropGraphClearIndices() const;
struct IndexStats {
std::vector<LabelId> 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
// 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 uint64_t ApproximateVertexCount(LabelId label) const = 0;
virtual void DropGraphClearIndices() = 0;
};
} // 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
// 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,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const = 0;
virtual void DropGraphClearIndices() = 0;
};
} // 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});
}
void InMemoryEdgeTypeIndex::DropGraphClearIndices() { index_.clear(); }
InMemoryEdgeTypeIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, EdgeTypeId edge_type,
View view, Storage *storage, Transaction *transaction)
: 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,
EdgeTypeId edge_type, const Transaction &tx) override;
void DropGraphClearIndices() override;
static constexpr std::size_t kEdgeTypeIdPos = 0U;
static constexpr std::size_t kVertexPos = 1U;
static constexpr std::size_t kEdgeRefPos = 2U;

View File

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

View File

@ -115,6 +115,8 @@ class InMemoryLabelIndex : public storage::LabelIndex {
bool DeleteIndexStats(const storage::LabelId &label);
void DropGraphClearIndices() override;
private:
std::map<LabelId, utils::SkipList<Entry>> index_;
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

View File

@ -154,6 +154,8 @@ class InMemoryLabelPropertyIndex : public storage::LabelPropertyIndex {
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Storage *storage,
Transaction *transaction);
void DropGraphClearIndices() override;
private:
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_;
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();
}
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) {
if (!desired_commit_timestamp) {
return timestamp_++;
@ -2337,4 +2397,9 @@ std::vector<std::pair<LabelId, PropertyId>> InMemoryStorage::InMemoryAccessor::D
return res;
}
void InMemoryStorage::InMemoryAccessor::DropGraph() {
auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
mem_storage->DropGraph();
}
} // namespace memgraph::storage

View File

@ -322,6 +322,8 @@ class InMemoryStorage final : public Storage {
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) override;
void DropGraph() override;
protected:
// TODO Better naming
/// @throw std::bad_alloc
@ -377,6 +379,8 @@ class InMemoryStorage final : public Storage {
const durability::Recovery &GetRecovery() const noexcept { return recovery_; }
void DropGraph();
private:
/// 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

View File

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

View File

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

View File

@ -296,6 +296,7 @@ class Storage {
virtual UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) = 0;
virtual void DropGraph() = 0;
auto GetTransaction() -> Transaction * { return std::addressof(transaction_); }
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
// 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_PATTERN_COMPREHENSION,
AST_COORDINATOR_QUERY,
AST_DROP_GRAPH_QUERY,
// Symbol
SYMBOL = 4000,

View File

@ -77,6 +77,7 @@ add_subdirectory(garbage_collection)
add_subdirectory(query_planning)
add_subdirectory(awesome_functions)
add_subdirectory(high_availability)
add_subdirectory(drop_graph)
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