Drop graph functionality
This commit is contained in:
parent
3a7e62f72c
commit
d89193dd6d
@ -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
|
||||
|
@ -242,6 +242,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};
|
||||
|
||||
|
@ -3106,6 +3106,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;
|
||||
|
@ -109,6 +109,7 @@ class ShowDatabasesQuery;
|
||||
class EdgeImportModeQuery;
|
||||
class PatternComprehension;
|
||||
class CoordinatorQuery;
|
||||
class DropGraphQuery;
|
||||
|
||||
using TreeCompositeVisitor = utils::CompositeVisitor<
|
||||
SingleQuery, CypherUnion, NamedExpression, OrOperator, XorOperator, AndOperator, NotOperator, AdditionOperator,
|
||||
@ -148,6 +149,6 @@ class QueryVisitor
|
||||
FreeMemoryQuery, TriggerQuery, IsolationLevelQuery, CreateSnapshotQuery, StreamQuery,
|
||||
SettingQuery, VersionQuery, ShowConfigQuery, TransactionQueueQuery, StorageModeQuery,
|
||||
AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery, EdgeImportModeQuery,
|
||||
CoordinatorQuery> {};
|
||||
CoordinatorQuery, DropGraphQuery> {};
|
||||
|
||||
} // namespace memgraph::query
|
||||
|
@ -320,6 +320,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;
|
||||
|
@ -238,6 +238,11 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
|
||||
*/
|
||||
antlrcpp::Any visitCoordinatorQuery(MemgraphCypher::CoordinatorQueryContext *ctx) override;
|
||||
|
||||
/**
|
||||
* @return DropGraphQuery*
|
||||
*/
|
||||
antlrcpp::Any visitDropGraphQuery(MemgraphCypher::DropGraphQueryContext *ctx) override;
|
||||
|
||||
/**
|
||||
* @return CoordinatorQuery*
|
||||
*/
|
||||
|
@ -157,6 +157,7 @@ query : cypherQuery
|
||||
| showDatabases
|
||||
| edgeImportModeQuery
|
||||
| coordinatorQuery
|
||||
| dropGraphQuery
|
||||
;
|
||||
|
||||
cypherQuery : ( indexHints )? singleQuery ( cypherUnion )* ( queryMemoryLimit )? ;
|
||||
@ -521,3 +522,5 @@ showDatabase : SHOW DATABASE ;
|
||||
showDatabases : SHOW DATABASES ;
|
||||
|
||||
edgeImportModeQuery : EDGE IMPORT MODE ( ACTIVE | INACTIVE ) ;
|
||||
|
||||
dropGraphQuery : DROP GRAPH ;
|
||||
|
@ -97,6 +97,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 {
|
||||
|
@ -3143,6 +3143,29 @@ Callback SwitchMemoryDevice(storage::StorageMode current_mode, storage::StorageM
|
||||
return callback;
|
||||
}
|
||||
|
||||
Callback DropGraph(memgraph::dbms::DatabaseAccess &db) {
|
||||
Callback callback;
|
||||
callback.fn = [&db]() mutable {
|
||||
if (!db.try_exclusively([](auto &in) {
|
||||
spdlog::info("Woohooooo!!");
|
||||
auto current_storage_mode = in.GetStorageMode();
|
||||
if (current_storage_mode != storage::StorageMode::IN_MEMORY_ANALYTICAL) {
|
||||
throw utils::BasicException("Drop graph can not be used without IN_MEMORY_ANALYTICAL storage mode!");
|
||||
}
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::seconds(10));
|
||||
in.UniqueAccess()->DropGraph();
|
||||
})) { // Try exclusively failed
|
||||
throw utils::BasicException(
|
||||
"Drop graph command failed! Ensure that you're the only transaction currently running!");
|
||||
}
|
||||
|
||||
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) {
|
||||
@ -3195,6 +3218,25 @@ 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_;
|
||||
|
||||
auto *drop_graph_query = utils::Downcast<DropGraphQuery>(parsed_query.query);
|
||||
MG_ASSERT(drop_graph_query);
|
||||
|
||||
std::function<void()> callback = DropGraph(db_acc).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();
|
||||
@ -4343,6 +4385,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
|
||||
@ -33,4 +33,9 @@ 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();
|
||||
}
|
||||
|
||||
} // 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
|
||||
@ -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
|
||||
@ -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;
|
||||
|
@ -173,6 +173,8 @@ class DiskStorage final : public Storage {
|
||||
|
||||
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
|
||||
const std::set<PropertyId> &properties) override;
|
||||
|
||||
void DropGraph() override{};
|
||||
};
|
||||
|
||||
using Storage::Access;
|
||||
|
@ -38,6 +38,11 @@ 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();
|
||||
}
|
||||
|
||||
void Indices::UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx) const {
|
||||
label_index_->UpdateOnAddLabel(label, vertex, tx);
|
||||
label_property_index_->UpdateOnAddLabel(label, vertex, tx);
|
||||
|
@ -43,6 +43,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
|
||||
|
@ -235,4 +235,11 @@ std::vector<LabelId> InMemoryLabelIndex::Analysis() const {
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
void InMemoryLabelIndex::DropGraphClearIndices() {
|
||||
for (auto &[label, storage] : index_) {
|
||||
storage.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_;
|
||||
|
@ -495,4 +495,11 @@ void InMemoryLabelPropertyIndex::AbortEntries(LabelId label,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void InMemoryLabelPropertyIndex::DropGraphClearIndices() {
|
||||
for (auto &[label_prop, storage] : index_) {
|
||||
storage.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_;
|
||||
|
@ -2164,6 +2164,68 @@ void InMemoryStorage::FreeMemory(std::unique_lock<utils::ResourceLock> main_guar
|
||||
edges_.run_gc();
|
||||
}
|
||||
|
||||
void InMemoryStorage::DropGraph() {
|
||||
std::unique_lock main_guard{main_lock_};
|
||||
|
||||
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_++;
|
||||
@ -2286,4 +2348,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
|
||||
|
@ -295,6 +295,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
|
||||
@ -350,6 +352,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
|
||||
|
@ -523,4 +523,9 @@ void InMemoryUniqueConstraints::Clear() {
|
||||
constraints_by_label_.clear();
|
||||
}
|
||||
|
||||
void InMemoryUniqueConstraints::DropGraphClearConstraints() {
|
||||
for (auto &[label_props, storage] : constraints_) {
|
||||
storage.clear();
|
||||
};
|
||||
}
|
||||
} // namespace memgraph::storage
|
||||
|
@ -126,6 +126,8 @@ class InMemoryUniqueConstraints : public UniqueConstraints {
|
||||
|
||||
void Clear() override;
|
||||
|
||||
void DropGraphClearConstraints();
|
||||
|
||||
static std::variant<MultipleThreadsConstraintValidation, SingleThreadConstraintValidation> GetCreationFunction(
|
||||
const std::optional<durability::ParallelizedSchemaCreationInfo> &);
|
||||
|
||||
|
@ -280,6 +280,8 @@ class Storage {
|
||||
virtual UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
|
||||
const std::set<PropertyId> &properties) = 0;
|
||||
|
||||
virtual void DropGraph() = 0;
|
||||
|
||||
protected:
|
||||
Storage *storage_;
|
||||
std::shared_lock<utils::ResourceLock> storage_guard_;
|
||||
|
@ -104,7 +104,8 @@ struct Gatekeeper {
|
||||
friend Gatekeeper;
|
||||
|
||||
private:
|
||||
explicit Accessor(Gatekeeper *owner) : owner_{owner->pimpl_.get()} { ++owner_->count_; }
|
||||
explicit Accessor(Gatekeeper *owner) : owner_{owner->pimpl_.get()} { (++owner_->count_;
|
||||
}
|
||||
|
||||
public:
|
||||
Accessor(Accessor const &other) : owner_{other.owner_} {
|
||||
|
@ -216,6 +216,7 @@ enum class TypeId : uint64_t {
|
||||
AST_EDGE_IMPORT_MODE_QUERY,
|
||||
AST_PATTERN_COMPREHENSION,
|
||||
AST_COORDINATOR_QUERY,
|
||||
AST_DROP_GRAPH_QUERY,
|
||||
|
||||
// Symbol
|
||||
SYMBOL = 4000,
|
||||
|
Loading…
Reference in New Issue
Block a user