Compare commits
14 Commits
master
...
drop-graph
Author | SHA1 | Date | |
---|---|---|---|
|
8e1d86eea1 | ||
|
e99c7dd45b | ||
|
84df6dcd02 | ||
|
15e7f1fa02 | ||
|
1e7dad000f | ||
|
dca71d3f7b | ||
|
95bcb9c040 | ||
|
4a6f062ef1 | ||
|
57794828e3 | ||
|
fe9f6f291b | ||
|
d055495b44 | ||
|
ffb5c751c8 | ||
|
6f47a860a4 | ||
|
d89193dd6d |
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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};
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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*
|
||||
*/
|
||||
|
@ -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 ;
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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 ¤t_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 ¤t_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!");
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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());
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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_;
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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_;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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)),
|
||||
|
@ -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;
|
||||
|
@ -235,4 +235,10 @@ std::vector<LabelId> InMemoryLabelIndex::Analysis() const {
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
void InMemoryLabelIndex::DropGraphClearIndices() {
|
||||
index_.clear();
|
||||
stats_->clear();
|
||||
}
|
||||
|
||||
} // namespace memgraph::storage
|
||||
|
@ -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_;
|
||||
|
@ -498,4 +498,11 @@ void InMemoryLabelPropertyIndex::AbortEntries(LabelId label,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void InMemoryLabelPropertyIndex::DropGraphClearIndices() {
|
||||
index_.clear();
|
||||
indices_by_property_.clear();
|
||||
stats_->clear();
|
||||
}
|
||||
|
||||
} // namespace memgraph::storage
|
||||
|
@ -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_;
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -129,6 +129,8 @@ class InMemoryUniqueConstraints : public UniqueConstraints {
|
||||
|
||||
void Clear() override;
|
||||
|
||||
void DropGraphClearConstraints();
|
||||
|
||||
static std::variant<MultipleThreadsConstraintValidation, SingleThreadConstraintValidation> GetCreationFunction(
|
||||
const std::optional<durability::ParallelizedSchemaCreationInfo> &);
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
||||
|
6
tests/e2e/drop_graph/CMakeLists.txt
Normal file
6
tests/e2e/drop_graph/CMakeLists.txt
Normal 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)
|
40
tests/e2e/drop_graph/common.py
Normal file
40
tests/e2e/drop_graph/common.py
Normal 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()
|
48
tests/e2e/drop_graph/drop_graph.py
Normal file
48
tests/e2e/drop_graph/drop_graph.py
Normal 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"]))
|
14
tests/e2e/drop_graph/workloads.yaml
Normal file
14
tests/e2e/drop_graph/workloads.yaml
Normal 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
|
Loading…
Reference in New Issue
Block a user