Expose query for cleaning memory and add memory limit flag (#100)

* Add memory flag, add additional meminfo utilities

* Add free memory query
This commit is contained in:
antonio2368 2021-03-04 12:20:11 +01:00 committed by Antonio Andelic
parent dee885d69c
commit cc56ac3dd8
26 changed files with 252 additions and 54 deletions

View File

@ -11,6 +11,8 @@
* Added support for programatically reading in data from CSV files through the * Added support for programatically reading in data from CSV files through the
`LOAD CSV` clause. We support CSV files with and without a header, the `LOAD CSV` clause. We support CSV files with and without a header, the
supported dialect being Excel. supported dialect being Excel.
* Added a new flag `--memory-limit` which enables the user to set the maximum total amount of memory
memgraph can allocate during its runtime.
### Bug Fixes ### Bug Fixes

View File

@ -83,6 +83,10 @@ modifications:
value: "/usr/lib/memgraph/auth_module/example.py" value: "/usr/lib/memgraph/auth_module/example.py"
override: false override: false
- name: "memory_limit"
value: "0"
override: true
undocumented: undocumented:
- "flag_file" - "flag_file"
- "also_log_to_stderr" - "also_log_to_stderr"

View File

@ -37,17 +37,17 @@ if (MG_ENTERPRISE)
glue/auth.cpp) glue/auth.cpp)
endif() endif()
set(MG_SINGLE_NODE_V2_LIBS stdc++fs Threads::Threads set(mg_single_node_v2_libs stdc++fs Threads::Threads
telemetry_lib mg-query mg-communication mg-new-delete) telemetry_lib mg-query mg-communication mg-new-delete mg-utils)
if (MG_ENTERPRISE) if (MG_ENTERPRISE)
# These are enterprise subsystems # These are enterprise subsystems
set(MG_SINGLE_NODE_V2_LIBS ${MG_SINGLE_NODE_V2_LIBS} mg-auth mg-audit) set(mg_single_node_v2_libs ${mg_single_node_v2_libs} mg-auth mg-audit)
endif() endif()
# memgraph main executable # memgraph main executable
add_executable(memgraph ${mg_single_node_v2_sources}) add_executable(memgraph ${mg_single_node_v2_sources})
target_include_directories(memgraph PUBLIC ${CMAKE_SOURCE_DIR}/include) target_include_directories(memgraph PUBLIC ${CMAKE_SOURCE_DIR}/include)
target_link_libraries(memgraph ${MG_SINGLE_NODE_V2_LIBS}) target_link_libraries(memgraph ${mg_single_node_v2_libs})
# NOTE: `include/mg_procedure.syms` describes a pattern match for symbols which # NOTE: `include/mg_procedure.syms` describes a pattern match for symbols which
# should be dynamically exported, so that `dlopen` can correctly link the # should be dynamically exported, so that `dlopen` can correctly link the
# symbols in custom procedure module libraries. # symbols in custom procedure module libraries.

View File

@ -45,6 +45,8 @@ std::string PermissionToString(Permission permission) {
return "LOCK_PATH"; return "LOCK_PATH";
case Permission::READ_FILE: case Permission::READ_FILE:
return "READ_FILE"; return "READ_FILE";
case Permission::FREE_MEMORY:
return "FREE_MEMORY";
case Permission::AUTH: case Permission::AUTH:
return "AUTH"; return "AUTH";
} }

View File

@ -24,15 +24,17 @@ enum class Permission : uint64_t {
REPLICATION = 1U << 10U, REPLICATION = 1U << 10U,
LOCK_PATH = 1U << 11U, LOCK_PATH = 1U << 11U,
READ_FILE = 1U << 12U, READ_FILE = 1U << 12U,
FREE_MEMORY = 1U << 13U,
AUTH = 1U << 16U AUTH = 1U << 16U
}; };
// clang-format on // clang-format on
// Constant list of all available permissions. // Constant list of all available permissions.
const std::vector<Permission> kPermissionsAll = { const std::vector<Permission> kPermissionsAll = {Permission::MATCH, Permission::CREATE, Permission::MERGE,
Permission::MATCH, Permission::CREATE, Permission::MERGE, Permission::DELETE, Permission::SET, Permission::DELETE, Permission::SET, Permission::REMOVE,
Permission::REMOVE, Permission::INDEX, Permission::STATS, Permission::CONSTRAINT, Permission::DUMP, Permission::INDEX, Permission::STATS, Permission::CONSTRAINT,
Permission::AUTH, Permission::REPLICATION, Permission::LOCK_PATH, Permission::READ_FILE}; Permission::DUMP, Permission::AUTH, Permission::REPLICATION,
Permission::LOCK_PATH, Permission::READ_FILE, Permission::FREE_MEMORY};
// Function that converts a permission to its string representation. // Function that converts a permission to its string representation.
std::string PermissionToString(Permission permission); std::string PermissionToString(Permission permission);

