From b6b32bec037eae3442cdfb69dd1a1e7ec34b919e Mon Sep 17 00:00:00 2001 From: Antonio Filipovic <61245998+antoniofilipovic@users.noreply.github.com> Date: Wed, 6 Sep 2023 11:30:21 +0200 Subject: [PATCH] Improve performance of delta creation (#1129) --- src/query/stream/streams.hpp | 2 +- src/storage/v2/disk/storage.cpp | 12 ++-- src/storage/v2/disk/storage.hpp | 1 + src/storage/v2/inmemory/storage.cpp | 48 ++++++++++------ src/storage/v2/inmemory/storage.hpp | 12 +++- .../v2/inmemory/unique_constraints.cpp | 2 +- src/storage/v2/mvcc.hpp | 18 +++--- src/storage/v2/storage.cpp | 2 +- src/storage/v2/transaction.hpp | 26 +++++---- src/utils/bond.hpp | 57 +++++++++++++++++++ src/utils/memory_tracker.cpp | 2 +- src/utils/temporal.cpp | 2 +- tests/unit/formatters.hpp | 2 +- tests/unit/storage_v2_wal_file.cpp | 7 ++- 14 files changed, 140 insertions(+), 53 deletions(-) create mode 100644 src/utils/bond.hpp diff --git a/src/query/stream/streams.hpp b/src/query/stream/streams.hpp index c8d57a685..9db28f808 100644 --- a/src/query/stream/streams.hpp +++ b/src/query/stream/streams.hpp @@ -1,4 +1,4 @@ -// Copyright 2022 Memgraph Ltd. +// 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 diff --git a/src/storage/v2/disk/storage.cpp b/src/storage/v2/disk/storage.cpp index 8f2c160fd..a94c01552 100644 --- a/src/storage/v2/disk/storage.cpp +++ b/src/storage/v2/disk/storage.cpp @@ -54,6 +54,7 @@ #include "utils/exceptions.hpp" #include "utils/file.hpp" #include "utils/logging.hpp" +#include "utils/memory.hpp" #include "utils/memory_tracker.hpp" #include "utils/message.hpp" #include "utils/on_scope_exit.hpp" @@ -63,6 +64,7 @@ #include "utils/skip_list.hpp" #include "utils/stat.hpp" #include "utils/string.hpp" +#include "utils/typeinfo.hpp" namespace memgraph::storage { @@ -335,6 +337,8 @@ DiskStorage::DiskAccessor::~DiskAccessor() { } FinalizeTransaction(); + + transaction_.deltas.~Bond(); } /// NOTE: This will create Delta object which will cause deletion of old key entry on the disk @@ -1638,9 +1642,9 @@ utils::BasicResult DiskStorage::DiskAccessor auto *disk_storage = static_cast(storage_); bool edge_import_mode_active = disk_storage->edge_import_status_ == EdgeImportMode::ACTIVE; - if (transaction_.deltas.empty() || + if (transaction_.deltas.use().empty() || (!edge_import_mode_active && - std::all_of(transaction_.deltas.begin(), transaction_.deltas.end(), + std::all_of(transaction_.deltas.use().begin(), transaction_.deltas.use().end(), [](const Delta &delta) { return delta.action == Delta::Action::DELETE_DESERIALIZED_OBJECT; }))) { } else { std::unique_lock engine_guard(storage_->engine_lock_); @@ -1758,9 +1762,9 @@ std::vector> DiskStorage::SerializeVerticesF void DiskStorage::DiskAccessor::UpdateObjectsCountOnAbort() { auto *disk_storage = static_cast(storage_); - uint64_t transaction_id = transaction_.transaction_id.load(std::memory_order_acquire); + uint64_t transaction_id = transaction_.transaction_id; - for (const auto &delta : transaction_.deltas) { + for (const auto &delta : transaction_.deltas.use()) { auto prev = delta.prev.Get(); switch (prev.type) { case PreviousPtr::Type::VERTEX: { diff --git a/src/storage/v2/disk/storage.hpp b/src/storage/v2/disk/storage.hpp index 1a15d2340..597a49b55 100644 --- a/src/storage/v2/disk/storage.hpp +++ b/src/storage/v2/disk/storage.hpp @@ -390,6 +390,7 @@ class DiskStorage final : public Storage { private: std::unique_ptr kvstore_; std::unique_ptr durability_kvstore_; + std::atomic vertex_count_{0}; }; diff --git a/src/storage/v2/inmemory/storage.cpp b/src/storage/v2/inmemory/storage.cpp index 0f35c1115..048096493 100644 --- a/src/storage/v2/inmemory/storage.cpp +++ b/src/storage/v2/inmemory/storage.cpp @@ -178,6 +178,9 @@ InMemoryStorage::~InMemoryStorage() { } } } + if (!committed_transactions_->empty()) { + committed_transactions_.WithLock([](auto &transactions) { transactions.clear(); }); + } } InMemoryStorage::InMemoryAccessor::InMemoryAccessor(InMemoryStorage *storage, IsolationLevel isolation_level, @@ -597,14 +600,14 @@ utils::BasicResult InMemoryStorage::InMemory auto *mem_storage = static_cast(storage_); - if (transaction_.deltas.empty()) { + if (transaction_.deltas.use().empty()) { // We don't have to update the commit timestamp here because no one reads // it. mem_storage->commit_log_->MarkFinished(transaction_.start_timestamp); } else { // Validate that existence constraints are satisfied for all modified // vertices. - for (const auto &delta : transaction_.deltas) { + for (const auto &delta : transaction_.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::VERTEX) { @@ -636,7 +639,7 @@ utils::BasicResult InMemoryStorage::InMemory // Before committing and validating vertices against unique constraints, // we have to update unique constraints with the vertices that are going // to be validated/committed. - for (const auto &delta : transaction_.deltas) { + for (const auto &delta : transaction_.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::VERTEX) { @@ -647,7 +650,7 @@ utils::BasicResult InMemoryStorage::InMemory // Validate that unique constraints are satisfied for all modified // vertices. - for (const auto &delta : transaction_.deltas) { + for (const auto &delta : transaction_.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::VERTEX) { @@ -726,15 +729,15 @@ void InMemoryStorage::InMemoryAccessor::Abort() { std::list my_deleted_vertices; std::list my_deleted_edges; - for (const auto &delta : transaction_.deltas) { + for (const auto &delta : transaction_.deltas.use()) { auto prev = delta.prev.Get(); switch (prev.type) { case PreviousPtr::Type::VERTEX: { auto *vertex = prev.vertex; auto guard = std::unique_lock{vertex->lock}; Delta *current = vertex->delta; - while (current != nullptr && current->timestamp->load(std::memory_order_acquire) == - transaction_.transaction_id.load(std::memory_order_acquire)) { + while (current != nullptr && + current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) { switch (current->action) { case Delta::Action::REMOVE_LABEL: { auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label); @@ -821,8 +824,8 @@ void InMemoryStorage::InMemoryAccessor::Abort() { auto *edge = prev.edge; auto guard = std::lock_guard{edge->lock}; Delta *current = edge->delta; - while (current != nullptr && current->timestamp->load(std::memory_order_acquire) == - transaction_.transaction_id.load(std::memory_order_acquire)) { + while (current != nullptr && + current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) { switch (current->action) { case Delta::Action::SET_PROPERTY: { edge->properties.SetProperty(current->property.key, current->property.value); @@ -874,6 +877,7 @@ void InMemoryStorage::InMemoryAccessor::Abort() { // Release engine lock because we don't have to hold it anymore and // emplace back could take a long time. engine_guard.unlock(); + garbage_undo_buffers.emplace_back(mark_timestamp, std::move(transaction_.deltas)); }); mem_storage->deleted_vertices_.WithLock( @@ -1191,7 +1195,7 @@ void InMemoryStorage::CollectGarbage(std::unique_lock main_guard) // We don't move undo buffers of unlinked transactions to garbage_undo_buffers // list immediately, because we would have to repeatedly take // garbage_undo_buffers lock. - std::list>> unlinked_undo_buffers; + std::list> unlinked_undo_buffers; // We will only free vertices deleted up until now in this GC cycle, and we // will do it after cleaning-up the indices. That way we are sure that all @@ -1259,7 +1263,7 @@ void InMemoryStorage::CollectGarbage(std::unique_lock main_guard) // chain in a broken state. // The chain can be only read without taking any locks. - for (Delta &delta : transaction->deltas) { + for (Delta &delta : transaction->deltas.use()) { while (true) { auto prev = delta.prev.Get(); switch (prev.type) { @@ -1365,10 +1369,10 @@ void InMemoryStorage::CollectGarbage(std::unique_lock main_guard) // TODO(mtomic): holding garbage_undo_buffers_ lock here prevents // transactions from aborting until we're done marking, maybe we should // add them one-by-one or something - for (auto &[timestamp, undo_buffer] : unlinked_undo_buffers) { + for (auto &[timestamp, transaction_deltas] : unlinked_undo_buffers) { timestamp = mark_timestamp; } - garbage_undo_buffers.splice(garbage_undo_buffers.end(), unlinked_undo_buffers); + garbage_undo_buffers.splice(garbage_undo_buffers.end(), std::move(unlinked_undo_buffers)); }); for (auto vertex : current_deleted_vertices) { garbage_vertices_.emplace_back(mark_timestamp, vertex); @@ -1379,9 +1383,17 @@ void InMemoryStorage::CollectGarbage(std::unique_lock main_guard) // if force is set to true we can simply delete all the leftover undos because // no transaction is active if constexpr (force) { + for (auto &[timestamp, transaction_deltas] : undo_buffers) { + transaction_deltas.~Bond(); + } undo_buffers.clear(); + } else { while (!undo_buffers.empty() && undo_buffers.front().first <= oldest_active_start_timestamp) { + auto &[timestamp, transaction_deltas] = undo_buffers.front(); + transaction_deltas.~Bond(); + // this will trigger destory of object + // but since we release pointer, it will just destory other stuff undo_buffers.pop_front(); } } @@ -1527,7 +1539,7 @@ bool InMemoryStorage::AppendToWalDataManipulation(const Transaction &transaction // 1. Process all Vertex deltas and store all operations that create vertices // and modify vertex data. - for (const auto &delta : transaction.deltas) { + for (const auto &delta : transaction.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::VERTEX) continue; @@ -1550,7 +1562,7 @@ bool InMemoryStorage::AppendToWalDataManipulation(const Transaction &transaction }); } // 2. Process all Vertex deltas and store all operations that create edges. - for (const auto &delta : transaction.deltas) { + for (const auto &delta : transaction.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::VERTEX) continue; @@ -1572,7 +1584,7 @@ bool InMemoryStorage::AppendToWalDataManipulation(const Transaction &transaction }); } // 3. Process all Edge deltas and store all operations that modify edge data. - for (const auto &delta : transaction.deltas) { + for (const auto &delta : transaction.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::EDGE) continue; @@ -1594,7 +1606,7 @@ bool InMemoryStorage::AppendToWalDataManipulation(const Transaction &transaction }); } // 4. Process all Vertex deltas and store all operations that delete edges. - for (const auto &delta : transaction.deltas) { + for (const auto &delta : transaction.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::VERTEX) continue; @@ -1616,7 +1628,7 @@ bool InMemoryStorage::AppendToWalDataManipulation(const Transaction &transaction }); } // 5. Process all Vertex deltas and store all operations that delete vertices. - for (const auto &delta : transaction.deltas) { + for (const auto &delta : transaction.deltas.use()) { auto prev = delta.prev.Get(); MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); if (prev.type != PreviousPtr::Type::VERTEX) continue; diff --git a/src/storage/v2/inmemory/storage.hpp b/src/storage/v2/inmemory/storage.hpp index a0b56fe03..f7b645824 100644 --- a/src/storage/v2/inmemory/storage.hpp +++ b/src/storage/v2/inmemory/storage.hpp @@ -12,6 +12,9 @@ #pragma once #include +#include +#include +#include #include "storage/v2/inmemory/label_index.hpp" #include "storage/v2/inmemory/label_property_index.hpp" #include "storage/v2/storage.hpp" @@ -23,6 +26,9 @@ #include "storage/v2/replication/replication_persistence_helper.hpp" #include "storage/v2/replication/rpc.hpp" #include "storage/v2/replication/serialization.hpp" +#include "storage/v2/transaction.hpp" +#include "utils/memory.hpp" +#include "utils/synchronized.hpp" namespace memgraph::storage { @@ -438,12 +444,14 @@ class InMemoryStorage final : public Storage { // `timestamp_` in a sensible unit, something like TransactionClock or // whatever. std::optional commit_log_; + utils::Synchronized, utils::SpinLock> committed_transactions_; utils::Scheduler gc_runner_; std::mutex gc_lock_; - // Undo buffers that were unlinked and now are waiting to be freed. - utils::Synchronized>>, utils::SpinLock> garbage_undo_buffers_; + using BondPmrLd = Bond>; + // Ownership of unlinked deltas is transfered to garabage_undo_buffers once transaction is commited + utils::Synchronized>, utils::SpinLock> garbage_undo_buffers_; // Vertices that are logically deleted but still have to be removed from // indices before removing them from the main storage. diff --git a/src/storage/v2/inmemory/unique_constraints.cpp b/src/storage/v2/inmemory/unique_constraints.cpp index 71ff1e160..002385cf5 100644 --- a/src/storage/v2/inmemory/unique_constraints.cpp +++ b/src/storage/v2/inmemory/unique_constraints.cpp @@ -65,7 +65,7 @@ bool LastCommittedVersionHasLabelProperty(const Vertex &vertex, LabelId label, c while (delta != nullptr) { auto ts = delta->timestamp->load(std::memory_order_acquire); - if (ts < commit_timestamp || ts == transaction.transaction_id.load(std::memory_order_acquire)) { + if (ts < commit_timestamp || ts == transaction.transaction_id) { break; } diff --git a/src/storage/v2/mvcc.hpp b/src/storage/v2/mvcc.hpp index 9963b6761..2be20f803 100644 --- a/src/storage/v2/mvcc.hpp +++ b/src/storage/v2/mvcc.hpp @@ -41,7 +41,8 @@ inline std::size_t ApplyDeltasForRead(Transaction const *transaction, const Delt // This allows the transaction to see its changes even though it's committed. const auto commit_timestamp = transaction->commit_timestamp ? transaction->commit_timestamp->load(std::memory_order_acquire) - : transaction->transaction_id.load(std::memory_order_acquire); + : transaction->transaction_id; + std::size_t n_processed = 0; while (delta != nullptr) { // For SNAPSHOT ISOLATION -> we can only see the changes which were committed before the start of the current @@ -94,7 +95,7 @@ template inline bool PrepareForWrite(Transaction *transaction, TObj *object) { if (object->delta == nullptr) return true; auto ts = object->delta->timestamp->load(std::memory_order_acquire); - if (ts == transaction->transaction_id.load(std::memory_order_acquire) || ts < transaction->start_timestamp) { + if (ts == transaction->transaction_id || ts < transaction->start_timestamp) { return true; } @@ -112,8 +113,8 @@ inline Delta *CreateDeleteObjectDelta(Transaction *transaction) { return nullptr; } transaction->EnsureCommitTimestampExists(); - return &transaction->deltas.emplace_back(Delta::DeleteObjectTag(), transaction->commit_timestamp.get(), - transaction->command_id); + return &transaction->deltas.use().emplace_back(Delta::DeleteObjectTag(), transaction->commit_timestamp.get(), + transaction->command_id); } inline Delta *CreateDeleteObjectDelta(Transaction *transaction, std::list *deltas) { @@ -125,11 +126,12 @@ inline Delta *CreateDeleteObjectDelta(Transaction *transaction, std::list } /// TODO: what if in-memory analytical + inline Delta *CreateDeleteDeserializedObjectDelta(Transaction *transaction, std::optional old_disk_key, std::string &&ts) { - // Should use utils::DecodeFixed64(ts.c_str()) once we will move to RocksDB real timestamps transaction->EnsureCommitTimestampExists(); - return &transaction->deltas.emplace_back(Delta::DeleteDeserializedObjectTag(), std::stoull(ts), old_disk_key); + // Should use utils::DecodeFixed64(ts.c_str()) once we will move to RocksDB real timestamps + return &transaction->deltas.use().emplace_back(Delta::DeleteDeserializedObjectTag(), std::stoull(ts), old_disk_key); } inline Delta *CreateDeleteDeserializedObjectDelta(std::list *deltas, std::optional old_disk_key, @@ -159,8 +161,8 @@ inline void CreateAndLinkDelta(Transaction *transaction, TObj *object, Args &&.. return; } transaction->EnsureCommitTimestampExists(); - auto delta = &transaction->deltas.emplace_back(std::forward(args)..., transaction->commit_timestamp.get(), - transaction->command_id); + auto delta = &transaction->deltas.use().emplace_back(std::forward(args)..., transaction->commit_timestamp.get(), + transaction->command_id); // The operations are written in such order so that both `next` and `prev` // chains are valid at all times. The chains must be valid at all times diff --git a/src/storage/v2/storage.cpp b/src/storage/v2/storage.cpp index bda814ddb..1ac88cfb1 100644 --- a/src/storage/v2/storage.cpp +++ b/src/storage/v2/storage.cpp @@ -117,7 +117,7 @@ StorageMode Storage::Accessor::GetCreationStorageMode() const { return creation_ std::optional Storage::Accessor::GetTransactionId() const { if (is_transaction_active_) { - return transaction_.transaction_id.load(std::memory_order_acquire); + return transaction_.transaction_id; } return {}; } diff --git a/src/storage/v2/transaction.hpp b/src/storage/v2/transaction.hpp index 1c2cb3165..8f0132062 100644 --- a/src/storage/v2/transaction.hpp +++ b/src/storage/v2/transaction.hpp @@ -16,6 +16,7 @@ #include #include +#include "utils/memory.hpp" #include "utils/skip_list.hpp" #include "storage/v2/delta.hpp" @@ -27,11 +28,14 @@ #include "storage/v2/vertex.hpp" #include "storage/v2/vertex_info_cache.hpp" #include "storage/v2/view.hpp" +#include "utils/bond.hpp" +#include "utils/pmr/list.hpp" namespace memgraph::storage { const uint64_t kTimestampInitialId = 0; const uint64_t kTransactionInitialId = 1ULL << 63U; +using PmrListDelta = utils::pmr::list; struct Transaction { Transaction(uint64_t transaction_id, uint64_t start_timestamp, IsolationLevel isolation_level, @@ -39,13 +43,14 @@ struct Transaction { : transaction_id(transaction_id), start_timestamp(start_timestamp), command_id(0), + deltas(1024UL), must_abort(false), isolation_level(isolation_level), storage_mode(storage_mode), edge_import_mode_active(edge_import_mode_active) {} Transaction(Transaction &&other) noexcept - : transaction_id(other.transaction_id.load(std::memory_order_acquire)), + : transaction_id(other.transaction_id), start_timestamp(other.start_timestamp), commit_timestamp(std::move(other.commit_timestamp)), command_id(other.command_id), @@ -67,7 +72,7 @@ struct Transaction { /// @throw std::bad_alloc if failed to create the `commit_timestamp` void EnsureCommitTimestampExists() { if (commit_timestamp != nullptr) return; - commit_timestamp = std::make_unique>(transaction_id.load(std::memory_order_relaxed)); + commit_timestamp = std::make_unique>(transaction_id); } void AddModifiedEdge(Gid gid, ModifiedEdgeInfo modified_edge) { @@ -78,7 +83,7 @@ struct Transaction { void RemoveModifiedEdge(const Gid &gid) { modified_edges_.erase(gid); } - std::atomic transaction_id; + uint64_t transaction_id; uint64_t start_timestamp; // The `Transaction` object is stack allocated, but the `commit_timestamp` // must be heap allocated because `Delta`s have a pointer to it, and that @@ -86,7 +91,8 @@ struct Transaction { // `commited_transactions_` list for GC. std::unique_ptr> commit_timestamp; uint64_t command_id; - std::list deltas; + + Bond deltas; bool must_abort; IsolationLevel isolation_level; StorageMode storage_mode; @@ -102,16 +108,12 @@ struct Transaction { }; inline bool operator==(const Transaction &first, const Transaction &second) { - return first.transaction_id.load(std::memory_order_acquire) == second.transaction_id.load(std::memory_order_acquire); + return first.transaction_id == second.transaction_id; } inline bool operator<(const Transaction &first, const Transaction &second) { - return first.transaction_id.load(std::memory_order_acquire) < second.transaction_id.load(std::memory_order_acquire); -} -inline bool operator==(const Transaction &first, const uint64_t &second) { - return first.transaction_id.load(std::memory_order_acquire) == second; -} -inline bool operator<(const Transaction &first, const uint64_t &second) { - return first.transaction_id.load(std::memory_order_acquire) < second; + return first.transaction_id < second.transaction_id; } +inline bool operator==(const Transaction &first, const uint64_t &second) { return first.transaction_id == second; } +inline bool operator<(const Transaction &first, const uint64_t &second) { return first.transaction_id < second; } } // namespace memgraph::storage diff --git a/src/utils/bond.hpp b/src/utils/bond.hpp new file mode 100644 index 000000000..7ea3c2aba --- /dev/null +++ b/src/utils/bond.hpp @@ -0,0 +1,57 @@ +// 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. + +#include +#include +#include "storage/v2/delta.hpp" +#include "utils/memory.hpp" +#include "utils/pmr/list.hpp" + +/// struct Bond presents the association of a collection and its resource +/// and makes them tightly bound for easier handling of construction, moving and destruction +/// of container +template +struct Bond { + using resource = memgraph::utils::MonotonicBufferResource; + + explicit Bond(std::size_t initial_size) + : res_(std::make_unique(initial_size)), + container_(memgraph::utils::Allocator(res_.get()).template new_object()){}; + + Bond(Bond &&other) noexcept : res_(std::exchange(other.res_, nullptr)), container_(other.container_) { + other.container_ = nullptr; + } + + Bond(const Bond &other) = delete; + + Bond &operator=(const Bond &other) = delete; + + Bond &operator=(Bond &&other) = delete; + + auto use() -> Container & { return *container_; } + + auto use() const -> const Container & { return *container_; } + + auto res() -> resource * { return res_.get(); } + + ~Bond() { + if (res_) { + memgraph::utils::Allocator(res_.get()).delete_object(container_); + container_ = nullptr; + res_->Release(); + res_ = nullptr; + } + } + + private: + std::unique_ptr res_{nullptr}; + Container *container_{nullptr}; +}; diff --git a/src/utils/memory_tracker.cpp b/src/utils/memory_tracker.cpp index e8087c8dc..fa9910e5e 100644 --- a/src/utils/memory_tracker.cpp +++ b/src/utils/memory_tracker.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 Memgraph Ltd. +// 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 diff --git a/src/utils/temporal.cpp b/src/utils/temporal.cpp index dd7fceeea..e384bcd54 100644 --- a/src/utils/temporal.cpp +++ b/src/utils/temporal.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 Memgraph Ltd. +// 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 diff --git a/tests/unit/formatters.hpp b/tests/unit/formatters.hpp index f9b8fbba4..a5ee49166 100644 --- a/tests/unit/formatters.hpp +++ b/tests/unit/formatters.hpp @@ -1,4 +1,4 @@ -// Copyright 2022 Memgraph Ltd. +// 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 diff --git a/tests/unit/storage_v2_wal_file.cpp b/tests/unit/storage_v2_wal_file.cpp index 8aeb42fc2..dd15c511b 100644 --- a/tests/unit/storage_v2_wal_file.cpp +++ b/tests/unit/storage_v2_wal_file.cpp @@ -25,6 +25,7 @@ #include "storage_test_utils.hpp" #include "utils/file.hpp" #include "utils/file_locker.hpp" +#include "utils/memory.hpp" #include "utils/uuid.hpp" // Helper function used to convert between enum types. @@ -144,8 +145,8 @@ class DeltaGenerator final { void Finalize(bool append_transaction_end = true) { auto commit_timestamp = gen_->timestamp_++; - if (transaction_.deltas.empty()) return; - for (const auto &delta : transaction_.deltas) { + if (transaction_.deltas.use().empty()) return; + for (const auto &delta : transaction_.deltas.use()) { auto owner = delta.prev.Get(); while (owner.type == memgraph::storage::PreviousPtr::Type::DELTA) { owner = owner.delta->prev.Get(); @@ -161,7 +162,7 @@ class DeltaGenerator final { if (append_transaction_end) { gen_->wal_file_.AppendTransactionEnd(commit_timestamp); if (gen_->valid_) { - gen_->UpdateStats(commit_timestamp, transaction_.deltas.size() + 1); + gen_->UpdateStats(commit_timestamp, transaction_.deltas.use().size() + 1); for (auto &data : data_) { if (data.type == memgraph::storage::durability::WalDeltaData::Type::VERTEX_SET_PROPERTY) { // We need to put the final property value into the SET_PROPERTY