View File

@ -30,6 +30,8 @@ auth::Permission PrivilegeToPermission(query::AuthQuery::Privilege privilege) {
return auth::Permission::LOCK_PATH; return auth::Permission::LOCK_PATH;
case query::AuthQuery::Privilege::READ_FILE: case query::AuthQuery::Privilege::READ_FILE:
return auth::Permission::READ_FILE; return auth::Permission::READ_FILE;
case query::AuthQuery::Privilege::FREE_MEMORY:
return auth::Permission::FREE_MEMORY;
case query::AuthQuery::Privilege::AUTH: case query::AuthQuery::Privilege::AUTH:
return auth::Permission::AUTH; return auth::Permission::AUTH;
} }

View File

@ -167,6 +167,10 @@ DEFINE_VALIDATED_string(query_modules_directory, "",
DEFINE_bool(also_log_to_stderr, false, "Log messages go to stderr in addition to logfiles"); DEFINE_bool(also_log_to_stderr, false, "Log messages go to stderr in addition to logfiles");
DEFINE_string(log_file, "", "Path to where the log should be stored."); DEFINE_string(log_file, "", "Path to where the log should be stored.");
DEFINE_uint64(
memory_limit, 0,
"Total memory limit in MiB. Set to 0 to use the default values which are 100\% of the phyisical memory if the swap "
"is enabled and 90\% of the physical memory otherwise.");
namespace { namespace {
constexpr std::array log_level_mappings{ constexpr std::array log_level_mappings{
std::pair{"TRACE", spdlog::level::trace}, std::pair{"DEBUG", spdlog::level::debug}, std::pair{"TRACE", spdlog::level::trace}, std::pair{"DEBUG", spdlog::level::debug},
@ -236,6 +240,25 @@ void ConfigureLogging() {
spdlog::flush_on(spdlog::level::trace); spdlog::flush_on(spdlog::level::trace);
ParseLogLevel(); ParseLogLevel();
} }
int64_t GetMemoryLimit() {
if (FLAGS_memory_limit == 0) {
auto maybe_total_memory = utils::sysinfo::TotalMemory();
MG_ASSERT(maybe_total_memory, "Failed to fetch the total physical memory");
const auto maybe_swap_memory = utils::sysinfo::SwapTotalMemory();
MG_ASSERT(maybe_swap_memory, "Failed to fetch the total swap memory");
if (*maybe_swap_memory == 0) {
// take only 90% of the total memory
*maybe_total_memory *= 9;
*maybe_total_memory /= 10;
}
return *maybe_total_memory * 1024;
}
// We parse the memory as MiB every time
return FLAGS_memory_limit * 1024 * 1024;
}
} // namespace } // namespace
/// Encapsulates Dbms and Interpreter that are passed through the network server /// Encapsulates Dbms and Interpreter that are passed through the network server
@ -876,10 +899,10 @@ int main(int argc, char **argv) {
// Start memory warning logger. // Start memory warning logger.
utils::Scheduler mem_log_scheduler; utils::Scheduler mem_log_scheduler;
if (FLAGS_memory_warning_threshold > 0) { if (FLAGS_memory_warning_threshold > 0) {
auto free_ram = utils::sysinfo::AvailableMemoryKilobytes(); auto free_ram = utils::sysinfo::AvailableMemory();
if (free_ram) { if (free_ram) {
mem_log_scheduler.Run("Memory warning", std::chrono::seconds(3), [] { mem_log_scheduler.Run("Memory warning", std::chrono::seconds(3), [] {
auto free_ram = utils::sysinfo::AvailableMemoryKilobytes(); auto free_ram = utils::sysinfo::AvailableMemory();
if (free_ram && *free_ram / 1024 < FLAGS_memory_warning_threshold) if (free_ram && *free_ram / 1024 < FLAGS_memory_warning_threshold)
spdlog::warn("Running out of available RAM, only {} MB left", *free_ram / 1024); spdlog::warn("Running out of available RAM, only {} MB left", *free_ram / 1024);
}); });
@ -925,8 +948,9 @@ int main(int argc, char **argv) {
// End enterprise features initialization // End enterprise features initialization
#endif #endif
// Main storage and execution engines initialization utils::total_memory_tracker.SetHardLimit(GetMemoryLimit());
// Main storage and execution engines initialization
storage::Config db_config{ storage::Config db_config{
.gc = {.type = storage::Config::Gc::Type::PERIODIC, .interval = std::chrono::seconds(FLAGS_storage_gc_cycle_sec)}, .gc = {.type = storage::Config::Gc::Type::PERIODIC, .interval = std::chrono::seconds(FLAGS_storage_gc_cycle_sec)},
.items = {.properties_on_edges = FLAGS_storage_properties_on_edges}, .items = {.properties_on_edges = FLAGS_storage_properties_on_edges},

View File

@ -164,4 +164,9 @@ class LockPathModificationInMulticommandTxException : public QueryException {
: QueryException("Lock path clause not allowed in multicommand transactions.") {} : QueryException("Lock path clause not allowed in multicommand transactions.") {}
}; };
class FreeMemoryModificationInMulticommandTxException : public QueryException {
public:
FreeMemoryModificationInMulticommandTxException()
: QueryException("Lock path clause not allowed in multicommand transactions.") {}
};
} // namespace query } // namespace query

View File

@ -2191,7 +2191,7 @@ cpp<#
(:serialize)) (:serialize))
(lcp:define-enum privilege (lcp:define-enum privilege
(create delete match merge set remove index stats auth constraint (create delete match merge set remove index stats auth constraint
dump replication lock_path read_file) dump replication lock_path read_file free_memory)
(:serialize)) (:serialize))
#>cpp #>cpp
AuthQuery() = default; AuthQuery() = default;
@ -2228,7 +2228,8 @@ const std::vector<AuthQuery::Privilege> kPrivilegesAll = {
AuthQuery::Privilege::AUTH, AuthQuery::Privilege::AUTH,
AuthQuery::Privilege::CONSTRAINT, AuthQuery::Privilege::DUMP, AuthQuery::Privilege::CONSTRAINT, AuthQuery::Privilege::DUMP,
AuthQuery::Privilege::REPLICATION, AuthQuery::Privilege::REPLICATION,
AuthQuery::Privilege::LOCK_PATH}; AuthQuery::Privilege::LOCK_PATH,
AuthQuery::Privilege::FREE_MEMORY};
cpp<# cpp<#
(lcp:define-class info-query (query) (lcp:define-class info-query (query)
@ -2394,4 +2395,12 @@ cpp<#
(:serialize (:slk)) (:serialize (:slk))
(:clone)) (:clone))
(lcp:define-class free-memory-query (query) ()
(:public
#>cpp
DEFVISITABLE(QueryVisitor<void>);
cpp<#)
(:serialize (:slk))
(:clone))
(lcp:pop-namespace) ;; namespace query (lcp:pop-namespace) ;; namespace query

View File

@ -75,6 +75,7 @@ class DumpQuery;
class ReplicationQuery; class ReplicationQuery;
class LockPathQuery; class LockPathQuery;
class LoadCsv; class LoadCsv;
class FreeMemoryQuery;
using TreeCompositeVisitor = ::utils::CompositeVisitor< using TreeCompositeVisitor = ::utils::CompositeVisitor<
SingleQuery, CypherUnion, NamedExpression, OrOperator, XorOperator, AndOperator, NotOperator, AdditionOperator, SingleQuery, CypherUnion, NamedExpression, OrOperator, XorOperator, AndOperator, NotOperator, AdditionOperator,
@ -108,6 +109,6 @@ class ExpressionVisitor
template <class TResult> template <class TResult>
class QueryVisitor class QueryVisitor
: public ::utils::Visitor<TResult, CypherQuery, ExplainQuery, ProfileQuery, IndexQuery, AuthQuery, InfoQuery, : public ::utils::Visitor<TResult, CypherQuery, ExplainQuery, ProfileQuery, IndexQuery, AuthQuery, InfoQuery,
ConstraintQuery, DumpQuery, ReplicationQuery, LockPathQuery, LoadCsv> {}; ConstraintQuery, DumpQuery, ReplicationQuery, LockPathQuery, LoadCsv, FreeMemoryQuery> {};
} // namespace query } // namespace query

View File

@ -304,6 +304,12 @@ antlrcpp::Any CypherMainVisitor::visitLoadCsv(MemgraphCypher::LoadCsvContext *ct
return load_csv; return load_csv;
} }
antlrcpp::Any CypherMainVisitor::visitFreeMemoryQuery(MemgraphCypher::FreeMemoryQueryContext *ctx) {
auto *free_memory_query = storage_->Create<FreeMemoryQuery>();
query_ = free_memory_query;
return free_memory_query;
}
antlrcpp::Any CypherMainVisitor::visitCypherUnion(MemgraphCypher::CypherUnionContext *ctx) { antlrcpp::Any CypherMainVisitor::visitCypherUnion(MemgraphCypher::CypherUnionContext *ctx) {
bool distinct = !ctx->ALL(); bool distinct = !ctx->ALL();
auto *cypher_union = storage_->Create<CypherUnion>(distinct); auto *cypher_union = storage_->Create<CypherUnion>(distinct);

View File

@ -213,6 +213,11 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
*/ */
antlrcpp::Any visitLoadCsv(MemgraphCypher::LoadCsvContext *ctx) override; antlrcpp::Any visitLoadCsv(MemgraphCypher::LoadCsvContext *ctx) override;
/**
* @return FreeMemoryQuery*
*/
antlrcpp::Any visitFreeMemoryQuery(MemgraphCypher::FreeMemoryQueryContext *ctx) override;
/** /**
* @return CypherUnion* * @return CypherUnion*
*/ */

View File

@ -20,6 +20,7 @@ memgraphCypherKeyword : cypherKeyword
| DROP | DROP
| DUMP | DUMP
| FOR | FOR
| FREE
| FROM | FROM
| GRANT | GRANT
| HEADER | HEADER
@ -64,6 +65,7 @@ query : cypherQuery
| dumpQuery | dumpQuery
| replicationQuery | replicationQuery
| lockPathQuery | lockPathQuery
| freeMemoryQuery
; ;
authQuery : createRole authQuery : createRole
@ -176,4 +178,4 @@ showReplicas : SHOW REPLICAS ;
lockPathQuery : ( LOCK | UNLOCK ) DATA DIRECTORY ; lockPathQuery : ( LOCK | UNLOCK ) DATA DIRECTORY ;
freeMemoryQuery : FREE MEMORY ;

View File

@ -24,6 +24,7 @@ DIRECTORY : D I R E C T O R Y ;
DROP : D R O P ; DROP : D R O P ;
DUMP : D U M P ; DUMP : D U M P ;
FOR : F O R ; FOR : F O R ;
FREE : F R E E ;
FROM : F R O M ; FROM : F R O M ;
GRANT : G R A N T ; GRANT : G R A N T ;
GRANTS : G R A N T S ; GRANTS : G R A N T S ;

View File

@ -1,4 +1,5 @@
#include "query/frontend/ast/ast.hpp" #include "query/frontend/ast/ast.hpp"
#include "query/frontend/ast/ast_visitor.hpp"
namespace query { namespace query {
@ -52,6 +53,8 @@ class PrivilegeExtractor : public QueryVisitor<void>, public HierarchicalTreeVis
void Visit(LoadCsv &load_csv) override { AddPrivilege(AuthQuery::Privilege::READ_FILE); } void Visit(LoadCsv &load_csv) override { AddPrivilege(AuthQuery::Privilege::READ_FILE); }
void Visit(FreeMemoryQuery &free_memory_query) override { AddPrivilege(AuthQuery::Privilege::FREE_MEMORY); }
void Visit(ReplicationQuery &replication_query) override { void Visit(ReplicationQuery &replication_query) override {
switch (replication_query.action_) { switch (replication_query.action_) {
case ReplicationQuery::Action::SET_REPLICATION_ROLE: case ReplicationQuery::Action::SET_REPLICATION_ROLE:

View File

@ -87,7 +87,7 @@ const trie::Trie kKeywords = {
"single", "true", "false", "reduce", "coalesce", "user", "password", "alter", "drop", "single", "true", "false", "reduce", "coalesce", "user", "password", "alter", "drop",
"show", "stats", "unique", "explain", "profile", "storage", "index", "info", "exists", "show", "stats", "unique", "explain", "profile", "storage", "index", "info", "exists",
"assert", "constraint", "node", "key", "dump", "database", "call", "yield", "memory", "assert", "constraint", "node", "key", "dump", "database", "call", "yield", "memory",
"mb", "kb", "unlimited"}; "mb", "kb", "unlimited", "free"};
// Unicode codepoints that are allowed at the start of the unescaped name. // Unicode codepoints that are allowed at the start of the unescaped name.
const std::bitset<kBitsetSize> kUnescapedNameAllowedStarts( const std::bitset<kBitsetSize> kUnescapedNameAllowedStarts(

View File

@ -1162,6 +1162,22 @@ PreparedQuery PrepareLockPathQuery(ParsedQuery parsed_query, const bool in_expli
RWType::NONE}; RWType::NONE};
} }
PreparedQuery PrepareFreeMemoryQuery(ParsedQuery parsed_query, const bool in_explicit_transaction,
InterpreterContext *interpreter_context) {
if (in_explicit_transaction) {
throw FreeMemoryModificationInMulticommandTxException();
}
interpreter_context->db->FreeMemory();
return PreparedQuery{{},
std::move(parsed_query.required_privileges),
[](AnyStream *stream, std::optional<int> n) -> std::optional<QueryHandlerResult> {
return QueryHandlerResult::COMMIT;
},
RWType::NONE};
}
PreparedQuery PrepareInfoQuery(ParsedQuery parsed_query, bool in_explicit_transaction, PreparedQuery PrepareInfoQuery(ParsedQuery parsed_query, bool in_explicit_transaction,
std::map<std::string, TypedValue> *summary, InterpreterContext *interpreter_context, std::map<std::string, TypedValue> *summary, InterpreterContext *interpreter_context,
storage::Storage *db, utils::MonotonicBufferResource *execution_memory) { storage::Storage *db, utils::MonotonicBufferResource *execution_memory) {
@ -1488,6 +1504,8 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
} else if (utils::Downcast<LockPathQuery>(parsed_query.query)) { } else if (utils::Downcast<LockPathQuery>(parsed_query.query)) {
prepared_query = PrepareLockPathQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_, prepared_query = PrepareLockPathQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_,
&*execution_db_accessor_); &*execution_db_accessor_);
} else if (utils::Downcast<FreeMemoryQuery>(parsed_query.query)) {
prepared_query = PrepareFreeMemoryQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_);
} else { } else {
LOG_FATAL("Should not get here -- unknown query type!"); LOG_FATAL("Should not get here -- unknown query type!");
} }

View File

@ -349,6 +349,12 @@ LabelIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor,
constraints_(constraints), constraints_(constraints),
config_(config) {} config_(config) {}
void LabelIndex::RunGC() {
for (auto &index_entry : index_) {
index_entry.second.run_gc();
}
}
bool LabelPropertyIndex::Entry::operator<(const Entry &rhs) { bool LabelPropertyIndex::Entry::operator<(const Entry &rhs) {
if (value < rhs.value) { if (value < rhs.value) {
return true; return true;
@ -661,6 +667,12 @@ int64_t LabelPropertyIndex::ApproximateVertexCount(LabelId label, PropertyId pro
return acc.estimate_range_count(lower, upper, utils::SkipListLayerForCountEstimation(acc.size())); return acc.estimate_range_count(lower, upper, utils::SkipListLayerForCountEstimation(acc.size()));
} }
void LabelPropertyIndex::RunGC() {
for (auto &index_entry : index_) {
index_entry.second.run_gc();
}
}
void RemoveObsoleteEntries(Indices *indices, uint64_t oldest_active_start_timestamp) { void RemoveObsoleteEntries(Indices *indices, uint64_t oldest_active_start_timestamp) {
indices->label_index.RemoveObsoleteEntries(oldest_active_start_timestamp); indices->label_index.RemoveObsoleteEntries(oldest_active_start_timestamp);
indices->label_property_index.RemoveObsoleteEntries(oldest_active_start_timestamp); indices->label_property_index.RemoveObsoleteEntries(oldest_active_start_timestamp);

View File

@ -110,6 +110,8 @@ class LabelIndex {
void Clear() { index_.clear(); } void Clear() { index_.clear(); }
void RunGC();
private: private:
std::map<LabelId, utils::SkipList<Entry>> index_; std::map<LabelId, utils::SkipList<Entry>> index_;
Indices *indices_; Indices *indices_;
@ -225,6 +227,8 @@ class LabelPropertyIndex {
void Clear() { index_.clear(); } void Clear() { index_.clear(); }
void RunGC();
private: private:
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_; std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_;
Indices *indices_; Indices *indices_;
@ -263,5 +267,4 @@ void UpdateOnAddLabel(Indices *indices, LabelId label, Vertex *vertex, const Tra
/// @throw std::bad_alloc /// @throw std::bad_alloc
void UpdateOnSetProperty(Indices *indices, PropertyId property, const PropertyValue &value, Vertex *vertex, void UpdateOnSetProperty(Indices *indices, PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx); const Transaction &tx);
} // namespace storage } // namespace storage

View File

@ -361,7 +361,7 @@ Storage::Storage(Config config)
snapshot_runner_.Run("Snapshot", config_.durability.snapshot_interval, [this] { this->CreateSnapshot(); }); snapshot_runner_.Run("Snapshot", config_.durability.snapshot_interval, [this] { this->CreateSnapshot(); });
} }
if (config_.gc.type == Config::Gc::Type::PERIODIC) { if (config_.gc.type == Config::Gc::Type::PERIODIC) {
gc_runner_.Run("Storage GC", config_.gc.interval, [this] { this->CollectGarbage(); }); gc_runner_.Run("Storage GC", config_.gc.interval, [this] { this->CollectGarbage<false>(); });
} }
if (timestamp_ == kTimestampInitialId) { if (timestamp_ == kTimestampInitialId) {
@ -1221,11 +1221,29 @@ Transaction Storage::CreateTransaction() {
return {transaction_id, start_timestamp}; return {transaction_id, start_timestamp};
} }
template <bool force>
void Storage::CollectGarbage() { void Storage::CollectGarbage() {
// Because the garbage collector iterates through the indices and constraints if constexpr (force) {
// to clean them up, it must take the main lock for reading to make sure that // We take the unique lock on the main storage lock so we can forcefully clean
// the indices and constraints aren't concurrently being modified. // everything we can
std::shared_lock<utils::RWLock> main_guard(main_lock_); if (!main_lock_.try_lock()) {
CollectGarbage<false>();
return;
}
} else {
// Because the garbage collector iterates through the indices and constraints
// to clean them up, it must take the main lock for reading to make sure that
// the indices and constraints aren't concurrently being modified.
main_lock_.lock_shared();
}
utils::OnScopeExit lock_releaser{[&] {
if constexpr (force) {
main_lock_.unlock();
} else {
main_lock_.unlock_shared();
}
}};
// Garbage collection must be performed in two phases. In the first phase, // Garbage collection must be performed in two phases. In the first phase,
// deltas that won't be applied by any transaction anymore are unlinked from // deltas that won't be applied by any transaction anymore are unlinked from
@ -1418,19 +1436,32 @@ void Storage::CollectGarbage() {
} }
} }
while (true) { garbage_undo_buffers_.WithLock([&](auto &undo_buffers) {
auto garbage_undo_buffers_ptr = garbage_undo_buffers_.Lock(); // if force is set to true we can simply delete all the leftover undos because
if (garbage_undo_buffers_ptr->empty() || garbage_undo_buffers_ptr->front().first > oldest_active_start_timestamp) { // no transaction is active
break; if constexpr (force) {
undo_buffers.clear();
} else {
while (!undo_buffers.empty() && undo_buffers.front().first <= oldest_active_start_timestamp) {
undo_buffers.pop_front();
}
} }
garbage_undo_buffers_ptr->pop_front(); });
}
{ {
auto vertex_acc = vertices_.access(); auto vertex_acc = vertices_.access();
while (!garbage_vertices_.empty() && garbage_vertices_.front().first < oldest_active_start_timestamp) { if constexpr (force) {
MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!"); // if force is set to true, then we have unique_lock and no transactions are active
garbage_vertices_.pop_front(); // so we can clean all of the deleted vertices
while (!garbage_vertices_.empty()) {
MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!");
garbage_vertices_.pop_front();
}
} else {
while (!garbage_vertices_.empty() && garbage_vertices_.front().first < oldest_active_start_timestamp) {
MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!");
garbage_vertices_.pop_front();
}
} }
} }
{ {
@ -1441,6 +1472,10 @@ void Storage::CollectGarbage() {
} }
} }
// tell the linker he can find the CollectGarbage definitions here
template void Storage::CollectGarbage<true>();
template void Storage::CollectGarbage<false>();
bool Storage::InitializeWalFile() { bool Storage::InitializeWalFile() {
if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL) if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL)
return false; return false;
@ -1702,6 +1737,16 @@ bool Storage::UnlockPath() {
return true; return true;
} }
void Storage::FreeMemory() {
CollectGarbage<true>();
// SkipList is already threadsafe
vertices_.run_gc();
edges_.run_gc();
indices_.label_index.RunGC();
indices_.label_property_index.RunGC();
}
uint64_t Storage::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) { uint64_t Storage::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) {
if (!desired_commit_timestamp) { if (!desired_commit_timestamp) {
return timestamp_++; return timestamp_++;

View File

@ -21,6 +21,7 @@
#include "storage/v2/vertex.hpp" #include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp" #include "storage/v2/vertex_accessor.hpp"
#include "utils/file_locker.hpp" #include "utils/file_locker.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/rw_lock.hpp" #include "utils/rw_lock.hpp"
#include "utils/scheduler.hpp" #include "utils/scheduler.hpp"
#include "utils/skip_list.hpp" #include "utils/skip_list.hpp"
@ -412,11 +413,23 @@ class Storage final {
std::vector<ReplicaInfo> ReplicasInfo(); std::vector<ReplicaInfo> ReplicasInfo();
void FreeMemory();
private: private:
Transaction CreateTransaction(); Transaction CreateTransaction();
/// 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
/// be part of a transaction, and no other transaction can be active at the same time.
/// This allows it to delete immediately vertices without worrying that some other
/// transaction is possibly using it. If there are active transactions when this method
/// is called with force set to true, it will fallback to the same method with the force
/// set to false.
/// If it's set to false, it will execute in parallel with other transactions, ensuring
/// that no object in use can be deleted.
/// @throw std::system_error /// @throw std::system_error
/// @throw std::bad_alloc /// @throw std::bad_alloc
template <bool force>
void CollectGarbage(); void CollectGarbage();
bool InitializeWalFile(); bool InitializeWalFile();

View File

@ -6,6 +6,7 @@ set(utils_src_files
memory.cpp memory.cpp
memory_tracker.cpp memory_tracker.cpp
signals.cpp signals.cpp
sysinfo/memory.cpp
thread.cpp thread.cpp
thread_pool.cpp thread_pool.cpp
uuid.cpp) uuid.cpp)

View File

@ -259,7 +259,7 @@ class SkipListGc final {
} }
void Collect(TNode *node) { void Collect(TNode *node) {
std::lock_guard<SpinLock> guard(lock_); std::unique_lock guard(lock_);
deleted_.Push({accessor_id_.load(std::memory_order_acquire), node}); deleted_.Push({accessor_id_.load(std::memory_order_acquire), node});
} }
@ -895,6 +895,8 @@ class SkipList final {
gc_.Clear(); gc_.Clear();
} }
void run_gc() { gc_.Run(); }
private: private:
template <typename TKey> template <typename TKey>
int find_node(const TKey &key, TNode *preds[], TNode *succs[]) const { int find_node(const TKey &key, TNode *preds[], TNode *succs[]) const {

View File

@ -0,0 +1,33 @@
#include "utils/sysinfo/memory.hpp"
namespace utils::sysinfo {
namespace {
std::optional<uint64_t> ExtractAmountFromMemInfo(const std::string_view header_name) {
std::string token;
std::ifstream meminfo("/proc/meminfo");
const auto meminfo_header = fmt::format("{}:", header_name);
while (meminfo >> token) {
if (token == meminfo_header) {
uint64_t mem = 0;
if (meminfo >> mem) {
return mem;
} else {
return std::nullopt;
}
}
meminfo.ignore(std::numeric_limits<std::streamsize>::max(), '\n');
}
SPDLOG_WARN("Failed to read {} from /proc/meminfo", header_name);
return std::nullopt;
}
} // namespace
std::optional<uint64_t> AvailableMemory() { return ExtractAmountFromMemInfo("MemAvailable"); }
std::optional<uint64_t> TotalMemory() { return ExtractAmountFromMemInfo("MemTotal"); }
std::optional<uint64_t> SwapTotalMemory() { return ExtractAmountFromMemInfo("SwapTotal"); }
} // namespace utils::sysinfo

View File

@ -1,3 +1,5 @@
#pragma once
#include <fstream> #include <fstream>
#include <iostream> #include <iostream>
#include <limits> #include <limits>
@ -8,25 +10,21 @@
namespace utils::sysinfo { namespace utils::sysinfo {
/** /**
* Gets the amount of available RAM in kilobytes. If the information is * Gets the amount of available RAM in KiB. If the information is
* unavalable an empty value is returned. * unavalable an empty value is returned.
*/ */
inline std::optional<uint64_t> AvailableMemoryKilobytes() { std::optional<uint64_t> AvailableMemory();
std::string token;
std::ifstream meminfo("/proc/meminfo"); /**
while (meminfo >> token) { * Gets the amount of total RAM in KiB. If the information is
if (token == "MemAvailable:") { * unavalable an empty value is returned.
uint64_t mem = 0; */
if (meminfo >> mem) { std::optional<uint64_t> TotalMemory();
return mem;
} else { /**
return std::nullopt; * Gets the amount of total swap space in KiB. If the information is
} * unavalable an empty value is returned.
} */
meminfo.ignore(std::numeric_limits<std::streamsize>::max(), '\n'); std::optional<uint64_t> SwapTotalMemory();
}
SPDLOG_WARN("Failed to read amount of available memory from /proc/meminfo");
return std::nullopt;
}
} // namespace utils::sysinfo } // namespace utils::sysinfo

View File

@ -3,6 +3,7 @@
#include <gtest/gtest.h> #include <gtest/gtest.h>
#include <utils/memory_tracker.hpp> #include <utils/memory_tracker.hpp>
#include <utils/on_scope_exit.hpp>
TEST(MemoryTrackerTest, ExceptionEnabler) { TEST(MemoryTrackerTest, ExceptionEnabler) {
utils::MemoryTracker memory_tracker; utils::MemoryTracker memory_tracker;
@ -16,11 +17,15 @@ TEST(MemoryTrackerTest, ExceptionEnabler) {
// wait until the second thread creates exception enabler // wait until the second thread creates exception enabler
while (!enabler_created) while (!enabler_created)
; ;
ASSERT_NO_THROW(memory_tracker.Alloc(hard_limit + 1));
ASSERT_EQ(memory_tracker.Amount(), hard_limit + 1);
// tell the second thread it can finish its test // we use the OnScopeExit so the test doesn't deadlock when
can_continue = true; // an ASSERT fails
utils::OnScopeExit thread_notifier{[&] {
// tell the second thread it can finish its test
can_continue = true;
}};
ASSERT_NO_THROW(memory_tracker.Alloc(hard_limit + 1));
}}; }};
std::thread t2{[&] { std::thread t2{[&] {