From a6fcdfd90553e57cf2ec5329a873defa25d2bbda Mon Sep 17 00:00:00 2001 From: Gareth Andrew Lloyd <gareth.lloyd@memgraph.io> Date: Tue, 27 Feb 2024 14:45:08 +0000 Subject: [PATCH 01/18] Make GC + snapshot, main lock friendly (#1759) - Only IN_MEMORY_ANALYTICAL requires unique lock during snapshot - GC in some cases will be provide with unique lock - This fact can be used for optimisations - In all other cases, optimisations should be done with alternative check. Not via getting a unique lock Also: - Faster property lookup - Faster index iteration (better conditional branching) --- src/storage/v2/disk/storage.hpp | 2 +- .../v2/inmemory/label_property_index.cpp | 23 +-- src/storage/v2/inmemory/storage.cpp | 142 +++++++++--------- src/storage/v2/inmemory/storage.hpp | 4 +- src/storage/v2/property_store.cpp | 38 ++++- src/storage/v2/storage.hpp | 12 +- src/utils/resource_lock.hpp | 11 ++ 7 files changed, 138 insertions(+), 94 deletions(-) diff --git a/src/storage/v2/disk/storage.hpp b/src/storage/v2/disk/storage.hpp index 5803f15d3..4d71fd10b 100644 --- a/src/storage/v2/disk/storage.hpp +++ b/src/storage/v2/disk/storage.hpp @@ -310,7 +310,7 @@ class DiskStorage final : public Storage { StorageInfo GetBaseInfo() override; StorageInfo GetInfo(memgraph::replication_coordination_glue::ReplicationRole replication_role) override; - void FreeMemory(std::unique_lock<utils::ResourceLock> /*lock*/) override {} + void FreeMemory(std::unique_lock<utils::ResourceLock> /*lock*/, bool /*periodic*/) override {} void PrepareForNewEpoch() override { throw utils::BasicException("Disk storage mode does not support replication."); } diff --git a/src/storage/v2/inmemory/label_property_index.cpp b/src/storage/v2/inmemory/label_property_index.cpp index 59b12a779..71e3be4b8 100644 --- a/src/storage/v2/inmemory/label_property_index.cpp +++ b/src/storage/v2/inmemory/label_property_index.cpp @@ -144,27 +144,30 @@ void InMemoryLabelPropertyIndex::RemoveObsoleteEntries(uint64_t oldest_active_st auto maybe_stop = utils::ResettableCounter<2048>(); for (auto &[label_property, index] : index_) { + auto [label_id, prop_id] = label_property; // before starting index, check if stop_requested if (token.stop_requested()) return; auto index_acc = index.access(); - for (auto it = index_acc.begin(); it != index_acc.end();) { + auto it = index_acc.begin(); + auto end_it = index_acc.end(); + if (it == end_it) continue; + while (true) { // Hot loop, don't check stop_requested every time if (maybe_stop() && token.stop_requested()) return; auto next_it = it; ++next_it; - if (it->timestamp >= oldest_active_start_timestamp) { - it = next_it; - continue; - } - - if ((next_it != index_acc.end() && it->vertex == next_it->vertex && it->value == next_it->value) || - !AnyVersionHasLabelProperty(*it->vertex, label_property.first, label_property.second, it->value, - oldest_active_start_timestamp)) { - index_acc.remove(*it); + bool has_next = next_it != end_it; + if (it->timestamp < oldest_active_start_timestamp) { + bool redundant_duplicate = has_next && it->vertex == next_it->vertex && it->value == next_it->value; + if (redundant_duplicate || + !AnyVersionHasLabelProperty(*it->vertex, label_id, prop_id, it->value, oldest_active_start_timestamp)) { + index_acc.remove(*it); + } } + if (!has_next) break; it = next_it; } } diff --git a/src/storage/v2/inmemory/storage.cpp b/src/storage/v2/inmemory/storage.cpp index b09d75a30..bd8534673 100644 --- a/src/storage/v2/inmemory/storage.cpp +++ b/src/storage/v2/inmemory/storage.cpp @@ -143,9 +143,7 @@ InMemoryStorage::InMemoryStorage(Config config) if (config_.gc.type == Config::Gc::Type::PERIODIC) { // TODO: move out of storage have one global gc_runner_ - gc_runner_.Run("Storage GC", config_.gc.interval, [this] { - this->FreeMemory(std::unique_lock<utils::ResourceLock>{main_lock_, std::defer_lock}); - }); + gc_runner_.Run("Storage GC", config_.gc.interval, [this] { this->FreeMemory({}, true); }); } if (timestamp_ == kTimestampInitialId) { commit_log_.emplace(); @@ -1425,28 +1423,27 @@ void InMemoryStorage::SetStorageMode(StorageMode new_storage_mode) { } storage_mode_ = new_storage_mode; - FreeMemory(std::move(main_guard)); + FreeMemory(std::move(main_guard), false); } } -template <bool force> -void InMemoryStorage::CollectGarbage(std::unique_lock<utils::ResourceLock> main_guard) { +template <bool aggressive = true> +void InMemoryStorage::CollectGarbage(std::unique_lock<utils::ResourceLock> main_guard, bool periodic) { // NOTE: You do not need to consider cleanup of deleted object that occurred in // different storage modes within the same CollectGarbage call. This is because // SetStorageMode will ensure CollectGarbage is called before any new transactions // with the new storage mode can start. // SetStorageMode will pass its unique_lock of main_lock_. We will use that lock, - // as reacquiring the lock would cause deadlock. Otherwise, we need to get our own + // as reacquiring the lock would cause deadlock. Otherwise, we need to get our own // lock. if (!main_guard.owns_lock()) { - if constexpr (force) { - // We take the unique lock on the main storage lock, so we can forcefully clean - // everything we can - if (!main_lock_.try_lock()) { - CollectGarbage<false>(); - return; - } + if constexpr (aggressive) { + // We tried to be aggressive but we do not already have main lock continue as not aggressive + // Perf note: Do not try to get unique lock if it was not already passed in. GC maybe expensive, + // do not assume it is fast, unique lock will blocks all new storage transactions. + CollectGarbage<false>({}, periodic); + 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 @@ -1458,17 +1455,24 @@ void InMemoryStorage::CollectGarbage(std::unique_lock<utils::ResourceLock> main_ } utils::OnScopeExit lock_releaser{[&] { - if (!main_guard.owns_lock()) { - if constexpr (force) { - main_lock_.unlock(); - } else { - main_lock_.unlock_shared(); - } - } else { + if (main_guard.owns_lock()) { main_guard.unlock(); + } else { + main_lock_.unlock_shared(); } }}; + // Only one gc run at a time + std::unique_lock<std::mutex> gc_guard(gc_lock_, std::try_to_lock); + if (!gc_guard.owns_lock()) { + return; + } + + // Diagnostic trace + spdlog::trace("Storage GC on '{}' started [{}]", name(), periodic ? "periodic" : "forced"); + auto trace_on_exit = utils::OnScopeExit{ + [&] { spdlog::trace("Storage GC on '{}' finished [{}]", name(), periodic ? "periodic" : "forced"); }}; + // 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 // the version chains. They cannot be deleted immediately, because there @@ -1476,27 +1480,29 @@ void InMemoryStorage::CollectGarbage(std::unique_lock<utils::ResourceLock> main_ // chain traversal. They are instead marked for deletion and will be deleted // in the second GC phase in this GC iteration or some of the following // ones. - std::unique_lock<std::mutex> gc_guard(gc_lock_, std::try_to_lock); - if (!gc_guard.owns_lock()) { - return; - } uint64_t oldest_active_start_timestamp = commit_log_->OldestActive(); - // Deltas from previous GC runs or from aborts can be cleaned up here - garbage_undo_buffers_.WithLock([&](auto &garbage_undo_buffers) { - if constexpr (force) { - // if force is set to true we can simply delete all the leftover undos because - // no transaction is active - garbage_undo_buffers.clear(); - } else { - // garbage_undo_buffers is ordered, pop until we can't - while (!garbage_undo_buffers.empty() && - garbage_undo_buffers.front().mark_timestamp_ <= oldest_active_start_timestamp) { - garbage_undo_buffers.pop_front(); + { + std::unique_lock<utils::SpinLock> guard(engine_lock_); + uint64_t mark_timestamp = timestamp_; // a timestamp no active transaction can currently have + + // Deltas from previous GC runs or from aborts can be cleaned up here + garbage_undo_buffers_.WithLock([&](auto &garbage_undo_buffers) { + guard.unlock(); + if (aggressive or mark_timestamp == oldest_active_start_timestamp) { + // We know no transaction is active, it is safe to simply delete all the garbage undos + // Nothing can be reading them + garbage_undo_buffers.clear(); + } else { + // garbage_undo_buffers is ordered, pop until we can't + while (!garbage_undo_buffers.empty() && + garbage_undo_buffers.front().mark_timestamp_ <= oldest_active_start_timestamp) { + garbage_undo_buffers.pop_front(); + } } - } - }); + }); + } // We don't move undo buffers of unlinked transactions to garbage_undo_buffers // list immediately, because we would have to repeatedly take @@ -1694,7 +1700,8 @@ void InMemoryStorage::CollectGarbage(std::unique_lock<utils::ResourceLock> main_ std::unique_lock<utils::SpinLock> guard(engine_lock_); uint64_t mark_timestamp = timestamp_; // a timestamp no active transaction can currently have - if (force or mark_timestamp == oldest_active_start_timestamp) { + if (aggressive or mark_timestamp == oldest_active_start_timestamp) { + guard.unlock(); // if lucky, there are no active transactions, hence nothing looking at the deltas // remove them now unlinked_undo_buffers.clear(); @@ -1756,8 +1763,8 @@ void InMemoryStorage::CollectGarbage(std::unique_lock<utils::ResourceLock> main_ } // tell the linker he can find the CollectGarbage definitions here -template void InMemoryStorage::CollectGarbage<true>(std::unique_lock<utils::ResourceLock>); -template void InMemoryStorage::CollectGarbage<false>(std::unique_lock<utils::ResourceLock>); +template void InMemoryStorage::CollectGarbage<true>(std::unique_lock<utils::ResourceLock> main_guard, bool periodic); +template void InMemoryStorage::CollectGarbage<false>(std::unique_lock<utils::ResourceLock> main_guard, bool periodic); StorageInfo InMemoryStorage::GetBaseInfo() { StorageInfo info{}; @@ -2108,50 +2115,35 @@ void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOpera utils::BasicResult<InMemoryStorage::CreateSnapshotError> InMemoryStorage::CreateSnapshot( memgraph::replication_coordination_glue::ReplicationRole replication_role) { - if (replication_role == memgraph::replication_coordination_glue::ReplicationRole::REPLICA) { + using memgraph::replication_coordination_glue::ReplicationRole; + if (replication_role == ReplicationRole::REPLICA) { return InMemoryStorage::CreateSnapshotError::DisabledForReplica; } - auto const &epoch = repl_storage_state_.epoch_; - auto snapshot_creator = [this, &epoch]() { - utils::Timer timer; - auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION, storage_mode_, - memgraph::replication_coordination_glue::ReplicationRole::MAIN); - durability::CreateSnapshot(this, &transaction, recovery_.snapshot_directory_, recovery_.wal_directory_, &vertices_, - &edges_, uuid_, epoch, repl_storage_state_.history, &file_retainer_); - // Finalize snapshot transaction. - commit_log_->MarkFinished(transaction.start_timestamp); - - memgraph::metrics::Measure(memgraph::metrics::SnapshotCreationLatency_us, - std::chrono::duration_cast<std::chrono::microseconds>(timer.Elapsed()).count()); - }; std::lock_guard snapshot_guard(snapshot_lock_); - auto should_try_shared{true}; - auto max_num_tries{10}; - while (max_num_tries) { - if (should_try_shared) { - std::shared_lock storage_guard(main_lock_); - if (storage_mode_ == memgraph::storage::StorageMode::IN_MEMORY_TRANSACTIONAL) { - snapshot_creator(); - return {}; - } + auto accessor = std::invoke([&]() { + if (storage_mode_ == StorageMode::IN_MEMORY_ANALYTICAL) { + // For analytical no other txn can be in play + return UniqueAccess(ReplicationRole::MAIN, IsolationLevel::SNAPSHOT_ISOLATION); } else { - std::unique_lock main_guard{main_lock_}; - if (storage_mode_ == memgraph::storage::StorageMode::IN_MEMORY_ANALYTICAL) { - snapshot_creator(); - return {}; - } + return Access(ReplicationRole::MAIN, IsolationLevel::SNAPSHOT_ISOLATION); } - should_try_shared = !should_try_shared; - max_num_tries--; - } + }); - return CreateSnapshotError::ReachedMaxNumTries; + utils::Timer timer; + Transaction *transaction = accessor->GetTransaction(); + auto const &epoch = repl_storage_state_.epoch_; + durability::CreateSnapshot(this, transaction, recovery_.snapshot_directory_, recovery_.wal_directory_, &vertices_, + &edges_, uuid_, epoch, repl_storage_state_.history, &file_retainer_); + + memgraph::metrics::Measure(memgraph::metrics::SnapshotCreationLatency_us, + std::chrono::duration_cast<std::chrono::microseconds>(timer.Elapsed()).count()); + return {}; } -void InMemoryStorage::FreeMemory(std::unique_lock<utils::ResourceLock> main_guard) { - CollectGarbage<true>(std::move(main_guard)); +void InMemoryStorage::FreeMemory(std::unique_lock<utils::ResourceLock> main_guard, bool periodic) { + CollectGarbage(std::move(main_guard), periodic); static_cast<InMemoryLabelIndex *>(indices_.label_index_.get())->RunGC(); static_cast<InMemoryLabelPropertyIndex *>(indices_.label_property_index_.get())->RunGC(); diff --git a/src/storage/v2/inmemory/storage.hpp b/src/storage/v2/inmemory/storage.hpp index 0b62be7b3..c0e46d0c9 100644 --- a/src/storage/v2/inmemory/storage.hpp +++ b/src/storage/v2/inmemory/storage.hpp @@ -332,7 +332,7 @@ class InMemoryStorage final : public Storage { std::unique_ptr<Accessor> UniqueAccess(memgraph::replication_coordination_glue::ReplicationRole replication_role, std::optional<IsolationLevel> override_isolation_level) override; - void FreeMemory(std::unique_lock<utils::ResourceLock> main_guard) override; + void FreeMemory(std::unique_lock<utils::ResourceLock> main_guard, bool periodic) override; utils::FileRetainer::FileLockerAccessor::ret_type IsPathLocked(); utils::FileRetainer::FileLockerAccessor::ret_type LockPath(); @@ -363,7 +363,7 @@ class InMemoryStorage final : public Storage { /// @throw std::system_error /// @throw std::bad_alloc template <bool force> - void CollectGarbage(std::unique_lock<utils::ResourceLock> main_guard = {}); + void CollectGarbage(std::unique_lock<utils::ResourceLock> main_guard, bool periodic); bool InitializeWalFile(memgraph::replication::ReplicationEpoch &epoch); void FinalizeWalFile(); diff --git a/src/storage/v2/property_store.cpp b/src/storage/v2/property_store.cpp index e6e4dbbaf..adf3440a2 100644 --- a/src/storage/v2/property_store.cpp +++ b/src/storage/v2/property_store.cpp @@ -1051,6 +1051,14 @@ struct SpecificPropertyAndBufferInfo { uint64_t all_size; }; +// Struct used to return info about the property position +struct SpecificPropertyAndBufferInfoMinimal { + uint64_t property_begin; + uint64_t property_end; + + auto property_size() const { return property_end - property_begin; } +}; + // Function used to find the position where the property should be in the data // buffer. It keeps the properties in the buffer sorted by `PropertyId` and // returns the positions in the buffer where the seeked property starts and @@ -1083,6 +1091,27 @@ SpecificPropertyAndBufferInfo FindSpecificPropertyAndBufferInfo(Reader *reader, return {property_begin, property_end, property_end - property_begin, all_begin, all_end, all_end - all_begin}; } +// Like FindSpecificPropertyAndBufferInfo, but will early exit. No need to find the "all" information +SpecificPropertyAndBufferInfoMinimal FindSpecificPropertyAndBufferInfoMinimal(Reader *reader, PropertyId property) { + uint64_t property_begin = reader->GetPosition(); + while (true) { + switch (HasExpectedProperty(reader, property)) { + case ExpectedPropertyStatus::MISSING_DATA: + [[fallthrough]]; + case ExpectedPropertyStatus::GREATER: { + return {0, 0}; + } + case ExpectedPropertyStatus::EQUAL: { + return {property_begin, reader->GetPosition()}; + } + case ExpectedPropertyStatus::SMALLER: { + property_begin = reader->GetPosition(); + break; + } + } + } +} + // All data buffers will be allocated to a power of 8 size. uint64_t ToPowerOf8(uint64_t size) { uint64_t mod = size % 8; @@ -1254,11 +1283,12 @@ bool PropertyStore::IsPropertyEqual(PropertyId property, const PropertyValue &va BufferInfo buffer_info = GetBufferInfo(buffer_); Reader reader(buffer_info.data, buffer_info.size); - auto info = FindSpecificPropertyAndBufferInfo(&reader, property); - if (info.property_size == 0) return value.IsNull(); - Reader prop_reader(buffer_info.data + info.property_begin, info.property_size); + auto info = FindSpecificPropertyAndBufferInfoMinimal(&reader, property); + auto property_size = info.property_size(); + if (property_size == 0) return value.IsNull(); + Reader prop_reader(buffer_info.data + info.property_begin, property_size); if (!CompareExpectedProperty(&prop_reader, property, value)) return false; - return prop_reader.GetPosition() == info.property_size; + return prop_reader.GetPosition() == property_size; } std::map<PropertyId, PropertyValue> PropertyStore::Properties() const { diff --git a/src/storage/v2/storage.hpp b/src/storage/v2/storage.hpp index ddbe6d8a7..5868d70a3 100644 --- a/src/storage/v2/storage.hpp +++ b/src/storage/v2/storage.hpp @@ -284,6 +284,8 @@ class Storage { virtual UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties) = 0; + auto GetTransaction() -> Transaction * { return std::addressof(transaction_); } + protected: Storage *storage_; std::shared_lock<utils::ResourceLock> storage_guard_; @@ -336,9 +338,15 @@ class Storage { StorageMode GetStorageMode() const noexcept; - virtual void FreeMemory(std::unique_lock<utils::ResourceLock> main_guard) = 0; + virtual void FreeMemory(std::unique_lock<utils::ResourceLock> main_guard, bool periodic) = 0; - void FreeMemory() { FreeMemory({}); } + void FreeMemory() { + if (storage_mode_ == StorageMode::IN_MEMORY_ANALYTICAL) { + FreeMemory(std::unique_lock{main_lock_}, false); + } else { + FreeMemory({}, false); + } + } virtual std::unique_ptr<Accessor> Access(memgraph::replication_coordination_glue::ReplicationRole replication_role, std::optional<IsolationLevel> override_isolation_level) = 0; diff --git a/src/utils/resource_lock.hpp b/src/utils/resource_lock.hpp index 7a3ef9444..1a812935a 100644 --- a/src/utils/resource_lock.hpp +++ b/src/utils/resource_lock.hpp @@ -66,6 +66,17 @@ struct ResourceLock { } return false; } + + template <typename Rep, typename Period> + bool try_lock_shared_for(std::chrono::duration<Rep, Period> const &time) { + auto lock = std::unique_lock{mtx}; + // block until available + if (!cv.wait_for(lock, time, [this] { return state != UNIQUE; })) return false; + state = SHARED; + ++count; + return true; + } + void unlock() { auto lock = std::unique_lock{mtx}; state = UNLOCKED; From da898be8f901e85518334ce9ae0a2e8175e7dbc7 Mon Sep 17 00:00:00 2001 From: Gareth Andrew Lloyd <gareth.lloyd@memgraph.io> Date: Tue, 27 Feb 2024 17:21:52 +0000 Subject: [PATCH 02/18] Compact Delta 80B -> 56B (#1747) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Make special structure for old_disk_key. std::optional<std::string> was 40B, which is the largest member of out action union. Replaced with 8B, structure. This makes largest member now vertex_edge at 24B, this means Delta is now only 56B. 🥳🎉 Now less than a cacheline 🎊✨ --- src/memory/global_memory_control.cpp | 4 ++-- src/mg_import_csv.cpp | 2 +- src/query/procedure/mg_procedure_impl.cpp | 2 +- src/storage/v2/delta.hpp | 26 ++++++++++++++++++++--- src/utils/disk_utils.hpp | 2 +- src/utils/string.hpp | 2 +- 6 files changed, 29 insertions(+), 9 deletions(-) diff --git a/src/memory/global_memory_control.cpp b/src/memory/global_memory_control.cpp index bcf12bd2c..6073f9d9a 100644 --- a/src/memory/global_memory_control.cpp +++ b/src/memory/global_memory_control.cpp @@ -122,11 +122,11 @@ static bool my_commit(extent_hooks_t *extent_hooks, void *addr, size_t size, siz [[maybe_unused]] auto blocker = memgraph::utils::MemoryTracker::OutOfMemoryExceptionBlocker{}; if (GetQueriesMemoryControl().IsThreadTracked()) [[unlikely]] { - bool ok = GetQueriesMemoryControl().TrackAllocOnCurrentThread(length); + [[maybe_unused]] bool ok = GetQueriesMemoryControl().TrackAllocOnCurrentThread(length); DMG_ASSERT(ok); } - auto ok = memgraph::utils::total_memory_tracker.Alloc(static_cast<int64_t>(length)); + [[maybe_unused]] auto ok = memgraph::utils::total_memory_tracker.Alloc(static_cast<int64_t>(length)); DMG_ASSERT(ok); return false; diff --git a/src/mg_import_csv.cpp b/src/mg_import_csv.cpp index 2d77c2db2..75a7c3bbb 100644 --- a/src/mg_import_csv.cpp +++ b/src/mg_import_csv.cpp @@ -416,7 +416,7 @@ memgraph::storage::PropertyValue StringToValue(const std::string &str, const std std::string GetIdSpace(const std::string &type) { // The format of this field is as follows: // [START_|END_]ID[(<id_space>)] - std::regex format(R"(^(START_|END_)?ID(\(([^\(\)]+)\))?$)", std::regex::extended); + static std::regex format(R"(^(START_|END_)?ID(\(([^\(\)]+)\))?$)", std::regex::extended); std::smatch res; if (!std::regex_match(type, res, format)) throw LoadException( diff --git a/src/query/procedure/mg_procedure_impl.cpp b/src/query/procedure/mg_procedure_impl.cpp index 647f3e14d..d6ce3c7b7 100644 --- a/src/query/procedure/mg_procedure_impl.cpp +++ b/src/query/procedure/mg_procedure_impl.cpp @@ -3798,7 +3798,7 @@ void PrintFuncSignature(const mgp_func &func, std::ostream &stream) { bool IsValidIdentifierName(const char *name) { if (!name) return false; - std::regex regex("[_[:alpha:]][_[:alnum:]]*"); + static std::regex regex("[_[:alpha:]][_[:alnum:]]*"); return std::regex_match(name, regex); } diff --git a/src/storage/v2/delta.hpp b/src/storage/v2/delta.hpp index 9c70bdc4c..60d589dcd 100644 --- a/src/storage/v2/delta.hpp +++ b/src/storage/v2/delta.hpp @@ -123,6 +123,26 @@ inline bool operator==(const PreviousPtr::Pointer &a, const PreviousPtr::Pointer inline bool operator!=(const PreviousPtr::Pointer &a, const PreviousPtr::Pointer &b) { return !(a == b); } +struct opt_str { + opt_str(std::optional<std::string> const &other) : str_{other ? new_cstr(*other) : nullptr} {} + + ~opt_str() { delete[] str_; } + + auto as_opt_str() const -> std::optional<std::string> { + if (!str_) return std::nullopt; + return std::optional<std::string>{std::in_place, str_}; + } + + private: + static auto new_cstr(std::string const &str) -> char const * { + auto *mem = new char[str.length() + 1]; + strcpy(mem, str.c_str()); + return mem; + } + + char const *str_ = nullptr; +}; + struct Delta { enum class Action : std::uint8_t { /// Use for Vertex and Edge @@ -160,7 +180,7 @@ struct Delta { // Because of this object was created in past txs, we create timestamp by ourselves inside instead of having it from // current tx. This timestamp we got from RocksDB timestamp stored in key. Delta(DeleteDeserializedObjectTag /*tag*/, uint64_t ts, std::optional<std::string> old_disk_key) - : timestamp(new std::atomic<uint64_t>(ts)), command_id(0), old_disk_key{.value = std::move(old_disk_key)} {} + : timestamp(new std::atomic<uint64_t>(ts)), command_id(0), old_disk_key{.value = old_disk_key} {} Delta(DeleteObjectTag /*tag*/, std::atomic<uint64_t> *timestamp, uint64_t command_id) : timestamp(timestamp), command_id(command_id), action(Action::DELETE_OBJECT) {} @@ -222,7 +242,7 @@ struct Delta { case Action::REMOVE_OUT_EDGE: break; case Action::DELETE_DESERIALIZED_OBJECT: - old_disk_key.value.reset(); + std::destroy_at(&old_disk_key.value); delete timestamp; timestamp = nullptr; break; @@ -242,7 +262,7 @@ struct Delta { Action action; struct { Action action = Action::DELETE_DESERIALIZED_OBJECT; - std::optional<std::string> value; + opt_str value; } old_disk_key; struct { Action action; diff --git a/src/utils/disk_utils.hpp b/src/utils/disk_utils.hpp index c4b9accd6..29e0c9893 100644 --- a/src/utils/disk_utils.hpp +++ b/src/utils/disk_utils.hpp @@ -21,7 +21,7 @@ inline std::optional<std::string> GetOldDiskKeyOrNull(storage::Delta *head) { head = head->next; } if (head->action == storage::Delta::Action::DELETE_DESERIALIZED_OBJECT) { - return head->old_disk_key.value; + return head->old_disk_key.value.as_opt_str(); } return std::nullopt; } diff --git a/src/utils/string.hpp b/src/utils/string.hpp index e5c4c4f3c..31723bb65 100644 --- a/src/utils/string.hpp +++ b/src/utils/string.hpp @@ -242,7 +242,7 @@ std::vector<TString, TAllocator> *Split(std::vector<TString, TAllocator> *out, c if (src.empty()) return out; // TODO: Investigate how much regex allocate and perhaps replace with custom // solution doing no allocations. - std::regex not_whitespace("[^\\s]+"); + static std::regex not_whitespace("[^\\s]+"); auto matches_begin = std::cregex_iterator(src.data(), src.data() + src.size(), not_whitespace); auto matches_end = std::cregex_iterator(); out->reserve(std::distance(matches_begin, matches_end)); From b7de79d5a0b9b00966f17d67e49acbf1ba01658f Mon Sep 17 00:00:00 2001 From: DavIvek <david.ivekovic@memgraph.io> Date: Tue, 27 Feb 2024 22:40:55 +0100 Subject: [PATCH 03/18] Fix schema.node_type_properties() and schema.rel_type_properties() (#1718) --- query_modules/schema.cpp | 125 +++++++++++++------------ tests/e2e/query_modules/schema_test.py | 111 +++++++++++++++++++--- 2 files changed, 161 insertions(+), 75 deletions(-) diff --git a/query_modules/schema.cpp b/query_modules/schema.cpp index 848ccedc4..9c2380284 100644 --- a/query_modules/schema.cpp +++ b/query_modules/schema.cpp @@ -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 @@ -9,10 +9,11 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. +#include <boost/functional/hash.hpp> #include <mgp.hpp> #include "utils/string.hpp" -#include <optional> +#include <unordered_set> namespace Schema { @@ -37,6 +38,7 @@ constexpr std::string_view kParameterIndices = "indices"; constexpr std::string_view kParameterUniqueConstraints = "unique_constraints"; constexpr std::string_view kParameterExistenceConstraints = "existence_constraints"; constexpr std::string_view kParameterDropExisting = "drop_existing"; +constexpr int kInitialNumberOfPropertyOccurances = 1; std::string TypeOf(const mgp::Type &type); @@ -108,83 +110,79 @@ void Schema::ProcessPropertiesRel(mgp::Record &record, const std::string_view &t record.Insert(std::string(kReturnMandatory).c_str(), mandatory); } -struct Property { - std::string name; - mgp::Value value; +struct PropertyInfo { + std::unordered_set<std::string> property_types; // property types + int64_t number_of_property_occurrences = 0; - Property(const std::string &name, mgp::Value &&value) : name(name), value(std::move(value)) {} + PropertyInfo() = default; + explicit PropertyInfo(std::string &&property_type) + : property_types({std::move(property_type)}), + number_of_property_occurrences(Schema::kInitialNumberOfPropertyOccurances) {} +}; + +struct LabelsInfo { + std::unordered_map<std::string, PropertyInfo> properties; // key is a property name + int64_t number_of_label_occurrences = 0; }; struct LabelsHash { - std::size_t operator()(const std::set<std::string> &set) const { - std::size_t seed = set.size(); - for (const auto &i : set) { - seed ^= std::hash<std::string>{}(i) + 0x9e3779b9 + (seed << 6) + (seed >> 2); - } - return seed; - } + std::size_t operator()(const std::set<std::string> &s) const { return boost::hash_range(s.begin(), s.end()); } }; struct LabelsComparator { bool operator()(const std::set<std::string> &lhs, const std::set<std::string> &rhs) const { return lhs == rhs; } }; -struct PropertyComparator { - bool operator()(const Property &lhs, const Property &rhs) const { return lhs.name < rhs.name; } -}; - -struct PropertyInfo { - std::set<Property, PropertyComparator> properties; - bool mandatory; -}; - void Schema::NodeTypeProperties(mgp_list * /*args*/, mgp_graph *memgraph_graph, mgp_result *result, mgp_memory *memory) { mgp::MemoryDispatcherGuard guard{memory}; const auto record_factory = mgp::RecordFactory(result); try { - std::unordered_map<std::set<std::string>, PropertyInfo, LabelsHash, LabelsComparator> node_types_properties; + std::unordered_map<std::set<std::string>, LabelsInfo, LabelsHash, LabelsComparator> node_types_properties; - for (auto node : mgp::Graph(memgraph_graph).Nodes()) { + for (const auto node : mgp::Graph(memgraph_graph).Nodes()) { std::set<std::string> labels_set = {}; - for (auto label : node.Labels()) { + for (const auto label : node.Labels()) { labels_set.emplace(label); } - if (node_types_properties.find(labels_set) == node_types_properties.end()) { - node_types_properties[labels_set] = PropertyInfo{std::set<Property, PropertyComparator>(), true}; - } + node_types_properties[labels_set].number_of_label_occurrences++; if (node.Properties().empty()) { - node_types_properties[labels_set].mandatory = false; // if there is node with no property, it is not mandatory continue; } - auto &property_info = node_types_properties.at(labels_set); - for (auto &[key, prop] : node.Properties()) { - property_info.properties.emplace(key, std::move(prop)); - if (property_info.mandatory) { - property_info.mandatory = - property_info.properties.size() == 1; // if there is only one property, it is mandatory + auto &labels_info = node_types_properties.at(labels_set); + for (const auto &[key, prop] : node.Properties()) { + auto prop_type = TypeOf(prop.Type()); + if (labels_info.properties.find(key) == labels_info.properties.end()) { + labels_info.properties[key] = PropertyInfo{std::move(prop_type)}; + } else { + labels_info.properties[key].property_types.emplace(prop_type); + labels_info.properties[key].number_of_property_occurrences++; } } } - for (auto &[labels, property_info] : node_types_properties) { + for (auto &[node_type, labels_info] : node_types_properties) { // node type is a set of labels std::string label_type; - mgp::List labels_list = mgp::List(); - for (auto const &label : labels) { + auto labels_list = mgp::List(); + for (const auto &label : node_type) { label_type += ":`" + std::string(label) + "`"; labels_list.AppendExtend(mgp::Value(label)); } - for (auto const &prop : property_info.properties) { + for (const auto &prop : labels_info.properties) { + auto prop_types = mgp::List(); + for (const auto &prop_type : prop.second.property_types) { + prop_types.AppendExtend(mgp::Value(prop_type)); + } + bool mandatory = prop.second.number_of_property_occurrences == labels_info.number_of_label_occurrences; auto record = record_factory.NewRecord(); - ProcessPropertiesNode(record, label_type, labels_list, prop.name, TypeOf(prop.value.Type()), - property_info.mandatory); + ProcessPropertiesNode(record, label_type, labels_list, prop.first, prop_types, mandatory); } - if (property_info.properties.empty()) { + if (labels_info.properties.empty()) { auto record = record_factory.NewRecord(); - ProcessPropertiesNode<std::string>(record, label_type, labels_list, "", "", false); + ProcessPropertiesNode<mgp::List>(record, label_type, labels_list, "", mgp::List(), false); } } @@ -197,40 +195,45 @@ void Schema::NodeTypeProperties(mgp_list * /*args*/, mgp_graph *memgraph_graph, void Schema::RelTypeProperties(mgp_list * /*args*/, mgp_graph *memgraph_graph, mgp_result *result, mgp_memory *memory) { mgp::MemoryDispatcherGuard guard{memory}; - std::unordered_map<std::string, PropertyInfo> rel_types_properties; + std::unordered_map<std::string, LabelsInfo> rel_types_properties; const auto record_factory = mgp::RecordFactory(result); try { - const mgp::Graph graph = mgp::Graph(memgraph_graph); - for (auto rel : graph.Relationships()) { + const auto graph = mgp::Graph(memgraph_graph); + for (const auto rel : graph.Relationships()) { std::string rel_type = std::string(rel.Type()); - if (rel_types_properties.find(rel_type) == rel_types_properties.end()) { - rel_types_properties[rel_type] = PropertyInfo{std::set<Property, PropertyComparator>(), true}; - } + + rel_types_properties[rel_type].number_of_label_occurrences++; if (rel.Properties().empty()) { - rel_types_properties[rel_type].mandatory = false; // if there is rel with no property, it is not mandatory continue; } - auto &property_info = rel_types_properties.at(rel_type); + auto &labels_info = rel_types_properties.at(rel_type); for (auto &[key, prop] : rel.Properties()) { - property_info.properties.emplace(key, std::move(prop)); - if (property_info.mandatory) { - property_info.mandatory = - property_info.properties.size() == 1; // if there is only one property, it is mandatory + auto prop_type = TypeOf(prop.Type()); + if (labels_info.properties.find(key) == labels_info.properties.end()) { + labels_info.properties[key] = PropertyInfo{std::move(prop_type)}; + } else { + labels_info.properties[key].property_types.emplace(prop_type); + labels_info.properties[key].number_of_property_occurrences++; } } } - for (auto &[type, property_info] : rel_types_properties) { - std::string type_str = ":`" + std::string(type) + "`"; - for (auto const &prop : property_info.properties) { + for (auto &[rel_type, labels_info] : rel_types_properties) { + std::string type_str = ":`" + std::string(rel_type) + "`"; + for (const auto &prop : labels_info.properties) { + auto prop_types = mgp::List(); + for (const auto &prop_type : prop.second.property_types) { + prop_types.AppendExtend(mgp::Value(prop_type)); + } + bool mandatory = prop.second.number_of_property_occurrences == labels_info.number_of_label_occurrences; auto record = record_factory.NewRecord(); - ProcessPropertiesRel(record, type_str, prop.name, TypeOf(prop.value.Type()), property_info.mandatory); + ProcessPropertiesRel(record, type_str, prop.first, prop_types, mandatory); } - if (property_info.properties.empty()) { + if (labels_info.properties.empty()) { auto record = record_factory.NewRecord(); - ProcessPropertiesRel<std::string>(record, type_str, "", "", false); + ProcessPropertiesRel<mgp::List>(record, type_str, "", mgp::List(), false); } } diff --git a/tests/e2e/query_modules/schema_test.py b/tests/e2e/query_modules/schema_test.py index fbb376a22..e819a430e 100644 --- a/tests/e2e/query_modules/schema_test.py +++ b/tests/e2e/query_modules/schema_test.py @@ -431,7 +431,7 @@ def test_node_type_properties1(): f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", )[0] ) - assert (result) == [":`Activity`", ["Activity"], "location", "String", False] + assert (result) == [":`Activity`", ["Activity"], "location", ["String"], True] result = list( execute_and_fetch_all( @@ -439,7 +439,7 @@ def test_node_type_properties1(): f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", )[1] ) - assert (result) == [":`Activity`", ["Activity"], "name", "String", False] + assert (result) == [":`Activity`", ["Activity"], "name", ["String"], True] result = list( execute_and_fetch_all( @@ -447,7 +447,7 @@ def test_node_type_properties1(): f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", )[2] ) - assert (result) == [":`Dog`", ["Dog"], "name", "String", False] + assert (result) == [":`Dog`", ["Dog"], "name", ["String"], True] result = list( execute_and_fetch_all( @@ -455,7 +455,7 @@ def test_node_type_properties1(): f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", )[3] ) - assert (result) == [":`Dog`", ["Dog"], "owner", "String", False] + assert (result) == [":`Dog`", ["Dog"], "owner", ["String"], True] def test_node_type_properties2(): @@ -471,7 +471,8 @@ def test_node_type_properties2(): cursor, f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", ) - assert (list(result[0])) == [":`MyNode`", ["MyNode"], "", "", False] + + assert (list(result[0])) == [":`MyNode`", ["MyNode"], "", [], False] assert (result.__len__()) == 1 @@ -489,8 +490,8 @@ def test_node_type_properties3(): f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", ) - assert (list(result[0])) == [":`Dog`", ["Dog"], "name", "String", False] - assert (list(result[1])) == [":`Dog`", ["Dog"], "owner", "String", False] + assert (list(result[0])) == [":`Dog`", ["Dog"], "name", ["String"], False] + assert (list(result[1])) == [":`Dog`", ["Dog"], "owner", ["String"], False] assert (result.__len__()) == 2 @@ -509,9 +510,9 @@ def test_node_type_properties4(): f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", ) ) - assert (list(result[0])) == [":`Label1`:`Label2`", ["Label1", "Label2"], "property1", "String", False] - assert (list(result[1])) == [":`Label1`:`Label2`", ["Label1", "Label2"], "property2", "String", False] - assert (list(result[2])) == [":`Label1`:`Label2`", ["Label1", "Label2"], "property3", "String", False] + assert (list(result[0])) == [":`Label1`:`Label2`", ["Label1", "Label2"], "property1", ["String"], False] + assert (list(result[1])) == [":`Label1`:`Label2`", ["Label1", "Label2"], "property2", ["String"], False] + assert (list(result[2])) == [":`Label1`:`Label2`", ["Label1", "Label2"], "property3", ["String"], False] assert (result.__len__()) == 3 @@ -528,7 +529,49 @@ def test_node_type_properties5(): f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", ) - assert (list(result[0])) == [":`Dog`", ["Dog"], "name", "String", True] + assert (list(result[0])) == [":`Dog`", ["Dog"], "name", ["String"], True] + assert (result.__len__()) == 1 + + +def test_node_type_properties6(): + cursor = connect().cursor() + execute_and_fetch_all( + cursor, + """ + CREATE (d:Dog {name: 'Rex'}) + CREATE (n:Dog {name: 'Simba', owner: 'Lucy'}) + """, + ) + result = execute_and_fetch_all( + cursor, + f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", + ) + + assert (list(result[0])) == [":`Dog`", ["Dog"], "name", ["String"], True] + assert (list(result[1])) == [":`Dog`", ["Dog"], "owner", ["String"], False] + assert (result.__len__()) == 2 + + +def test_node_type_properties_multiple_property_types(): + cursor = connect().cursor() + execute_and_fetch_all( + cursor, + """ + CREATE (n:Node {prop1: 1}) + CREATE (m:Node {prop1: '1'}) + """, + ) + result = execute_and_fetch_all( + cursor, + f"CALL libschema.node_type_properties() YIELD nodeType, nodeLabels, propertyName, propertyTypes , mandatory RETURN nodeType, nodeLabels, propertyName, propertyTypes , mandatory ORDER BY propertyName, nodeLabels[0];", + ) + assert (list(result[0])) == [":`Node`", ["Node"], "prop1", ["Int", "String"], True] or (list(result[0])) == [ + ":`Node`", + ["Node"], + "prop1", + ["String", "Int"], + True, + ] assert (result.__len__()) == 1 @@ -544,7 +587,7 @@ def test_rel_type_properties1(): f"CALL libschema.rel_type_properties() YIELD relType,propertyName, propertyTypes , mandatory RETURN relType, propertyName, propertyTypes , mandatory;", )[0] ) - assert (result) == [":`LOVES`", "", "", False] + assert (result) == [":`LOVES`", "", [], False] def test_rel_type_properties2(): @@ -560,7 +603,7 @@ def test_rel_type_properties2(): cursor, f"CALL libschema.rel_type_properties() YIELD relType,propertyName, propertyTypes , mandatory RETURN relType, propertyName, propertyTypes , mandatory;", ) - assert (list(result[0])) == [":`LOVES`", "duration", "Int", False] + assert (list(result[0])) == [":`LOVES`", "duration", ["Int"], False] assert (result.__len__()) == 1 @@ -576,7 +619,47 @@ def test_rel_type_properties3(): cursor, f"CALL libschema.rel_type_properties() YIELD relType,propertyName, propertyTypes , mandatory RETURN relType, propertyName, propertyTypes , mandatory;", ) - assert (list(result[0])) == [":`LOVES`", "duration", "Int", True] + assert (list(result[0])) == [":`LOVES`", "duration", ["Int"], True] + assert (result.__len__()) == 1 + + +def test_rel_type_properties4(): + cursor = connect().cursor() + execute_and_fetch_all( + cursor, + """ + CREATE (n:Dog {name: 'Simba', owner: 'Lucy'})-[j:LOVES {duration: 30}]->(a:Activity {name: 'Running', location: 'Zadar'}) + CREATE (m:Dog {name: 'Rex', owner: 'Lucy'})-[r:LOVES {duration: 30, weather: 'sunny'}]->(b:Activity {name: 'Running', location: 'Zadar'}) + """, + ) + result = execute_and_fetch_all( + cursor, + f"CALL libschema.rel_type_properties() YIELD relType,propertyName, propertyTypes , mandatory RETURN relType, propertyName, propertyTypes , mandatory;", + ) + assert (list(result[0])) == [":`LOVES`", "weather", ["String"], False] + assert (list(result[1])) == [":`LOVES`", "duration", ["Int"], True] + assert (result.__len__()) == 2 + + +def test_rel_type_properties_multiple_property_types(): + cursor = connect().cursor() + execute_and_fetch_all( + cursor, + """ + CREATE (n:Dog {name: 'Simba', owner: 'Lucy'})-[j:LOVES {duration: 30}]->(a:Activity {name: 'Running', location: 'Zadar'}) + CREATE (m:Dog {name: 'Rex', owner: 'Lucy'})-[r:LOVES {duration: "30"}]->(b:Activity {name: 'Running', location: 'Zadar'}) + """, + ) + result = execute_and_fetch_all( + cursor, + f"CALL libschema.rel_type_properties() YIELD relType,propertyName, propertyTypes , mandatory RETURN relType, propertyName, propertyTypes , mandatory;", + ) + assert (list(result[0])) == [":`LOVES`", "duration", ["Int", "String"], True] or (list(result[0])) == [ + ":`LOVES`", + "duration", + ["String", "Int"], + True, + ] assert (result.__len__()) == 1 From b561c61b6455ef6a6dee35aa47057ea7871174f2 Mon Sep 17 00:00:00 2001 From: Antonio Filipovic <61245998+antoniofilipovic@users.noreply.github.com> Date: Wed, 28 Feb 2024 10:57:00 +0100 Subject: [PATCH 04/18] HA: Add initial logic for choosing new replica (#1729) --- src/coordination/coordinator_client.cpp | 26 +- src/coordination/coordinator_handlers.cpp | 11 + src/coordination/coordinator_instance.cpp | 302 ++++++++++++------ src/coordination/coordinator_rpc.cpp | 38 ++- .../coordination/coordinator_client.hpp | 12 +- .../coordination/coordinator_handlers.hpp | 3 + .../coordination/coordinator_instance.hpp | 24 +- .../include/coordination/coordinator_rpc.hpp | 33 ++ .../include/coordination/coordinator_slk.hpp | 14 + .../coordination/replication_instance.hpp | 24 +- .../include/coordination/rpc_errors.hpp | 1 + src/coordination/replication_instance.cpp | 27 +- src/dbms/dbms_handler.hpp | 4 - .../CMakeLists.txt | 1 + src/replication_coordination_glue/common.hpp | 32 ++ .../replication_handler.hpp | 3 + .../replication_handler.cpp | 21 +- .../v2/inmemory/replication/recovery.cpp | 6 +- .../v2/replication/replication_client.cpp | 2 + src/utils/functional.hpp | 7 +- src/utils/scheduler.hpp | 2 +- src/utils/typeinfo.hpp | 2 + tests/unit/CMakeLists.txt | 8 + tests/unit/coordination_utils.cpp | 246 ++++++++++++++ 24 files changed, 718 insertions(+), 131 deletions(-) create mode 100644 src/replication_coordination_glue/common.hpp create mode 100644 tests/unit/coordination_utils.cpp diff --git a/src/coordination/coordinator_client.cpp b/src/coordination/coordinator_client.cpp index 84044b04a..f4d2da838 100644 --- a/src/coordination/coordinator_client.cpp +++ b/src/coordination/coordinator_client.cpp @@ -16,6 +16,7 @@ #include "coordination/coordinator_config.hpp" #include "coordination/coordinator_rpc.hpp" +#include "replication_coordination_glue/common.hpp" #include "replication_coordination_glue/messages.hpp" #include "utils/result.hpp" @@ -30,7 +31,7 @@ auto CreateClientContext(memgraph::coordination::CoordinatorClientConfig const & } // namespace CoordinatorClient::CoordinatorClient(CoordinatorInstance *coord_instance, CoordinatorClientConfig config, - HealthCheckCallback succ_cb, HealthCheckCallback fail_cb) + HealthCheckClientCallback succ_cb, HealthCheckClientCallback fail_cb) : rpc_context_{CreateClientContext(config)}, rpc_client_{io::network::Endpoint(io::network::Endpoint::needs_resolving, config.ip_address, config.port), &rpc_context_}, @@ -68,6 +69,10 @@ void CoordinatorClient::StartFrequentCheck() { auto stream{rpc_client_.Stream<memgraph::replication_coordination_glue::FrequentHeartbeatRpc>()}; stream.AwaitResponse(); } + // Subtle race condition: + // acquiring of lock needs to happen before function call, as function callback can be changed + // for instance after lock is already acquired + // (failover case when instance is promoted to MAIN) succ_cb_(coord_instance_, instance_name); } catch (rpc::RpcFailedException const &) { fail_cb_(coord_instance_, instance_name); @@ -79,11 +84,6 @@ void CoordinatorClient::StopFrequentCheck() { instance_checker_.Stop(); } void CoordinatorClient::PauseFrequentCheck() { instance_checker_.Pause(); } void CoordinatorClient::ResumeFrequentCheck() { instance_checker_.Resume(); } -auto CoordinatorClient::SetCallbacks(HealthCheckCallback succ_cb, HealthCheckCallback fail_cb) -> void { - succ_cb_ = std::move(succ_cb); - fail_cb_ = std::move(fail_cb); -} - auto CoordinatorClient::ReplicationClientInfo() const -> ReplClientInfo { return config_.replication_client_info; } auto CoordinatorClient::SendPromoteReplicaToMainRpc(const utils::UUID &uuid, @@ -171,5 +171,19 @@ auto CoordinatorClient::SendEnableWritingOnMainRpc() const -> bool { return false; } +auto CoordinatorClient::SendGetInstanceTimestampsRpc() const + -> utils::BasicResult<GetInstanceUUIDError, replication_coordination_glue::DatabaseHistories> { + try { + auto stream{rpc_client_.Stream<coordination::GetDatabaseHistoriesRpc>()}; + auto res = stream.AwaitResponse(); + + return res.database_histories; + + } catch (const rpc::RpcFailedException &) { + spdlog::error("RPC error occured while sending GetInstance UUID RPC"); + return GetInstanceUUIDError::RPC_EXCEPTION; + } +} + } // namespace memgraph::coordination #endif diff --git a/src/coordination/coordinator_handlers.cpp b/src/coordination/coordinator_handlers.cpp index f605069fe..637360267 100644 --- a/src/coordination/coordinator_handlers.cpp +++ b/src/coordination/coordinator_handlers.cpp @@ -57,6 +57,17 @@ void CoordinatorHandlers::Register(memgraph::coordination::CoordinatorServer &se spdlog::info("Received GetInstanceUUIDRpc on coordinator server"); CoordinatorHandlers::GetInstanceUUIDHandler(replication_handler, req_reader, res_builder); }); + + server.Register<coordination::GetDatabaseHistoriesRpc>( + [&replication_handler](slk::Reader *req_reader, slk::Builder *res_builder) -> void { + spdlog::info("Received GetDatabasesHistoryRpc on coordinator server"); + CoordinatorHandlers::GetDatabaseHistoriesHandler(replication_handler, req_reader, res_builder); + }); +} + +void CoordinatorHandlers::GetDatabaseHistoriesHandler(replication::ReplicationHandler &replication_handler, + slk::Reader * /*req_reader*/, slk::Builder *res_builder) { + slk::Save(coordination::GetDatabaseHistoriesRes{replication_handler.GetDatabasesHistories()}, res_builder); } void CoordinatorHandlers::SwapMainUUIDHandler(replication::ReplicationHandler &replication_handler, diff --git a/src/coordination/coordinator_instance.cpp b/src/coordination/coordinator_instance.cpp index 90674cf3c..ba94d9d5f 100644 --- a/src/coordination/coordinator_instance.cpp +++ b/src/coordination/coordinator_instance.cpp @@ -15,10 +15,12 @@ #include "coordination/coordinator_exceptions.hpp" #include "coordination/fmt.hpp" +#include "dbms/constants.hpp" #include "nuraft/coordinator_state_machine.hpp" #include "nuraft/coordinator_state_manager.hpp" #include "utils/counter.hpp" #include "utils/functional.hpp" +#include "utils/resource_lock.hpp" #include <range/v3/view.hpp> #include <shared_mutex> @@ -32,96 +34,28 @@ CoordinatorInstance::CoordinatorInstance() : raft_state_(RaftState::MakeRaftState( [this] { std::ranges::for_each(repl_instances_, &ReplicationInstance::StartFrequentCheck); }, [this] { std::ranges::for_each(repl_instances_, &ReplicationInstance::StopFrequentCheck); })) { - auto find_repl_instance = [](CoordinatorInstance *self, - std::string_view repl_instance_name) -> ReplicationInstance & { - auto repl_instance = - std::ranges::find_if(self->repl_instances_, [repl_instance_name](ReplicationInstance const &instance) { - return instance.InstanceName() == repl_instance_name; - }); - - MG_ASSERT(repl_instance != self->repl_instances_.end(), "Instance {} not found during callback!", - repl_instance_name); - return *repl_instance; + client_succ_cb_ = [](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { + auto lock = std::unique_lock{self->coord_instance_lock_}; + auto &repl_instance = self->FindReplicationInstance(repl_instance_name); + std::invoke(repl_instance.GetSuccessCallback(), self, repl_instance_name); }; - replica_succ_cb_ = [find_repl_instance](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { - auto lock = std::lock_guard{self->coord_instance_lock_}; - spdlog::trace("Instance {} performing replica successful callback", repl_instance_name); - auto &repl_instance = find_repl_instance(self, repl_instance_name); - - // We need to get replicas UUID from time to time to ensure replica is listening to correct main - // and that it didn't go down for less time than we could notice - // We need to get id of main replica is listening to - // and swap if necessary - if (!repl_instance.EnsureReplicaHasCorrectMainUUID(self->GetMainUUID())) { - spdlog::error("Failed to swap uuid for replica instance {} which is alive", repl_instance.InstanceName()); - return; - } - - repl_instance.OnSuccessPing(); + client_fail_cb_ = [](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { + auto lock = std::unique_lock{self->coord_instance_lock_}; + auto &repl_instance = self->FindReplicationInstance(repl_instance_name); + std::invoke(repl_instance.GetFailCallback(), self, repl_instance_name); }; +} - replica_fail_cb_ = [find_repl_instance](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { - auto lock = std::lock_guard{self->coord_instance_lock_}; - spdlog::trace("Instance {} performing replica failure callback", repl_instance_name); - auto &repl_instance = find_repl_instance(self, repl_instance_name); - repl_instance.OnFailPing(); - }; +auto CoordinatorInstance::FindReplicationInstance(std::string_view replication_instance_name) -> ReplicationInstance & { + auto repl_instance = + std::ranges::find_if(repl_instances_, [replication_instance_name](ReplicationInstance const &instance) { + return instance.InstanceName() == replication_instance_name; + }); - main_succ_cb_ = [find_repl_instance](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { - auto lock = std::lock_guard{self->coord_instance_lock_}; - spdlog::trace("Instance {} performing main successful callback", repl_instance_name); - - auto &repl_instance = find_repl_instance(self, repl_instance_name); - - if (repl_instance.IsAlive()) { - repl_instance.OnSuccessPing(); - return; - } - - const auto &repl_instance_uuid = repl_instance.GetMainUUID(); - MG_ASSERT(repl_instance_uuid.has_value(), "Instance must have uuid set."); - - auto const curr_main_uuid = self->GetMainUUID(); - if (curr_main_uuid == repl_instance_uuid.value()) { - if (!repl_instance.EnableWritingOnMain()) { - spdlog::error("Failed to enable writing on main instance {}", repl_instance_name); - return; - } - - repl_instance.OnSuccessPing(); - return; - } - - // TODO(antoniof) make demoteToReplica idempotent since main can be demoted to replica but - // swapUUID can fail - if (repl_instance.DemoteToReplica(self->replica_succ_cb_, self->replica_fail_cb_)) { - repl_instance.OnSuccessPing(); - spdlog::info("Instance {} demoted to replica", repl_instance_name); - } else { - spdlog::error("Instance {} failed to become replica", repl_instance_name); - return; - } - - if (!repl_instance.SendSwapAndUpdateUUID(curr_main_uuid)) { - spdlog::error(fmt::format("Failed to swap uuid for demoted main instance {}", repl_instance.InstanceName())); - return; - } - }; - - main_fail_cb_ = [find_repl_instance](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { - auto lock = std::lock_guard{self->coord_instance_lock_}; - spdlog::trace("Instance {} performing main failure callback", repl_instance_name); - auto &repl_instance = find_repl_instance(self, repl_instance_name); - repl_instance.OnFailPing(); - const auto &repl_instance_uuid = repl_instance.GetMainUUID(); - MG_ASSERT(repl_instance_uuid.has_value(), "Instance must have uuid set"); - - if (!repl_instance.IsAlive() && self->GetMainUUID() == repl_instance_uuid.value()) { - spdlog::info("Cluster without main instance, trying automatic failover"); - self->TryFailover(); // TODO: (andi) Initiate failover - } - }; + MG_ASSERT(repl_instance != repl_instances_.end(), "Instance {} not found during callback!", + replication_instance_name); + return *repl_instance; } auto CoordinatorInstance::ShowInstances() const -> std::vector<InstanceStatus> { @@ -166,8 +100,36 @@ auto CoordinatorInstance::TryFailover() -> void { return; } - // TODO: Smarter choice - auto new_main = ranges::begin(alive_replicas); + // for each DB in instance we get one DatabaseHistory + using DatabaseHistories = replication_coordination_glue::DatabaseHistories; + std::vector<std::pair<std::string, DatabaseHistories>> instance_database_histories; + + bool success{true}; + std::ranges::for_each(alive_replicas, [&success, &instance_database_histories](ReplicationInstance &replica) { + if (!success) { + return; + } + auto res = replica.GetClient().SendGetInstanceTimestampsRpc(); + if (res.HasError()) { + spdlog::error("Could get per db history data for instance {}", replica.InstanceName()); + success = false; + return; + } + instance_database_histories.emplace_back(replica.InstanceName(), std::move(res.GetValue())); + }); + + if (!success) { + spdlog::error("Aborting failover as at least one instance didn't provide per database history."); + return; + } + + auto [most_up_to_date_instance, latest_epoch, latest_commit_timestamp] = + ChooseMostUpToDateInstance(instance_database_histories); + + spdlog::trace("The most up to date instance is {} with epoch {} and {} latest commit timestamp", + most_up_to_date_instance, *latest_epoch, *latest_commit_timestamp); + + auto *new_main = &FindReplicationInstance(most_up_to_date_instance); new_main->PauseFrequentCheck(); utils::OnScopeExit scope_exit{[&new_main] { new_main->ResumeFrequentCheck(); }}; @@ -191,7 +153,8 @@ auto CoordinatorInstance::TryFailover() -> void { ranges::views::transform(&ReplicationInstance::ReplicationClientInfo) | ranges::to<ReplicationClientsInfo>(); - if (!new_main->PromoteToMain(new_main_uuid, std::move(repl_clients_info), main_succ_cb_, main_fail_cb_)) { + if (!new_main->PromoteToMain(new_main_uuid, std::move(repl_clients_info), &CoordinatorInstance::MainSuccessCallback, + &CoordinatorInstance::MainFailCallback)) { spdlog::warn("Failover failed since promoting replica to main failed!"); return; } @@ -242,7 +205,8 @@ auto CoordinatorInstance::SetReplicationInstanceToMain(std::string instance_name std::ranges::transform(repl_instances_ | ranges::views::filter(is_not_new_main), std::back_inserter(repl_clients_info), &ReplicationInstance::ReplicationClientInfo); - if (!new_main->PromoteToMain(new_main_uuid, std::move(repl_clients_info), main_succ_cb_, main_fail_cb_)) { + if (!new_main->PromoteToMain(new_main_uuid, std::move(repl_clients_info), &CoordinatorInstance::MainSuccessCallback, + &CoordinatorInstance::MainFailCallback)) { return SetInstanceToMainCoordinatorStatus::COULD_NOT_PROMOTE_TO_MAIN; } @@ -290,7 +254,9 @@ auto CoordinatorInstance::RegisterReplicationInstance(CoordinatorClientConfig co spdlog::info("Request for registering instance {} accepted", instance_name); try { - repl_instances_.emplace_back(this, std::move(config), replica_succ_cb_, replica_fail_cb_); + repl_instances_.emplace_back(this, std::move(config), client_succ_cb_, client_fail_cb_, + &CoordinatorInstance::ReplicaSuccessCallback, + &CoordinatorInstance::ReplicaFailCallback); } catch (CoordinatorRegisterInstanceException const &) { return RegisterInstanceCoordinatorStatus::RPC_FAILED; } @@ -304,6 +270,85 @@ auto CoordinatorInstance::RegisterReplicationInstance(CoordinatorClientConfig co return RegisterInstanceCoordinatorStatus::SUCCESS; } +void CoordinatorInstance::MainFailCallback(std::string_view repl_instance_name) { + auto &repl_instance = FindReplicationInstance(repl_instance_name); + repl_instance.OnFailPing(); + const auto &repl_instance_uuid = repl_instance.GetMainUUID(); + MG_ASSERT(repl_instance_uuid.has_value(), "Instance must have uuid set"); + + if (!repl_instance.IsAlive() && GetMainUUID() == repl_instance_uuid.value()) { + spdlog::info("Cluster without main instance, trying automatic failover"); + TryFailover(); + } +} + +void CoordinatorInstance::MainSuccessCallback(std::string_view repl_instance_name) { + auto &repl_instance = FindReplicationInstance(repl_instance_name); + spdlog::trace("Instance {} performing main successful callback", repl_instance_name); + + if (repl_instance.IsAlive()) { + repl_instance.OnSuccessPing(); + return; + } + + const auto &repl_instance_uuid = repl_instance.GetMainUUID(); + MG_ASSERT(repl_instance_uuid.has_value(), "Instance must have uuid set."); + + auto const curr_main_uuid = GetMainUUID(); + if (curr_main_uuid == repl_instance_uuid.value()) { + if (!repl_instance.EnableWritingOnMain()) { + spdlog::error("Failed to enable writing on main instance {}", repl_instance_name); + return; + } + + repl_instance.OnSuccessPing(); + return; + } + + if (repl_instance.DemoteToReplica(&CoordinatorInstance::ReplicaSuccessCallback, + &CoordinatorInstance::ReplicaFailCallback)) { + repl_instance.OnSuccessPing(); + spdlog::info("Instance {} demoted to replica", repl_instance_name); + } else { + spdlog::error("Instance {} failed to become replica", repl_instance_name); + return; + } + + if (!repl_instance.SendSwapAndUpdateUUID(curr_main_uuid)) { + spdlog::error(fmt::format("Failed to swap uuid for demoted main instance {}", repl_instance.InstanceName())); + return; + } +} + +void CoordinatorInstance::ReplicaSuccessCallback(std::string_view repl_instance_name) { + auto &repl_instance = FindReplicationInstance(repl_instance_name); + if (!repl_instance.IsReplica()) { + spdlog::error("Aborting replica callback since instance {} is not replica anymore", repl_instance_name); + return; + } + spdlog::trace("Instance {} performing replica successful callback", repl_instance_name); + // We need to get replicas UUID from time to time to ensure replica is listening to correct main + // and that it didn't go down for less time than we could notice + // We need to get id of main replica is listening to + // and swap if necessary + if (!repl_instance.EnsureReplicaHasCorrectMainUUID(GetMainUUID())) { + spdlog::error("Failed to swap uuid for replica instance {} which is alive", repl_instance.InstanceName()); + return; + } + + repl_instance.OnSuccessPing(); +} + +void CoordinatorInstance::ReplicaFailCallback(std::string_view repl_instance_name) { + auto &repl_instance = FindReplicationInstance(repl_instance_name); + if (!repl_instance.IsReplica()) { + spdlog::error("Aborting replica fail callback since instance {} is not replica anymore", repl_instance_name); + return; + } + spdlog::trace("Instance {} performing replica failure callback", repl_instance_name); + repl_instance.OnFailPing(); +} + auto CoordinatorInstance::UnregisterReplicationInstance(std::string instance_name) -> UnregisterInstanceCoordinatorStatus { auto lock = std::lock_guard{coord_instance_lock_}; @@ -343,5 +388,82 @@ auto CoordinatorInstance::GetMainUUID() const -> utils::UUID { return main_uuid_ // TODO: (andi) Add to the RAFT log. auto CoordinatorInstance::SetMainUUID(utils::UUID new_uuid) -> void { main_uuid_ = new_uuid; } +auto CoordinatorInstance::ChooseMostUpToDateInstance( + const std::vector<std::pair<std::string, replication_coordination_glue::DatabaseHistories>> + &instance_database_histories) -> NewMainRes { + NewMainRes new_main_res; + std::for_each( + instance_database_histories.begin(), instance_database_histories.end(), + [&new_main_res](const InstanceNameDbHistories &instance_res_pair) { + const auto &[instance_name, instance_db_histories] = instance_res_pair; + + // Find default db for instance and its history + auto default_db_history_data = std::ranges::find_if( + instance_db_histories, [default_db = memgraph::dbms::kDefaultDB]( + const replication_coordination_glue::DatabaseHistory &db_timestamps) { + return db_timestamps.name == default_db; + }); + + std::ranges::for_each( + instance_db_histories, + [&instance_name = instance_name](const replication_coordination_glue::DatabaseHistory &db_history) { + spdlog::trace("Instance {}: name {}, default db {}", instance_name, db_history.name, + memgraph::dbms::kDefaultDB); + }); + + MG_ASSERT(default_db_history_data != instance_db_histories.end(), "No history for instance"); + + const auto &instance_default_db_history = default_db_history_data->history; + + std::ranges::for_each(instance_default_db_history | ranges::views::reverse, + [&instance_name = instance_name](const auto &epoch_history_it) { + spdlog::trace("Instance {}: epoch {}, last_commit_timestamp: {}", instance_name, + std::get<0>(epoch_history_it), std::get<1>(epoch_history_it)); + }); + + // get latest epoch + // get latest timestamp + + if (!new_main_res.latest_epoch) { + const auto &[epoch, timestamp] = *instance_default_db_history.crbegin(); + new_main_res = NewMainRes{ + .most_up_to_date_instance = instance_name, + .latest_epoch = epoch, + .latest_commit_timestamp = timestamp, + }; + spdlog::trace("Currently the most up to date instance is {} with epoch {} and {} latest commit timestamp", + instance_name, epoch, timestamp); + return; + } + + bool found_same_point{false}; + std::string last_most_up_to_date_epoch{*new_main_res.latest_epoch}; + for (auto [epoch, timestamp] : ranges::reverse_view(instance_default_db_history)) { + if (*new_main_res.latest_commit_timestamp < timestamp) { + new_main_res = NewMainRes{ + .most_up_to_date_instance = instance_name, + .latest_epoch = epoch, + .latest_commit_timestamp = timestamp, + }; + + spdlog::trace("Found the new most up to date instance {} with epoch {} and {} latest commit timestamp", + instance_name, epoch, timestamp); + } + + // we found point at which they were same + if (epoch == last_most_up_to_date_epoch) { + found_same_point = true; + break; + } + } + + if (!found_same_point) { + spdlog::error("Didn't find same history epoch {} for instance {} and instance {}", last_most_up_to_date_epoch, + new_main_res.most_up_to_date_instance, instance_name); + } + }); + + return new_main_res; +} } // namespace memgraph::coordination #endif diff --git a/src/coordination/coordinator_rpc.cpp b/src/coordination/coordinator_rpc.cpp index 4115f1979..815693824 100644 --- a/src/coordination/coordinator_rpc.cpp +++ b/src/coordination/coordinator_rpc.cpp @@ -76,9 +76,9 @@ void EnableWritingOnMainRes::Load(EnableWritingOnMainRes *self, memgraph::slk::R memgraph::slk::Load(self, reader); } -void EnableWritingOnMainReq::Save(EnableWritingOnMainReq const &self, memgraph::slk::Builder *builder) {} +void EnableWritingOnMainReq::Save(EnableWritingOnMainReq const & /*self*/, memgraph::slk::Builder * /*builder*/) {} -void EnableWritingOnMainReq::Load(EnableWritingOnMainReq *self, memgraph::slk::Reader *reader) {} +void EnableWritingOnMainReq::Load(EnableWritingOnMainReq * /*self*/, memgraph::slk::Reader * /*reader*/) {} // GetInstanceUUID void GetInstanceUUIDReq::Save(const GetInstanceUUIDReq &self, memgraph::slk::Builder *builder) { @@ -97,6 +97,24 @@ void GetInstanceUUIDRes::Load(GetInstanceUUIDRes *self, memgraph::slk::Reader *r memgraph::slk::Load(self, reader); } +// GetDatabaseHistoriesRpc + +void GetDatabaseHistoriesReq::Save(const GetDatabaseHistoriesReq & /*self*/, memgraph::slk::Builder * /*builder*/) { + /* nothing to serialize */ +} + +void GetDatabaseHistoriesReq::Load(GetDatabaseHistoriesReq * /*self*/, memgraph::slk::Reader * /*reader*/) { + /* nothing to serialize */ +} + +void GetDatabaseHistoriesRes::Save(const GetDatabaseHistoriesRes &self, memgraph::slk::Builder *builder) { + memgraph::slk::Save(self, builder); +} + +void GetDatabaseHistoriesRes::Load(GetDatabaseHistoriesRes *self, memgraph::slk::Reader *reader) { + memgraph::slk::Load(self, reader); +} + } // namespace coordination constexpr utils::TypeInfo coordination::PromoteReplicaToMainReq::kType{utils::TypeId::COORD_FAILOVER_REQ, @@ -130,6 +148,12 @@ constexpr utils::TypeInfo coordination::GetInstanceUUIDReq::kType{utils::TypeId: constexpr utils::TypeInfo coordination::GetInstanceUUIDRes::kType{utils::TypeId::COORD_GET_UUID_RES, "CoordGetUUIDRes", nullptr}; +constexpr utils::TypeInfo coordination::GetDatabaseHistoriesReq::kType{utils::TypeId::COORD_GET_INSTANCE_DATABASES_REQ, + "GetInstanceDatabasesReq", nullptr}; + +constexpr utils::TypeInfo coordination::GetDatabaseHistoriesRes::kType{utils::TypeId::COORD_GET_INSTANCE_DATABASES_RES, + "GetInstanceDatabasesRes", nullptr}; + namespace slk { // PromoteReplicaToMainRpc @@ -213,6 +237,16 @@ void Load(memgraph::coordination::GetInstanceUUIDRes *self, memgraph::slk::Reade memgraph::slk::Load(&self->uuid, reader); } +// GetInstanceTimestampsReq + +void Save(const memgraph::coordination::GetDatabaseHistoriesRes &self, memgraph::slk::Builder *builder) { + memgraph::slk::Save(self.database_histories, builder); +} + +void Load(memgraph::coordination::GetDatabaseHistoriesRes *self, memgraph::slk::Reader *reader) { + memgraph::slk::Load(&self->database_histories, reader); +} + } // namespace slk } // namespace memgraph diff --git a/src/coordination/include/coordination/coordinator_client.hpp b/src/coordination/include/coordination/coordinator_client.hpp index 5e10af89d..994c78d18 100644 --- a/src/coordination/include/coordination/coordinator_client.hpp +++ b/src/coordination/include/coordination/coordinator_client.hpp @@ -14,6 +14,7 @@ #ifdef MG_ENTERPRISE #include "coordination/coordinator_config.hpp" +#include "replication_coordination_glue/common.hpp" #include "rpc/client.hpp" #include "rpc_errors.hpp" #include "utils/result.hpp" @@ -23,13 +24,13 @@ namespace memgraph::coordination { class CoordinatorInstance; -using HealthCheckCallback = std::function<void(CoordinatorInstance *, std::string_view)>; +using HealthCheckClientCallback = std::function<void(CoordinatorInstance *, std::string_view)>; using ReplicationClientsInfo = std::vector<ReplClientInfo>; class CoordinatorClient { public: explicit CoordinatorClient(CoordinatorInstance *coord_instance, CoordinatorClientConfig config, - HealthCheckCallback succ_cb, HealthCheckCallback fail_cb); + HealthCheckClientCallback succ_cb, HealthCheckClientCallback fail_cb); ~CoordinatorClient() = default; @@ -62,7 +63,8 @@ class CoordinatorClient { auto ReplicationClientInfo() const -> ReplClientInfo; - auto SetCallbacks(HealthCheckCallback succ_cb, HealthCheckCallback fail_cb) -> void; + auto SendGetInstanceTimestampsRpc() const + -> utils::BasicResult<GetInstanceUUIDError, replication_coordination_glue::DatabaseHistories>; auto RpcClient() -> rpc::Client & { return rpc_client_; } @@ -82,8 +84,8 @@ class CoordinatorClient { CoordinatorClientConfig config_; CoordinatorInstance *coord_instance_; - HealthCheckCallback succ_cb_; - HealthCheckCallback fail_cb_; + HealthCheckClientCallback succ_cb_; + HealthCheckClientCallback fail_cb_; }; } // namespace memgraph::coordination diff --git a/src/coordination/include/coordination/coordinator_handlers.hpp b/src/coordination/include/coordination/coordinator_handlers.hpp index b9ed4b519..18aecc9cf 100644 --- a/src/coordination/include/coordination/coordinator_handlers.hpp +++ b/src/coordination/include/coordination/coordinator_handlers.hpp @@ -41,6 +41,9 @@ class CoordinatorHandlers { static void GetInstanceUUIDHandler(replication::ReplicationHandler &replication_handler, slk::Reader *req_reader, slk::Builder *res_builder); + + static void GetDatabaseHistoriesHandler(replication::ReplicationHandler &replication_handler, slk::Reader *req_reader, + slk::Builder *res_builder); }; } // namespace memgraph::dbms diff --git a/src/coordination/include/coordination/coordinator_instance.hpp b/src/coordination/include/coordination/coordinator_instance.hpp index 15b377ed9..bed202744 100644 --- a/src/coordination/include/coordination/coordinator_instance.hpp +++ b/src/coordination/include/coordination/coordinator_instance.hpp @@ -18,6 +18,7 @@ #include "coordination/raft_state.hpp" #include "coordination/register_main_replica_coordinator_status.hpp" #include "coordination/replication_instance.hpp" +#include "utils/resource_lock.hpp" #include "utils/rw_lock.hpp" #include "utils/thread_pool.hpp" @@ -25,6 +26,13 @@ namespace memgraph::coordination { +struct NewMainRes { + std::string most_up_to_date_instance; + std::optional<std::string> latest_epoch; + std::optional<uint64_t> latest_commit_timestamp; +}; +using InstanceNameDbHistories = std::pair<std::string, replication_coordination_glue::DatabaseHistories>; + class CoordinatorInstance { public: CoordinatorInstance(); @@ -44,12 +52,24 @@ class CoordinatorInstance { auto SetMainUUID(utils::UUID new_uuid) -> void; + auto FindReplicationInstance(std::string_view replication_instance_name) -> ReplicationInstance &; + + void MainFailCallback(std::string_view); + + void MainSuccessCallback(std::string_view); + + void ReplicaSuccessCallback(std::string_view); + + void ReplicaFailCallback(std::string_view); + + static auto ChooseMostUpToDateInstance(const std::vector<InstanceNameDbHistories> &) -> NewMainRes; + private: - HealthCheckCallback main_succ_cb_, main_fail_cb_, replica_succ_cb_, replica_fail_cb_; + HealthCheckClientCallback client_succ_cb_, client_fail_cb_; // NOTE: Must be std::list because we rely on pointer stability std::list<ReplicationInstance> repl_instances_; - mutable utils::RWLock coord_instance_lock_{utils::RWLock::Priority::READ}; + mutable utils::ResourceLock coord_instance_lock_{}; utils::UUID main_uuid_; diff --git a/src/coordination/include/coordination/coordinator_rpc.hpp b/src/coordination/include/coordination/coordinator_rpc.hpp index 1578b4577..2bf88fe46 100644 --- a/src/coordination/include/coordination/coordinator_rpc.hpp +++ b/src/coordination/include/coordination/coordinator_rpc.hpp @@ -15,6 +15,7 @@ #ifdef MG_ENTERPRISE #include "coordination/coordinator_config.hpp" +#include "replication_coordination_glue/common.hpp" #include "rpc/messages.hpp" #include "slk/serialization.hpp" @@ -161,6 +162,32 @@ struct GetInstanceUUIDRes { using GetInstanceUUIDRpc = rpc::RequestResponse<GetInstanceUUIDReq, GetInstanceUUIDRes>; +struct GetDatabaseHistoriesReq { + static const utils::TypeInfo kType; + static const utils::TypeInfo &GetTypeInfo() { return kType; } + + static void Load(GetDatabaseHistoriesReq *self, memgraph::slk::Reader *reader); + static void Save(const GetDatabaseHistoriesReq &self, memgraph::slk::Builder *builder); + + GetDatabaseHistoriesReq() = default; +}; + +struct GetDatabaseHistoriesRes { + static const utils::TypeInfo kType; + static const utils::TypeInfo &GetTypeInfo() { return kType; } + + static void Load(GetDatabaseHistoriesRes *self, memgraph::slk::Reader *reader); + static void Save(const GetDatabaseHistoriesRes &self, memgraph::slk::Builder *builder); + + explicit GetDatabaseHistoriesRes(const replication_coordination_glue::DatabaseHistories &database_histories) + : database_histories(database_histories) {} + GetDatabaseHistoriesRes() = default; + + replication_coordination_glue::DatabaseHistories database_histories; +}; + +using GetDatabaseHistoriesRpc = rpc::RequestResponse<GetDatabaseHistoriesReq, GetDatabaseHistoriesRes>; + } // namespace memgraph::coordination // SLK serialization declarations @@ -183,15 +210,21 @@ void Save(const memgraph::coordination::GetInstanceUUIDReq &self, memgraph::slk: void Load(memgraph::coordination::GetInstanceUUIDReq *self, memgraph::slk::Reader *reader); void Save(const memgraph::coordination::GetInstanceUUIDRes &self, memgraph::slk::Builder *builder); void Load(memgraph::coordination::GetInstanceUUIDRes *self, memgraph::slk::Reader *reader); + // UnregisterReplicaRpc void Save(memgraph::coordination::UnregisterReplicaRes const &self, memgraph::slk::Builder *builder); void Load(memgraph::coordination::UnregisterReplicaRes *self, memgraph::slk::Reader *reader); void Save(memgraph::coordination::UnregisterReplicaReq const &self, memgraph::slk::Builder *builder); void Load(memgraph::coordination::UnregisterReplicaReq *self, memgraph::slk::Reader *reader); +// EnableWritingOnMainRpc void Save(memgraph::coordination::EnableWritingOnMainRes const &self, memgraph::slk::Builder *builder); void Load(memgraph::coordination::EnableWritingOnMainRes *self, memgraph::slk::Reader *reader); +// GetDatabaseHistoriesRpc +void Save(const memgraph::coordination::GetDatabaseHistoriesRes &self, memgraph::slk::Builder *builder); +void Load(memgraph::coordination::GetDatabaseHistoriesRes *self, memgraph::slk::Reader *reader); + } // namespace memgraph::slk #endif diff --git a/src/coordination/include/coordination/coordinator_slk.hpp b/src/coordination/include/coordination/coordinator_slk.hpp index 49834be41..ee393b7b6 100644 --- a/src/coordination/include/coordination/coordinator_slk.hpp +++ b/src/coordination/include/coordination/coordinator_slk.hpp @@ -14,6 +14,7 @@ #ifdef MG_ENTERPRISE #include "coordination/coordinator_config.hpp" +#include "replication_coordination_glue/common.hpp" #include "slk/serialization.hpp" #include "slk/streams.hpp" @@ -34,5 +35,18 @@ inline void Load(ReplicationClientInfo *obj, Reader *reader) { Load(&obj->replication_ip_address, reader); Load(&obj->replication_port, reader); } + +inline void Save(const replication_coordination_glue::DatabaseHistory &obj, Builder *builder) { + Save(obj.db_uuid, builder); + Save(obj.history, builder); + Save(obj.name, builder); +} + +inline void Load(replication_coordination_glue::DatabaseHistory *obj, Reader *reader) { + Load(&obj->db_uuid, reader); + Load(&obj->history, reader); + Load(&obj->name, reader); +} + } // namespace memgraph::slk #endif diff --git a/src/coordination/include/coordination/replication_instance.hpp b/src/coordination/include/coordination/replication_instance.hpp index 8001d0905..e8e00a0a8 100644 --- a/src/coordination/include/coordination/replication_instance.hpp +++ b/src/coordination/include/coordination/replication_instance.hpp @@ -18,17 +18,22 @@ #include "replication_coordination_glue/role.hpp" #include <libnuraft/nuraft.hxx> +#include "utils/resource_lock.hpp" #include "utils/result.hpp" #include "utils/uuid.hpp" namespace memgraph::coordination { class CoordinatorInstance; +class ReplicationInstance; + +using HealthCheckInstanceCallback = void (CoordinatorInstance::*)(std::string_view); class ReplicationInstance { public: - ReplicationInstance(CoordinatorInstance *peer, CoordinatorClientConfig config, HealthCheckCallback succ_cb, - HealthCheckCallback fail_cb); + ReplicationInstance(CoordinatorInstance *peer, CoordinatorClientConfig config, HealthCheckClientCallback succ_cb, + HealthCheckClientCallback fail_cb, HealthCheckInstanceCallback succ_instance_cb, + HealthCheckInstanceCallback fail_instance_cb); ReplicationInstance(ReplicationInstance const &other) = delete; ReplicationInstance &operator=(ReplicationInstance const &other) = delete; @@ -50,9 +55,10 @@ class ReplicationInstance { auto IsReplica() const -> bool; auto IsMain() const -> bool; - auto PromoteToMain(utils::UUID uuid, ReplicationClientsInfo repl_clients_info, HealthCheckCallback main_succ_cb, - HealthCheckCallback main_fail_cb) -> bool; - auto DemoteToReplica(HealthCheckCallback replica_succ_cb, HealthCheckCallback replica_fail_cb) -> bool; + auto PromoteToMain(utils::UUID uuid, ReplicationClientsInfo repl_clients_info, + HealthCheckInstanceCallback main_succ_cb, HealthCheckInstanceCallback main_fail_cb) -> bool; + auto DemoteToReplica(HealthCheckInstanceCallback replica_succ_cb, HealthCheckInstanceCallback replica_fail_cb) + -> bool; auto StartFrequentCheck() -> void; auto StopFrequentCheck() -> void; @@ -66,16 +72,17 @@ class ReplicationInstance { auto SendSwapAndUpdateUUID(const utils::UUID &new_main_uuid) -> bool; auto SendUnregisterReplicaRpc(std::string const &instance_name) -> bool; - auto SendGetInstanceUUID() -> utils::BasicResult<coordination::GetInstanceUUIDError, std::optional<utils::UUID>>; auto GetClient() -> CoordinatorClient &; auto EnableWritingOnMain() -> bool; auto SetNewMainUUID(utils::UUID const &main_uuid) -> void; - auto ResetMainUUID() -> void; auto GetMainUUID() const -> const std::optional<utils::UUID> &; + auto GetSuccessCallback() -> HealthCheckInstanceCallback &; + auto GetFailCallback() -> HealthCheckInstanceCallback &; + private: CoordinatorClient client_; replication_coordination_glue::ReplicationRole replication_role_; @@ -90,6 +97,9 @@ class ReplicationInstance { // so we need to send swap uuid again std::optional<utils::UUID> main_uuid_; + HealthCheckInstanceCallback succ_cb_; + HealthCheckInstanceCallback fail_cb_; + friend bool operator==(ReplicationInstance const &first, ReplicationInstance const &second) { return first.client_ == second.client_ && first.replication_role_ == second.replication_role_; } diff --git a/src/coordination/include/coordination/rpc_errors.hpp b/src/coordination/include/coordination/rpc_errors.hpp index f6bfbf3e0..3829d430a 100644 --- a/src/coordination/include/coordination/rpc_errors.hpp +++ b/src/coordination/include/coordination/rpc_errors.hpp @@ -11,4 +11,5 @@ namespace memgraph::coordination { enum class GetInstanceUUIDError { NO_RESPONSE, RPC_EXCEPTION }; +enum class GetInstanceTimestampsError { NO_RESPONSE, RPC_EXCEPTION }; } // namespace memgraph::coordination diff --git a/src/coordination/replication_instance.cpp b/src/coordination/replication_instance.cpp index 0d16db648..50f1be468 100644 --- a/src/coordination/replication_instance.cpp +++ b/src/coordination/replication_instance.cpp @@ -13,15 +13,21 @@ #include "coordination/replication_instance.hpp" +#include <utility> + #include "replication_coordination_glue/handler.hpp" #include "utils/result.hpp" namespace memgraph::coordination { ReplicationInstance::ReplicationInstance(CoordinatorInstance *peer, CoordinatorClientConfig config, - HealthCheckCallback succ_cb, HealthCheckCallback fail_cb) + HealthCheckClientCallback succ_cb, HealthCheckClientCallback fail_cb, + HealthCheckInstanceCallback succ_instance_cb, + HealthCheckInstanceCallback fail_instance_cb) : client_(peer, std::move(config), std::move(succ_cb), std::move(fail_cb)), - replication_role_(replication_coordination_glue::ReplicationRole::REPLICA) { + replication_role_(replication_coordination_glue::ReplicationRole::REPLICA), + succ_cb_(succ_instance_cb), + fail_cb_(fail_instance_cb) { if (!client_.DemoteToReplica()) { throw CoordinatorRegisterInstanceException("Failed to demote instance {} to replica", client_.InstanceName()); } @@ -57,26 +63,29 @@ auto ReplicationInstance::IsMain() const -> bool { } auto ReplicationInstance::PromoteToMain(utils::UUID new_uuid, ReplicationClientsInfo repl_clients_info, - HealthCheckCallback main_succ_cb, HealthCheckCallback main_fail_cb) -> bool { + HealthCheckInstanceCallback main_succ_cb, + HealthCheckInstanceCallback main_fail_cb) -> bool { if (!client_.SendPromoteReplicaToMainRpc(new_uuid, std::move(repl_clients_info))) { return false; } replication_role_ = replication_coordination_glue::ReplicationRole::MAIN; main_uuid_ = new_uuid; - client_.SetCallbacks(std::move(main_succ_cb), std::move(main_fail_cb)); + succ_cb_ = main_succ_cb; + fail_cb_ = main_fail_cb; return true; } -auto ReplicationInstance::DemoteToReplica(HealthCheckCallback replica_succ_cb, HealthCheckCallback replica_fail_cb) - -> bool { +auto ReplicationInstance::DemoteToReplica(HealthCheckInstanceCallback replica_succ_cb, + HealthCheckInstanceCallback replica_fail_cb) -> bool { if (!client_.DemoteToReplica()) { return false; } replication_role_ = replication_coordination_glue::ReplicationRole::REPLICA; - client_.SetCallbacks(std::move(replica_succ_cb), std::move(replica_fail_cb)); + succ_cb_ = replica_succ_cb; + fail_cb_ = replica_fail_cb; return true; } @@ -90,10 +99,12 @@ auto ReplicationInstance::ReplicationClientInfo() const -> CoordinatorClientConf return client_.ReplicationClientInfo(); } +auto ReplicationInstance::GetSuccessCallback() -> HealthCheckInstanceCallback & { return succ_cb_; } +auto ReplicationInstance::GetFailCallback() -> HealthCheckInstanceCallback & { return fail_cb_; } + auto ReplicationInstance::GetClient() -> CoordinatorClient & { return client_; } auto ReplicationInstance::SetNewMainUUID(utils::UUID const &main_uuid) -> void { main_uuid_ = main_uuid; } -auto ReplicationInstance::ResetMainUUID() -> void { main_uuid_ = std::nullopt; } auto ReplicationInstance::GetMainUUID() const -> std::optional<utils::UUID> const & { return main_uuid_; } auto ReplicationInstance::EnsureReplicaHasCorrectMainUUID(utils::UUID const &curr_main_uuid) -> bool { diff --git a/src/dbms/dbms_handler.hpp b/src/dbms/dbms_handler.hpp index 7b1d45335..87d1257a6 100644 --- a/src/dbms/dbms_handler.hpp +++ b/src/dbms/dbms_handler.hpp @@ -266,10 +266,6 @@ class DbmsHandler { bool IsMain() const { return repl_state_.IsMain(); } bool IsReplica() const { return repl_state_.IsReplica(); } -#ifdef MG_ENTERPRISE - // coordination::CoordinatorState &CoordinatorState() { return coordinator_state_; } -#endif - /** * @brief Return all active databases. * diff --git a/src/replication_coordination_glue/CMakeLists.txt b/src/replication_coordination_glue/CMakeLists.txt index f81aed4ba..f452e1c1f 100644 --- a/src/replication_coordination_glue/CMakeLists.txt +++ b/src/replication_coordination_glue/CMakeLists.txt @@ -7,6 +7,7 @@ target_sources(mg-repl_coord_glue mode.hpp role.hpp handler.hpp + common.hpp PRIVATE messages.cpp diff --git a/src/replication_coordination_glue/common.hpp b/src/replication_coordination_glue/common.hpp new file mode 100644 index 000000000..439e5cae8 --- /dev/null +++ b/src/replication_coordination_glue/common.hpp @@ -0,0 +1,32 @@ +// 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 +// 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. + +#pragma once + +#include "rpc/client.hpp" +#include "utils/uuid.hpp" + +#include <deque> +#include "messages.hpp" +#include "rpc/messages.hpp" +#include "utils/uuid.hpp" + +namespace memgraph::replication_coordination_glue { + +struct DatabaseHistory { + memgraph::utils::UUID db_uuid; + std::vector<std::pair<std::string, uint64_t>> history; + std::string name; +}; + +using DatabaseHistories = std::vector<DatabaseHistory>; + +} // namespace memgraph::replication_coordination_glue diff --git a/src/replication_handler/include/replication_handler/replication_handler.hpp b/src/replication_handler/include/replication_handler/replication_handler.hpp index b110e6015..d5c2bfa71 100644 --- a/src/replication_handler/include/replication_handler/replication_handler.hpp +++ b/src/replication_handler/include/replication_handler/replication_handler.hpp @@ -14,6 +14,7 @@ #include "dbms/dbms_handler.hpp" #include "flags/experimental.hpp" #include "replication/include/replication/state.hpp" +#include "replication_coordination_glue/common.hpp" #include "replication_handler/system_replication.hpp" #include "replication_handler/system_rpc.hpp" #include "utils/result.hpp" @@ -149,6 +150,8 @@ struct ReplicationHandler : public memgraph::query::ReplicationQueryHandler { auto GetReplicaUUID() -> std::optional<utils::UUID>; + auto GetDatabasesHistories() -> replication_coordination_glue::DatabaseHistories; + private: template <bool SendSwapUUID> auto RegisterReplica_(const memgraph::replication::ReplicationClientConfig &config) diff --git a/src/replication_handler/replication_handler.cpp b/src/replication_handler/replication_handler.cpp index 5f807779d..ea567eed0 100644 --- a/src/replication_handler/replication_handler.cpp +++ b/src/replication_handler/replication_handler.cpp @@ -14,6 +14,7 @@ #include "dbms/dbms_handler.hpp" #include "replication/replication_client.hpp" #include "replication_handler/system_replication.hpp" +#include "utils/functional.hpp" namespace memgraph::replication { @@ -265,8 +266,26 @@ auto ReplicationHandler::GetRole() const -> replication_coordination_glue::Repli return repl_state_.GetRole(); } +auto ReplicationHandler::GetDatabasesHistories() -> replication_coordination_glue::DatabaseHistories { + replication_coordination_glue::DatabaseHistories results; + dbms_handler_.ForEach([&results](memgraph::dbms::DatabaseAccess db_acc) { + auto &repl_storage_state = db_acc->storage()->repl_storage_state_; + + std::vector<std::pair<std::string, uint64_t>> history = + utils::fmap([](const auto &elem) { return std::pair<std::string, uint64_t>(elem.first, elem.second); }, + repl_storage_state.history); + + history.emplace_back(std::string(repl_storage_state.epoch_.id()), repl_storage_state.last_commit_timestamp_.load()); + replication_coordination_glue::DatabaseHistory repl{ + .db_uuid = utils::UUID{db_acc->storage()->uuid()}, .history = history, .name = std::string(db_acc->name())}; + results.emplace_back(repl); + }); + + return results; +} + auto ReplicationHandler::GetReplicaUUID() -> std::optional<utils::UUID> { - MG_ASSERT(repl_state_.IsReplica()); + MG_ASSERT(repl_state_.IsReplica(), "Instance is not replica"); return std::get<RoleReplicaData>(repl_state_.ReplicationData()).uuid_; } diff --git a/src/storage/v2/inmemory/replication/recovery.cpp b/src/storage/v2/inmemory/replication/recovery.cpp index 921c1f5c0..fe752bfd1 100644 --- a/src/storage/v2/inmemory/replication/recovery.cpp +++ b/src/storage/v2/inmemory/replication/recovery.cpp @@ -106,8 +106,8 @@ uint64_t ReplicateCurrentWal(const utils::UUID &main_uuid, const InMemoryStorage return response.current_commit_timestamp; } -/// This method tries to find the optimal path for recoverying a single replica. -/// Based on the last commit transfered to replica it tries to update the +/// This method tries to find the optimal path for recovering a single replica. +/// Based on the last commit transferred to replica it tries to update the /// replica using durability files - WALs and Snapshots. WAL files are much /// smaller in size as they contain only the Deltas (changes) made during the /// transactions while Snapshots contain all the data. For that reason we prefer @@ -175,7 +175,7 @@ std::vector<RecoveryStep> GetRecoverySteps(uint64_t replica_commit, utils::FileR auto add_snapshot = [&]() { if (!latest_snapshot) return; const auto lock_success = locker_acc.AddPath(latest_snapshot->path); - MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant snapshot path."); + MG_ASSERT(!lock_success.HasError(), "Tried to lock a non-existent snapshot path."); recovery_steps.emplace_back(std::in_place_type_t<RecoverySnapshot>{}, std::move(latest_snapshot->path)); }; diff --git a/src/storage/v2/replication/replication_client.cpp b/src/storage/v2/replication/replication_client.cpp index 16429d11f..1eb06bf10 100644 --- a/src/storage/v2/replication/replication_client.cpp +++ b/src/storage/v2/replication/replication_client.cpp @@ -53,11 +53,13 @@ void ReplicationStorageClient::UpdateReplicaState(Storage *storage, DatabaseAcce #endif std::optional<uint64_t> branching_point; + // different epoch id, replica was main if (replica.epoch_id != replStorageState.epoch_.id() && replica.current_commit_timestamp != kTimestampInitialId) { auto const &history = replStorageState.history; const auto epoch_info_iter = std::find_if(history.crbegin(), history.crend(), [&](const auto &main_epoch_info) { return main_epoch_info.first == replica.epoch_id; }); + // main didn't have that epoch, but why is here branching point if (epoch_info_iter == history.crend()) { branching_point = 0; } else if (epoch_info_iter->second != replica.current_commit_timestamp) { diff --git a/src/utils/functional.hpp b/src/utils/functional.hpp index e0714de2a..f5242944a 100644 --- a/src/utils/functional.hpp +++ b/src/utils/functional.hpp @@ -18,8 +18,11 @@ namespace memgraph::utils { -template <class F, class T, class R = typename std::invoke_result<F, T>::type> -auto fmap(F &&f, std::vector<T> const &v) -> std::vector<R> { +template <template <typename, typename...> class Container, typename T, typename Allocator = std::allocator<T>, + typename F, typename R = std::invoke_result_t<F, T>> +requires ranges::range<Container<T, Allocator>> && + (!std::same_as<Container<T, Allocator>, std::string>)auto fmap(F &&f, const Container<T, Allocator> &v) + -> std::vector<R> { return v | ranges::views::transform(std::forward<F>(f)) | ranges::to<std::vector<R>>(); } diff --git a/src/utils/scheduler.hpp b/src/utils/scheduler.hpp index 742271a95..45b2c8b04 100644 --- a/src/utils/scheduler.hpp +++ b/src/utils/scheduler.hpp @@ -57,7 +57,7 @@ class Scheduler { // program and there is probably no work to do in scheduled function at // the start of the program. Since Server will log some messages on // the program start we let him log first and we make sure by first - // waiting that funcion f will not log before it. + // waiting that function f will not log before it. // Check for pause also. std::unique_lock<std::mutex> lk(mutex_); auto now = std::chrono::system_clock::now(); diff --git a/src/utils/typeinfo.hpp b/src/utils/typeinfo.hpp index 1ca08a3f7..aadc8a07b 100644 --- a/src/utils/typeinfo.hpp +++ b/src/utils/typeinfo.hpp @@ -114,6 +114,8 @@ enum class TypeId : uint64_t { COORD_GET_UUID_REQ, COORD_GET_UUID_RES, + COORD_GET_INSTANCE_DATABASES_REQ, + COORD_GET_INSTANCE_DATABASES_RES, // AST AST_LABELIX = 3000, diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt index 6f7b3bbef..b92989f4e 100644 --- a/tests/unit/CMakeLists.txt +++ b/tests/unit/CMakeLists.txt @@ -430,3 +430,11 @@ target_include_directories(${test_prefix}distributed_lamport_clock PRIVATE ${CMA add_unit_test(query_hint_provider.cpp) target_link_libraries(${test_prefix}query_hint_provider mg-query mg-glue) + + +# Test coordination +if(MG_ENTERPRISE) +add_unit_test(coordination_utils.cpp) +target_link_libraries(${test_prefix}coordination_utils gflags mg-coordination mg-repl_coord_glue) +target_include_directories(${test_prefix}coordination_utils PRIVATE ${CMAKE_SOURCE_DIR}/include) +endif() diff --git a/tests/unit/coordination_utils.cpp b/tests/unit/coordination_utils.cpp new file mode 100644 index 000000000..1346dce2c --- /dev/null +++ b/tests/unit/coordination_utils.cpp @@ -0,0 +1,246 @@ +// 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 +// 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 <gflags/gflags.h> +#include <gtest/gtest.h> +#include "coordination/coordinator_instance.hpp" +#include "dbms/constants.hpp" +#include "replication_coordination_glue/common.hpp" +#include "utils/functional.hpp" + +class CoordinationUtils : public ::testing::Test { + protected: + void SetUp() override {} + + void TearDown() override {} + + std::filesystem::path test_folder_{std::filesystem::temp_directory_path() / "MG_tests_unit_coordination"}; +}; + +TEST_F(CoordinationUtils, MemgraphDbHistorySimple) { + // Choose any if everything is same + // X = dead + // Main : A(24) B(36) C(48) D(50) E(51) X + // replica 1: A(24) B(36) C(48) D(50) E(51) + // replica 2: A(24) B(36) C(48) D(50) E(51) + // replica 3: A(24) B(36) C(48) D(50) E(51) + std::vector<std::pair<std::string, memgraph::replication_coordination_glue::DatabaseHistories>> + instance_database_histories; + + std::vector<std::pair<memgraph::utils::UUID, uint64_t>> histories; + histories.emplace_back(memgraph::utils::UUID{}, 24); + histories.emplace_back(memgraph::utils::UUID{}, 36); + histories.emplace_back(memgraph::utils::UUID{}, 48); + histories.emplace_back(memgraph::utils::UUID{}, 50); + histories.emplace_back(memgraph::utils::UUID{}, 51); + + memgraph::utils::UUID db_uuid; + std::string default_name = std::string(memgraph::dbms::kDefaultDB); + + auto db_histories = memgraph::utils::fmap( + [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }, + histories); + + memgraph::replication_coordination_glue::DatabaseHistory history{ + .db_uuid = db_uuid, .history = db_histories, .name = default_name}; + + memgraph::replication_coordination_glue::DatabaseHistories instance_1_db_histories_{history}; + instance_database_histories.emplace_back("instance_1", instance_1_db_histories_); + + memgraph::replication_coordination_glue::DatabaseHistories instance_2_db_histories_{history}; + instance_database_histories.emplace_back("instance_2", instance_2_db_histories_); + + memgraph::replication_coordination_glue::DatabaseHistories instance_3_db_histories_{history}; + instance_database_histories.emplace_back("instance_3", instance_3_db_histories_); + memgraph::coordination::CoordinatorInstance instance; + + auto [instance_name, latest_epoch, latest_commit_timestamp] = + instance.ChooseMostUpToDateInstance(instance_database_histories); + ASSERT_TRUE(instance_name == "instance_1" || instance_name == "instance_2" || instance_name == "instance_3"); + ASSERT_TRUE(*latest_epoch == db_histories.back().first); + ASSERT_TRUE(*latest_commit_timestamp == db_histories.back().second); +} + +TEST_F(CoordinationUtils, MemgraphDbHistoryLastEpochDifferent) { + // Prioritize one with the biggest last commit timestamp on last epoch + // X = dead + // Main : A(24) B(36) C(48) D(50) E(59) X + // replica 1: A(24) B(12) C(15) D(17) E(51) + // replica 2: A(24) B(12) C(15) D(17) E(57) + // replica 3: A(24) B(12) C(15) D(17) E(59) + std::vector<std::pair<std::string, memgraph::replication_coordination_glue::DatabaseHistories>> + instance_database_histories; + + std::vector<std::pair<memgraph::utils::UUID, uint64_t>> histories; + histories.emplace_back(memgraph::utils::UUID{}, 24); + histories.emplace_back(memgraph::utils::UUID{}, 36); + histories.emplace_back(memgraph::utils::UUID{}, 48); + histories.emplace_back(memgraph::utils::UUID{}, 50); + histories.emplace_back(memgraph::utils::UUID{}, 59); + + memgraph::utils::UUID db_uuid; + std::string default_name = std::string(memgraph::dbms::kDefaultDB); + + auto db_histories = memgraph::utils::fmap( + [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }, + histories); + + db_histories.back().second = 51; + memgraph::replication_coordination_glue::DatabaseHistory history1{ + .db_uuid = db_uuid, .history = db_histories, .name = default_name}; + + memgraph::replication_coordination_glue::DatabaseHistories instance_1_db_histories_{history1}; + instance_database_histories.emplace_back("instance_1", instance_1_db_histories_); + + db_histories.back().second = 57; + memgraph::replication_coordination_glue::DatabaseHistory history2{ + .db_uuid = db_uuid, .history = db_histories, .name = default_name}; + memgraph::replication_coordination_glue::DatabaseHistories instance_2_db_histories_{history2}; + instance_database_histories.emplace_back("instance_2", instance_2_db_histories_); + + db_histories.back().second = 59; + memgraph::replication_coordination_glue::DatabaseHistory history3{ + .db_uuid = db_uuid, .history = db_histories, .name = default_name}; + memgraph::replication_coordination_glue::DatabaseHistories instance_3_db_histories_{history3}; + instance_database_histories.emplace_back("instance_3", instance_3_db_histories_); + + memgraph::coordination::CoordinatorInstance instance; + auto [instance_name, latest_epoch, latest_commit_timestamp] = + instance.ChooseMostUpToDateInstance(instance_database_histories); + + ASSERT_TRUE(instance_name == "instance_3"); + ASSERT_TRUE(*latest_epoch == db_histories.back().first); + ASSERT_TRUE(*latest_commit_timestamp == db_histories.back().second); +} + +TEST_F(CoordinationUtils, MemgraphDbHistoryOneInstanceAheadFewEpochs) { + // Prioritize one biggest commit timestamp + // X = dead + // Main : A(24) B(36) C(48) D(50) E(51) X X X X + // replica 1: A(24) B(36) C(48) D(50) E(51) F(60) G(65) X up + // replica 2: A(24) B(36) C(48) D(50) E(51) X X X up + // replica 3: A(24) B(36) C(48) D(50) E(51) X X X up + std::vector<std::pair<std::string, memgraph::replication_coordination_glue::DatabaseHistories>> + instance_database_histories; + + std::vector<std::pair<memgraph::utils::UUID, uint64_t>> histories; + histories.emplace_back(memgraph::utils::UUID{}, 24); + histories.emplace_back(memgraph::utils::UUID{}, 36); + histories.emplace_back(memgraph::utils::UUID{}, 48); + histories.emplace_back(memgraph::utils::UUID{}, 50); + histories.emplace_back(memgraph::utils::UUID{}, 51); + + memgraph::utils::UUID db_uuid; + std::string default_name = std::string(memgraph::dbms::kDefaultDB); + + auto db_histories = memgraph::utils::fmap( + [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }, + histories); + + memgraph::replication_coordination_glue::DatabaseHistory history{ + .db_uuid = db_uuid, .history = db_histories, .name = default_name}; + + memgraph::replication_coordination_glue::DatabaseHistories instance_1_db_histories_{history}; + instance_database_histories.emplace_back("instance_1", instance_1_db_histories_); + + memgraph::replication_coordination_glue::DatabaseHistories instance_2_db_histories_{history}; + instance_database_histories.emplace_back("instance_2", instance_2_db_histories_); + + histories.emplace_back(memgraph::utils::UUID{}, 60); + histories.emplace_back(memgraph::utils::UUID{}, 65); + auto db_histories_longest = memgraph::utils::fmap( + [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }, + histories); + + memgraph::replication_coordination_glue::DatabaseHistory history_longest{ + .db_uuid = db_uuid, .history = db_histories_longest, .name = default_name}; + + memgraph::replication_coordination_glue::DatabaseHistories instance_3_db_histories_{history_longest}; + instance_database_histories.emplace_back("instance_3", instance_3_db_histories_); + + memgraph::coordination::CoordinatorInstance instance; + auto [instance_name, latest_epoch, latest_commit_timestamp] = + instance.ChooseMostUpToDateInstance(instance_database_histories); + + ASSERT_TRUE(instance_name == "instance_3"); + ASSERT_TRUE(*latest_epoch == db_histories_longest.back().first); + ASSERT_TRUE(*latest_commit_timestamp == db_histories_longest.back().second); +} + +TEST_F(CoordinationUtils, MemgraphDbHistoryInstancesHistoryDiverged) { + // When history diverged, also prioritize one with biggest last commit timestamp + // Main : A(1) B(2) C(3) X + // replica 1: A(1) B(2) C(3) X X up + // replica 2: A(1) B(2) X D(5) X up + // replica 3: A(1) B(2) X D(4) X up + std::vector<std::pair<std::string, memgraph::replication_coordination_glue::DatabaseHistories>> + instance_database_histories; + + std::vector<std::pair<memgraph::utils::UUID, uint64_t>> histories; + histories.emplace_back(memgraph::utils::UUID{}, 1); + histories.emplace_back(memgraph::utils::UUID{}, 2); + histories.emplace_back(memgraph::utils::UUID{}, 3); + + memgraph::utils::UUID db_uuid; + std::string default_name = std::string(memgraph::dbms::kDefaultDB); + + auto db_histories = memgraph::utils::fmap( + [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }, + histories); + + memgraph::replication_coordination_glue::DatabaseHistory history{ + .db_uuid = db_uuid, .history = db_histories, .name = default_name}; + + memgraph::replication_coordination_glue::DatabaseHistories instance_1_db_histories_{history}; + instance_database_histories.emplace_back("instance_1", instance_1_db_histories_); + + db_histories.pop_back(); + + auto oldest_commit_timestamp{5}; + auto newest_different_epoch = memgraph::utils::UUID{}; + histories.emplace_back(newest_different_epoch, oldest_commit_timestamp); + auto db_histories_different = memgraph::utils::fmap( + [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }, + histories); + + memgraph::replication_coordination_glue::DatabaseHistory history_3{ + .db_uuid = db_uuid, .history = db_histories_different, .name = default_name}; + + memgraph::replication_coordination_glue::DatabaseHistories instance_3_db_histories_{history_3}; + instance_database_histories.emplace_back("instance_3", instance_3_db_histories_); + + db_histories_different.back().second = 4; + memgraph::replication_coordination_glue::DatabaseHistory history_2{ + .db_uuid = db_uuid, .history = db_histories_different, .name = default_name}; + + memgraph::replication_coordination_glue::DatabaseHistories instance_2_db_histories_{history_2}; + instance_database_histories.emplace_back("instance_2", instance_2_db_histories_); + + memgraph::coordination::CoordinatorInstance instance; + auto [instance_name, latest_epoch, latest_commit_timestamp] = + instance.ChooseMostUpToDateInstance(instance_database_histories); + + ASSERT_TRUE(instance_name == "instance_3"); + ASSERT_TRUE(*latest_epoch == std::string(newest_different_epoch)); + ASSERT_TRUE(*latest_commit_timestamp == oldest_commit_timestamp); +} From 55f224839e51e924a9256a4291f4af57a0132c63 Mon Sep 17 00:00:00 2001 From: Gareth Andrew Lloyd <gareth.lloyd@memgraph.io> Date: Wed, 28 Feb 2024 16:46:03 +0000 Subject: [PATCH 05/18] Do not use UUID_STR_LEN (#1770) Older libuuid did not have this macro, we need to publish for older distro with older libs. --- src/utils/uuid.hpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/utils/uuid.hpp b/src/utils/uuid.hpp index bca55d73b..f83811d10 100644 --- a/src/utils/uuid.hpp +++ b/src/utils/uuid.hpp @@ -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 @@ -39,9 +39,10 @@ struct UUID { UUID() { uuid_generate(uuid.data()); } explicit operator std::string() const { - auto decoded = std::array<char, UUID_STR_LEN>{}; + // Note not using UUID_STR_LEN so we can build with older libuuid + auto decoded = std::array<char, 37 /*UUID_STR_LEN*/>{}; uuid_unparse(uuid.data(), decoded.data()); - return std::string{decoded.data(), UUID_STR_LEN - 1}; + return std::string{decoded.data(), 37 /*UUID_STR_LEN*/ - 1}; } explicit operator arr_t() const { return uuid; } From f316f7db872f6bdc06aed66df250f6ba983eb959 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Bari=C5=A1i=C4=87?= <48765171+MarkoBarisic@users.noreply.github.com> Date: Wed, 28 Feb 2024 18:21:56 +0100 Subject: [PATCH 06/18] Add openssl to MEMGRAPH_BUILD_DEPS for amzn-2 and centos-7 (#1771) --- environment/os/amzn-2.sh | 1 + environment/os/centos-7.sh | 1 + 2 files changed, 2 insertions(+) diff --git a/environment/os/amzn-2.sh b/environment/os/amzn-2.sh index a9cc3e4b2..bac62233e 100755 --- a/environment/os/amzn-2.sh +++ b/environment/os/amzn-2.sh @@ -45,6 +45,7 @@ MEMGRAPH_BUILD_DEPS=( readline-devel # for memgraph console python3-devel # for query modules openssl-devel + openssl libseccomp-devel python3 python3-pip nmap-ncat # for tests # diff --git a/environment/os/centos-7.sh b/environment/os/centos-7.sh index d9fc93912..0e029fab5 100755 --- a/environment/os/centos-7.sh +++ b/environment/os/centos-7.sh @@ -43,6 +43,7 @@ MEMGRAPH_BUILD_DEPS=( readline-devel # for memgraph console python3-devel # for query modules openssl-devel + openssl libseccomp-devel python3 python-virtualenv python3-pip nmap-ncat # for qa, macro_benchmark and stress tests # From 33caa271617bf867b58bf34aa38ff91a3c478cb8 Mon Sep 17 00:00:00 2001 From: Antonio Filipovic <61245998+antoniofilipovic@users.noreply.github.com> Date: Fri, 1 Mar 2024 12:32:56 +0100 Subject: [PATCH 07/18] Ensure replication works on HA cluster in different scenarios (#1743) --- src/coordination/coordinator_client.cpp | 4 +- src/coordination/coordinator_instance.cpp | 33 +- .../coordination/coordinator_instance.hpp | 4 +- src/dbms/dbms_handler.cpp | 13 +- src/dbms/dbms_handler.hpp | 10 +- src/memgraph.cpp | 3 +- .../replication_handler.cpp | 5 +- src/storage/v2/durability/durability.cpp | 48 +- .../v2/inmemory/replication/recovery.cpp | 2 +- src/storage/v2/inmemory/storage.cpp | 1 + .../v2/replication/replication_client.cpp | 16 +- .../high_availability/single_coordinator.py | 828 +++++++++++++++++- .../show_while_creating_invalid_state.py | 28 +- tests/unit/coordination_utils.cpp | 16 +- 14 files changed, 924 insertions(+), 87 deletions(-) diff --git a/src/coordination/coordinator_client.cpp b/src/coordination/coordinator_client.cpp index f4d2da838..b4f82b60c 100644 --- a/src/coordination/coordinator_client.cpp +++ b/src/coordination/coordinator_client.cpp @@ -175,9 +175,7 @@ auto CoordinatorClient::SendGetInstanceTimestampsRpc() const -> utils::BasicResult<GetInstanceUUIDError, replication_coordination_glue::DatabaseHistories> { try { auto stream{rpc_client_.Stream<coordination::GetDatabaseHistoriesRpc>()}; - auto res = stream.AwaitResponse(); - - return res.database_histories; + return stream.AwaitResponse().database_histories; } catch (const rpc::RpcFailedException &) { spdlog::error("RPC error occured while sending GetInstance UUID RPC"); diff --git a/src/coordination/coordinator_instance.cpp b/src/coordination/coordinator_instance.cpp index ba94d9d5f..daf1d7138 100644 --- a/src/coordination/coordinator_instance.cpp +++ b/src/coordination/coordinator_instance.cpp @@ -127,7 +127,7 @@ auto CoordinatorInstance::TryFailover() -> void { ChooseMostUpToDateInstance(instance_database_histories); spdlog::trace("The most up to date instance is {} with epoch {} and {} latest commit timestamp", - most_up_to_date_instance, *latest_epoch, *latest_commit_timestamp); + most_up_to_date_instance, latest_epoch, latest_commit_timestamp); auto *new_main = &FindReplicationInstance(most_up_to_date_instance); @@ -391,7 +391,7 @@ auto CoordinatorInstance::SetMainUUID(utils::UUID new_uuid) -> void { main_uuid_ auto CoordinatorInstance::ChooseMostUpToDateInstance( const std::vector<std::pair<std::string, replication_coordination_glue::DatabaseHistories>> &instance_database_histories) -> NewMainRes { - NewMainRes new_main_res; + std::optional<NewMainRes> new_main_res; std::for_each( instance_database_histories.begin(), instance_database_histories.end(), [&new_main_res](const InstanceNameDbHistories &instance_res_pair) { @@ -407,7 +407,7 @@ auto CoordinatorInstance::ChooseMostUpToDateInstance( std::ranges::for_each( instance_db_histories, [&instance_name = instance_name](const replication_coordination_glue::DatabaseHistory &db_history) { - spdlog::trace("Instance {}: name {}, default db {}", instance_name, db_history.name, + spdlog::debug("Instance {}: name {}, default db {}", instance_name, db_history.name, memgraph::dbms::kDefaultDB); }); @@ -417,35 +417,26 @@ auto CoordinatorInstance::ChooseMostUpToDateInstance( std::ranges::for_each(instance_default_db_history | ranges::views::reverse, [&instance_name = instance_name](const auto &epoch_history_it) { - spdlog::trace("Instance {}: epoch {}, last_commit_timestamp: {}", instance_name, + spdlog::debug("Instance {}: epoch {}, last_commit_timestamp: {}", instance_name, std::get<0>(epoch_history_it), std::get<1>(epoch_history_it)); }); // get latest epoch // get latest timestamp - if (!new_main_res.latest_epoch) { + if (!new_main_res) { const auto &[epoch, timestamp] = *instance_default_db_history.crbegin(); - new_main_res = NewMainRes{ - .most_up_to_date_instance = instance_name, - .latest_epoch = epoch, - .latest_commit_timestamp = timestamp, - }; - spdlog::trace("Currently the most up to date instance is {} with epoch {} and {} latest commit timestamp", + new_main_res = std::make_optional<NewMainRes>({instance_name, epoch, timestamp}); + spdlog::debug("Currently the most up to date instance is {} with epoch {} and {} latest commit timestamp", instance_name, epoch, timestamp); return; } bool found_same_point{false}; - std::string last_most_up_to_date_epoch{*new_main_res.latest_epoch}; + std::string last_most_up_to_date_epoch{new_main_res->latest_epoch}; for (auto [epoch, timestamp] : ranges::reverse_view(instance_default_db_history)) { - if (*new_main_res.latest_commit_timestamp < timestamp) { - new_main_res = NewMainRes{ - .most_up_to_date_instance = instance_name, - .latest_epoch = epoch, - .latest_commit_timestamp = timestamp, - }; - + if (new_main_res->latest_commit_timestamp < timestamp) { + new_main_res = std::make_optional<NewMainRes>({instance_name, epoch, timestamp}); spdlog::trace("Found the new most up to date instance {} with epoch {} and {} latest commit timestamp", instance_name, epoch, timestamp); } @@ -459,11 +450,11 @@ auto CoordinatorInstance::ChooseMostUpToDateInstance( if (!found_same_point) { spdlog::error("Didn't find same history epoch {} for instance {} and instance {}", last_most_up_to_date_epoch, - new_main_res.most_up_to_date_instance, instance_name); + new_main_res->most_up_to_date_instance, instance_name); } }); - return new_main_res; + return std::move(*new_main_res); } } // namespace memgraph::coordination #endif diff --git a/src/coordination/include/coordination/coordinator_instance.hpp b/src/coordination/include/coordination/coordinator_instance.hpp index bed202744..ee5a6fb6e 100644 --- a/src/coordination/include/coordination/coordinator_instance.hpp +++ b/src/coordination/include/coordination/coordinator_instance.hpp @@ -28,8 +28,8 @@ namespace memgraph::coordination { struct NewMainRes { std::string most_up_to_date_instance; - std::optional<std::string> latest_epoch; - std::optional<uint64_t> latest_commit_timestamp; + std::string latest_epoch; + uint64_t latest_commit_timestamp; }; using InstanceNameDbHistories = std::pair<std::string, replication_coordination_glue::DatabaseHistories>; diff --git a/src/dbms/dbms_handler.cpp b/src/dbms/dbms_handler.cpp index 1c38106db..16927d7e2 100644 --- a/src/dbms/dbms_handler.cpp +++ b/src/dbms/dbms_handler.cpp @@ -185,6 +185,16 @@ DbmsHandler::DbmsHandler(storage::Config config, replication::ReplicationState & auto directories = std::set{std::string{kDefaultDB}}; // Recover previous databases + if (flags::AreExperimentsEnabled(flags::Experiments::SYSTEM_REPLICATION) && !recovery_on_startup) { + // This will result in dropping databases on SystemRecoveryHandler + // for MT case, and for single DB case we might not even set replication as commit timestamp is checked + spdlog::warn( + "Data recovery on startup not set, this will result in dropping database in case of multi-tenancy enabled."); + } + + // TODO: Problem is if user doesn't set this up "database" name won't be recovered + // but if storage-recover-on-startup is true storage will be recovered which is an issue + spdlog::info("Data recovery on startup set to {}", recovery_on_startup); if (recovery_on_startup) { auto it = durability_->begin(std::string(kDBPrefix)); auto end = durability_->end(std::string(kDBPrefix)); @@ -410,9 +420,10 @@ void DbmsHandler::UpdateDurability(const storage::Config &config, std::optional< if (!durability_) return; // Save database in a list of active databases const auto &key = Durability::GenKey(config.salient.name); - if (rel_dir == std::nullopt) + if (rel_dir == std::nullopt) { rel_dir = std::filesystem::relative(config.durability.storage_directory, default_config_.durability.storage_directory); + } const auto &val = Durability::GenVal(config.salient.uuid, *rel_dir); durability_->Put(key, val); } diff --git a/src/dbms/dbms_handler.hpp b/src/dbms/dbms_handler.hpp index 87d1257a6..b0bbd5758 100644 --- a/src/dbms/dbms_handler.hpp +++ b/src/dbms/dbms_handler.hpp @@ -155,6 +155,8 @@ class DbmsHandler { spdlog::debug("Trying to create db '{}' on replica which already exists.", config.name); auto db = Get_(config.name); + spdlog::debug("Aligning database with name {} which has UUID {}, where config UUID is {}", config.name, + std::string(db->uuid()), std::string(config.uuid)); if (db->uuid() == config.uuid) { // Same db return db; } @@ -163,18 +165,22 @@ class DbmsHandler { // TODO: Fix this hack if (config.name == kDefaultDB) { + spdlog::debug("Last commit timestamp for DB {} is {}", kDefaultDB, + db->storage()->repl_storage_state_.last_commit_timestamp_); + // This seems correct, if database made progress if (db->storage()->repl_storage_state_.last_commit_timestamp_ != storage::kTimestampInitialId) { spdlog::debug("Default storage is not clean, cannot update UUID..."); return NewError::GENERIC; // Update error } - spdlog::debug("Update default db's UUID"); + spdlog::debug("Updated default db's UUID"); // Default db cannot be deleted and remade, have to just update the UUID db->storage()->config_.salient.uuid = config.uuid; UpdateDurability(db->storage()->config_, "."); return db; } - spdlog::debug("Drop database and recreate with the correct UUID"); + spdlog::debug("Dropping database {} with UUID: {} and recreating with the correct UUID: {}", config.name, + std::string(db->uuid()), std::string(config.uuid)); // Defer drop (void)Delete_(db->name()); // Second attempt diff --git a/src/memgraph.cpp b/src/memgraph.cpp index 34d64f434..d896bcc4c 100644 --- a/src/memgraph.cpp +++ b/src/memgraph.cpp @@ -334,7 +334,8 @@ int main(int argc, char **argv) { .salient.items = {.properties_on_edges = FLAGS_storage_properties_on_edges, .enable_schema_metadata = FLAGS_storage_enable_schema_metadata}, .salient.storage_mode = memgraph::flags::ParseStorageMode()}; - + spdlog::info("config recover on startup {}, flags {} {}", db_config.durability.recover_on_startup, + FLAGS_storage_recover_on_startup, FLAGS_data_recovery_on_startup); memgraph::utils::Scheduler jemalloc_purge_scheduler; jemalloc_purge_scheduler.Run("Jemalloc purge", std::chrono::seconds(FLAGS_storage_gc_cycle_sec), [] { memgraph::memory::PurgeUnusedMemory(); }); diff --git a/src/replication_handler/replication_handler.cpp b/src/replication_handler/replication_handler.cpp index ea567eed0..fc3dd3da4 100644 --- a/src/replication_handler/replication_handler.cpp +++ b/src/replication_handler/replication_handler.cpp @@ -271,9 +271,8 @@ auto ReplicationHandler::GetDatabasesHistories() -> replication_coordination_glu dbms_handler_.ForEach([&results](memgraph::dbms::DatabaseAccess db_acc) { auto &repl_storage_state = db_acc->storage()->repl_storage_state_; - std::vector<std::pair<std::string, uint64_t>> history = - utils::fmap([](const auto &elem) { return std::pair<std::string, uint64_t>(elem.first, elem.second); }, - repl_storage_state.history); + std::vector<std::pair<std::string, uint64_t>> history = utils::fmap( + [](const auto &elem) { return std::make_pair(elem.first, elem.second); }, repl_storage_state.history); history.emplace_back(std::string(repl_storage_state.epoch_.id()), repl_storage_state.last_commit_timestamp_.load()); replication_coordination_glue::DatabaseHistory repl{ diff --git a/src/storage/v2/durability/durability.cpp b/src/storage/v2/durability/durability.cpp index 92c4d11e8..b81357902 100644 --- a/src/storage/v2/durability/durability.cpp +++ b/src/storage/v2/durability/durability.cpp @@ -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 @@ -118,6 +118,8 @@ std::optional<std::vector<WalDurabilityInfo>> GetWalFiles(const std::filesystem: if (!item.is_regular_file()) continue; try { auto info = ReadWalInfo(item.path()); + spdlog::trace("Getting wal file with following info: uuid: {}, epoch id: {}, from timestamp {}, to_timestamp {} ", + info.uuid, info.epoch_id, info.from_timestamp, info.to_timestamp); if ((uuid.empty() || info.uuid == uuid) && (!current_seq_num || info.seq_num < *current_seq_num)) { wal_files.emplace_back(info.seq_num, info.from_timestamp, info.to_timestamp, std::move(info.uuid), std::move(info.epoch_id), item.path()); @@ -356,6 +358,7 @@ std::optional<RecoveryInfo> Recovery::RecoverData(std::string *uuid, Replication spdlog::warn(utils::MessageWithLink("No snapshot or WAL file found.", "https://memgr.ph/durability")); return std::nullopt; } + // TODO(antoniofilipovic) What is the logic here? std::sort(wal_files.begin(), wal_files.end()); // UUID used for durability is the UUID of the last WAL file. // Same for the epoch id. @@ -410,22 +413,17 @@ std::optional<RecoveryInfo> Recovery::RecoverData(std::string *uuid, Replication std::optional<uint64_t> previous_seq_num; auto last_loaded_timestamp = snapshot_timestamp; spdlog::info("Trying to load WAL files."); + + if (last_loaded_timestamp) { + epoch_history->emplace_back(repl_storage_state.epoch_.id(), *last_loaded_timestamp); + } + for (auto &wal_file : wal_files) { if (previous_seq_num && (wal_file.seq_num - *previous_seq_num) > 1) { LOG_FATAL("You are missing a WAL file with the sequence number {}!", *previous_seq_num + 1); } previous_seq_num = wal_file.seq_num; - if (wal_file.epoch_id != repl_storage_state.epoch_.id()) { - // This way we skip WALs finalized only because of role change. - // We can also set the last timestamp to 0 if last loaded timestamp - // is nullopt as this can only happen if the WAL file with seq = 0 - // does not contain any deltas and we didn't find any snapshots. - if (last_loaded_timestamp) { - epoch_history->emplace_back(wal_file.epoch_id, *last_loaded_timestamp); - } - repl_storage_state.epoch_.SetEpoch(std::move(wal_file.epoch_id)); - } try { auto info = LoadWal(wal_file.path, &indices_constraints, last_loaded_timestamp, vertices, edges, name_id_mapper, edge_count, config.salient.items); @@ -434,13 +432,28 @@ std::optional<RecoveryInfo> Recovery::RecoverData(std::string *uuid, Replication recovery_info.next_timestamp = std::max(recovery_info.next_timestamp, info.next_timestamp); recovery_info.last_commit_timestamp = info.last_commit_timestamp; + + if (recovery_info.next_timestamp != 0) { + last_loaded_timestamp.emplace(recovery_info.next_timestamp - 1); + } + + bool epoch_history_empty = epoch_history->empty(); + bool epoch_not_recorded = !epoch_history_empty && epoch_history->back().first != wal_file.epoch_id; + auto last_loaded_timestamp_value = last_loaded_timestamp.value_or(0); + + if (epoch_history_empty || epoch_not_recorded) { + epoch_history->emplace_back(std::string(wal_file.epoch_id), last_loaded_timestamp_value); + } + + auto last_epoch_updated = !epoch_history_empty && epoch_history->back().first == wal_file.epoch_id && + epoch_history->back().second < last_loaded_timestamp_value; + if (last_epoch_updated) { + epoch_history->back().second = last_loaded_timestamp_value; + } + } catch (const RecoveryFailure &e) { LOG_FATAL("Couldn't recover WAL deltas from {} because of: {}", wal_file.path, e.what()); } - - if (recovery_info.next_timestamp != 0) { - last_loaded_timestamp.emplace(recovery_info.next_timestamp - 1); - } } // The sequence number needs to be recovered even though `LoadWal` didn't // load any deltas from that file. @@ -456,7 +469,12 @@ std::optional<RecoveryInfo> Recovery::RecoverData(std::string *uuid, Replication memgraph::metrics::Measure(memgraph::metrics::SnapshotRecoveryLatency_us, std::chrono::duration_cast<std::chrono::microseconds>(timer.Elapsed()).count()); + spdlog::info("Set epoch id: {} with commit timestamp {}", std::string(repl_storage_state.epoch_.id()), + repl_storage_state.last_commit_timestamp_); + std::for_each(repl_storage_state.history.begin(), repl_storage_state.history.end(), [](auto &history) { + spdlog::info("epoch id: {} with commit timestamp {}", std::string(history.first), history.second); + }); return recovery_info; } diff --git a/src/storage/v2/inmemory/replication/recovery.cpp b/src/storage/v2/inmemory/replication/recovery.cpp index fe752bfd1..5f1182c75 100644 --- a/src/storage/v2/inmemory/replication/recovery.cpp +++ b/src/storage/v2/inmemory/replication/recovery.cpp @@ -233,7 +233,7 @@ std::vector<RecoveryStep> GetRecoverySteps(uint64_t replica_commit, utils::FileR } } - // In all cases, if we have a current wal file we need to use itW + // In all cases, if we have a current wal file we need to use it if (current_wal_seq_num) { // NOTE: File not handled directly, so no need to lock it recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num}); diff --git a/src/storage/v2/inmemory/storage.cpp b/src/storage/v2/inmemory/storage.cpp index bd8534673..1437524d6 100644 --- a/src/storage/v2/inmemory/storage.cpp +++ b/src/storage/v2/inmemory/storage.cpp @@ -109,6 +109,7 @@ InMemoryStorage::InMemoryStorage(Config config) timestamp_ = std::max(timestamp_, info->next_timestamp); if (info->last_commit_timestamp) { repl_storage_state_.last_commit_timestamp_ = *info->last_commit_timestamp; + spdlog::info("Recovering last commit timestamp {}", *info->last_commit_timestamp); } } } else if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED || diff --git a/src/storage/v2/replication/replication_client.cpp b/src/storage/v2/replication/replication_client.cpp index 1eb06bf10..bd2c2cc7d 100644 --- a/src/storage/v2/replication/replication_client.cpp +++ b/src/storage/v2/replication/replication_client.cpp @@ -54,15 +54,24 @@ void ReplicationStorageClient::UpdateReplicaState(Storage *storage, DatabaseAcce std::optional<uint64_t> branching_point; // different epoch id, replica was main + // In case there is no epoch transfer, and MAIN doesn't hold all the epochs as it could have been down and miss it + // we need then just to check commit timestamp if (replica.epoch_id != replStorageState.epoch_.id() && replica.current_commit_timestamp != kTimestampInitialId) { + spdlog::trace( + "REPLICA: epoch UUID: {} and last_commit_timestamp: {}; MAIN: epoch UUID {} and last_commit_timestamp {}", + std::string(replica.epoch_id), replica.current_commit_timestamp, std::string(replStorageState.epoch_.id()), + replStorageState.last_commit_timestamp_); auto const &history = replStorageState.history; const auto epoch_info_iter = std::find_if(history.crbegin(), history.crend(), [&](const auto &main_epoch_info) { return main_epoch_info.first == replica.epoch_id; }); // main didn't have that epoch, but why is here branching point if (epoch_info_iter == history.crend()) { + spdlog::info("Couldn't find epoch {} in MAIN, setting branching point", std::string(replica.epoch_id)); branching_point = 0; - } else if (epoch_info_iter->second != replica.current_commit_timestamp) { + } else if (epoch_info_iter->second < replica.current_commit_timestamp) { + spdlog::info("Found epoch {} on MAIN with last_commit_timestamp {}, REPLICA's last_commit_timestamp {}", + std::string(epoch_info_iter->first), epoch_info_iter->second, replica.current_commit_timestamp); branching_point = epoch_info_iter->second; } } @@ -192,9 +201,6 @@ void ReplicationStorageClient::StartTransactionReplication(const uint64_t curren } } -//////// AF: you can't finialize transaction replication if you are not replicating -/////// AF: if there is no stream or it is Defunct than we need to set replica in MAYBE_BEHIND -> is that even used -/////// AF: bool ReplicationStorageClient::FinalizeTransactionReplication(Storage *storage, DatabaseAccessProtector db_acc) { // We can only check the state because it guarantees to be only // valid during a single transaction replication (if the assumption @@ -259,6 +265,8 @@ void ReplicationStorageClient::RecoverReplica(uint64_t replica_commit, memgraph: spdlog::debug("Starting replica recovery"); auto *mem_storage = static_cast<InMemoryStorage *>(storage); + // TODO(antoniofilipovic): Can we get stuck here in while loop if replica commit timestamp is not updated when using + // only snapshot while (true) { auto file_locker = mem_storage->file_retainer_.AddLocker(); diff --git a/tests/e2e/high_availability/single_coordinator.py b/tests/e2e/high_availability/single_coordinator.py index ecf063092..2d81298b2 100644 --- a/tests/e2e/high_availability/single_coordinator.py +++ b/tests/e2e/high_availability/single_coordinator.py @@ -37,6 +37,9 @@ MEMGRAPH_INSTANCES_DESCRIPTION = { "TRACE", "--coordinator-server-port", "10011", + "--replication-restore-state-on-startup=true", + "--storage-recover-on-startup=false", + "--data-recovery-on-startup=false", ], "log_file": "instance_1.log", "data_directory": f"{TEMP_DIR}/instance_1", @@ -51,6 +54,9 @@ MEMGRAPH_INSTANCES_DESCRIPTION = { "TRACE", "--coordinator-server-port", "10012", + "--replication-restore-state-on-startup=true", + "--storage-recover-on-startup=false", + "--data-recovery-on-startup=false", ], "log_file": "instance_2.log", "data_directory": f"{TEMP_DIR}/instance_2", @@ -65,6 +71,9 @@ MEMGRAPH_INSTANCES_DESCRIPTION = { "TRACE", "--coordinator-server-port", "10013", + "--replication-restore-state-on-startup=true", + "--storage-recover-on-startup=false", + "--data-recovery-on-startup=false", ], "log_file": "instance_3.log", "data_directory": f"{TEMP_DIR}/instance_3", @@ -90,14 +99,794 @@ MEMGRAPH_INSTANCES_DESCRIPTION = { } -def test_replication_works_on_failover(): +@pytest.mark.parametrize("data_recovery", ["false", "true"]) +def test_replication_works_on_failover_replica_1_epoch_2_commits_away(data_recovery): # Goal of this test is to check the replication works after failover command. - # 1. We start all replicas, main and coordinator manually: we want to be able to kill them ourselves without relying on external tooling to kill processes. + # 1. We start all replicas, main and coordinator manually + # 2. We check that main has correct state + # 3. Create initial data on MAIN + # 4. Expect data to be copied on all replicas + # 5. Kill instance_1 (replica 1) + # 6. Create data on MAIN and expect to be copied to only one replica (instance_2) + # 7. Kill main + # 8. Instance_2 new MAIN + # 9. Create vertex on instance 2 + # 10. Start instance_1(it should have one commit on old epoch and new epoch with new commit shouldn't be replicated) + # 11. Expect data to be copied on instance_1 + # 12. Start old MAIN (instance_3) + # 13. Expect data to be copied to instance_3 + + temp_dir = tempfile.TemporaryDirectory().name + + MEMGRAPH_INNER_INSTANCES_DESCRIPTION = { + "instance_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7688", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10011", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_1.log", + "data_directory": f"{temp_dir}/instance_1", + "setup_queries": [], + }, + "instance_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7689", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10012", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_2.log", + "data_directory": f"{temp_dir}/instance_2", + "setup_queries": [], + }, + "instance_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7687", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10013", + "--replication-restore-state-on-startup", + "true", + "--data-recovery-on-startup", + f"{data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_3.log", + "data_directory": f"{temp_dir}/instance_3", + "setup_queries": [], + }, + "coordinator": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7690", + "--log-level=TRACE", + "--raft-server-id=1", + "--raft-server-port=10111", + ], + "log_file": "coordinator.log", + "setup_queries": [ + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001';", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002';", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003';", + "SET INSTANCE instance_3 TO MAIN", + ], + }, + } + + # 1 + interactive_mg_runner.start_all(MEMGRAPH_INNER_INSTANCES_DESCRIPTION) + + # 2 + main_cursor = connect(host="localhost", port=7687).cursor() + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + expected_data_on_main = [ + ( + "instance_1", + "127.0.0.1:10001", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ] + mg_sleep_and_assert_collection(expected_data_on_main, retrieve_data_show_replicas) + + # 3 + execute_and_fetch_all(main_cursor, "CREATE (:EpochVertex1 {prop:1});") + + # 4 + + instance_1_cursor = connect(host="localhost", port=7688).cursor() + instance_2_cursor = connect(host="localhost", port=7689).cursor() + + assert execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n);")[0][0] == 1 + assert execute_and_fetch_all(instance_2_cursor, "MATCH (n) RETURN count(n);")[0][0] == 1 + + # 5 + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + + # 6 + + with pytest.raises(Exception) as e: + execute_and_fetch_all(main_cursor, "CREATE (:EpochVertex1 {prop:2});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + assert execute_and_fetch_all(instance_2_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + + # 7 + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") + + # 8. + coord_cursor = connect(host="localhost", port=7690).cursor() + + def retrieve_data_show_instances(): + return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", False, "unknown"), + ("instance_2", "", "127.0.0.1:10012", True, "main"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 9 + + with pytest.raises(Exception) as e: + execute_and_fetch_all(instance_2_cursor, "CREATE (:Epoch3 {prop:3});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + # 10 + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + + new_expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", True, "replica"), + ("instance_2", "", "127.0.0.1:10012", True, "main"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ] + mg_sleep_and_assert(new_expected_data_on_coord, retrieve_data_show_instances) + + # 11 + instance_1_cursor = connect(host="localhost", port=7688).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(3, get_vertex_count) + + # 12 + + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") + + new_expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", True, "replica"), + ("instance_2", "", "127.0.0.1:10012", True, "main"), + ("instance_3", "", "127.0.0.1:10013", True, "replica"), + ] + mg_sleep_and_assert(new_expected_data_on_coord, retrieve_data_show_instances) + + # 13 + + instance_3_cursor = connect(host="localhost", port=7687).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(instance_3_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(3, get_vertex_count) + + +@pytest.mark.parametrize("data_recovery", ["false", "true"]) +def test_replication_works_on_failover_replica_2_epochs_more_commits_away(data_recovery): + # Goal of this test is to check the replication works after failover command if one + # instance missed couple of epochs but data is still available on one of the instances + + # 1. We start all replicas, main and coordinator manually + # 2. Main does commit + # 3. instance_2 down + # 4. Main commits more + # 5. Main down + # 6. Instance_1 new main + # 7. Instance 1 commits + # 8. Instance 4 gets data + # 9. Instance 1 dies + # 10. Instance 4 new main + # 11. Instance 4 commits + # 12. Instance 2 wakes up + # 13. Instance 2 gets data from old epochs + # 14. All other instances wake up + # 15. Everything is replicated + + temp_dir = tempfile.TemporaryDirectory().name + + MEMGRAPH_INNER_INSTANCES_DESCRIPTION = { + "instance_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7688", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10011", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_1.log", + "data_directory": f"{temp_dir}/instance_1", + "setup_queries": [], + }, + "instance_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7689", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10012", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_2.log", + "data_directory": f"{temp_dir}/instance_2", + "setup_queries": [], + }, + "instance_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7687", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10013", + "--replication-restore-state-on-startup", + "true", + "--data-recovery-on-startup", + f"{data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_3.log", + "data_directory": f"{temp_dir}/instance_3", + "setup_queries": [], + }, + "instance_4": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7691", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10014", + "--replication-restore-state-on-startup", + "true", + "--data-recovery-on-startup", + f"{data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_4.log", + "data_directory": f"{temp_dir}/instance_4", + "setup_queries": [], + }, + "coordinator": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7690", + "--log-level=TRACE", + "--raft-server-id=1", + "--raft-server-port=10111", + ], + "log_file": "coordinator.log", + "setup_queries": [ + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001';", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002';", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003';", + "REGISTER INSTANCE instance_4 ON '127.0.0.1:10014' WITH '127.0.0.1:10004';", + "SET INSTANCE instance_3 TO MAIN", + ], + }, + } + + # 1 + + interactive_mg_runner.start_all(MEMGRAPH_INNER_INSTANCES_DESCRIPTION) + + expected_data_on_main = [ + ( + "instance_1", + "127.0.0.1:10001", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ( + "instance_4", + "127.0.0.1:10004", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ] + + main_cursor = connect(host="localhost", port=7687).cursor() + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert_collection(expected_data_on_main, retrieve_data_show_replicas) + + # 2 + + execute_and_fetch_all(main_cursor, "CREATE (:EpochVertex1 {prop:1});") + execute_and_fetch_all(main_cursor, "CREATE (:EpochVertex1 {prop:2});") + + instance_1_cursor = connect(host="localhost", port=7688).cursor() + instance_2_cursor = connect(host="localhost", port=7689).cursor() + instance_4_cursor = connect(host="localhost", port=7691).cursor() + + assert execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + assert execute_and_fetch_all(instance_2_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + assert execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + + # 3 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_2") + + coord_cursor = connect(host="localhost", port=7690).cursor() + + def retrieve_data_show_instances(): + return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", True, "replica"), + ("instance_2", "", "127.0.0.1:10012", False, "unknown"), + ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 4 + + with pytest.raises(Exception) as e: + execute_and_fetch_all(main_cursor, "CREATE (:EpochVertex1 {prop:1});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + assert execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n);")[0][0] == 3 + assert execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n);")[0][0] == 3 + + # 5 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") + + # 6 + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", True, "main"), + ("instance_2", "", "127.0.0.1:10012", False, "unknown"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 7 + + with pytest.raises(Exception) as e: + execute_and_fetch_all(instance_1_cursor, "CREATE (:Epoch2Vertex {prop:1});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + # 8 + + assert execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n);")[0][0] == 4 + + # 9 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + + # 10 + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", False, "unknown"), + ("instance_2", "", "127.0.0.1:10012", False, "unknown"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("instance_4", "", "127.0.0.1:10014", True, "main"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 11 + + with pytest.raises(Exception) as e: + execute_and_fetch_all(instance_4_cursor, "CREATE (:Epoch3Vertex {prop:1});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + # 12 + + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_2") + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", False, "unknown"), + ("instance_2", "", "127.0.0.1:10012", True, "replica"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("instance_4", "", "127.0.0.1:10014", True, "main"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 13 + + instance_2_cursor = connect(host="localhost", port=7689).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(instance_2_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(5, get_vertex_count) + + # 14 + + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", True, "replica"), + ("instance_2", "", "127.0.0.1:10012", True, "replica"), + ("instance_3", "", "127.0.0.1:10013", True, "replica"), + ("instance_4", "", "127.0.0.1:10014", True, "main"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 15 + instance_1_cursor = connect(host="localhost", port=7688).cursor() + instance_4_cursor = connect(host="localhost", port=7691).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(5, get_vertex_count) + + def get_vertex_count(): + return execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(5, get_vertex_count) + + +@pytest.mark.parametrize("data_recovery", ["false"]) +def test_replication_forcefully_works_on_failover_replica_misses_epoch(data_recovery): + # TODO(antoniofilipovic) Test should pass when logic is added + # Goal of this test is to check the replication works forcefully if replica misses epoch + # 1. We start all replicas, main and coordinator manually + # 2. We check that main has correct state + # 3. Create initial data on MAIN + # 4. Expect data to be copied on all replicas + # 5. Kill instance_1 ( this one will miss complete epoch) + # 6. Kill main (instance_3) + # 7. Instance_2 or instance_4 new main + # 8. New main commits + # 9. Instance_2 down (not main) + # 10. instance_4 down + # 11. Instance 1 up (missed epoch) + # 12 Instance 1 new main + # 13 instance 2 up + # 14 Force data from instance 1 to instance 2 + + temp_dir = tempfile.TemporaryDirectory().name + + pass + + +@pytest.mark.parametrize("data_recovery", ["false", "true"]) +def test_replication_correct_replica_chosen_up_to_date_data(data_recovery): + # Goal of this test is to check that correct replica instance as new MAIN is chosen + # 1. We start all replicas, main and coordinator manually + # 2. We check that main has correct state + # 3. Create initial data on MAIN + # 4. Expect data to be copied on all replicas + # 5. Kill instance_1 ( this one will miss complete epoch) + # 6. Kill main (instance_3) + # 7. Instance_2 new MAIN + # 8. Instance_2 commits and replicates data + # 9. Instance_4 down (not main) + # 10. instance_2 down (MAIN), instance 1 up (missed epoch), + # instance 4 up (In this case we should always choose instance_4 because it has up-to-date data) + # 11 Instance 4 new main + # 12 instance_1 gets up-to-date data, instance_4 has all data + + temp_dir = tempfile.TemporaryDirectory().name + + MEMGRAPH_INNER_INSTANCES_DESCRIPTION = { + "instance_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7688", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10011", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_1.log", + "data_directory": f"{temp_dir}/instance_1", + "setup_queries": [], + }, + "instance_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7689", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10012", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_2.log", + "data_directory": f"{temp_dir}/instance_2", + "setup_queries": [], + }, + "instance_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7687", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10013", + "--replication-restore-state-on-startup", + "true", + "--data-recovery-on-startup", + f"{data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_3.log", + "data_directory": f"{temp_dir}/instance_3", + "setup_queries": [], + }, + "instance_4": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7691", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10014", + "--replication-restore-state-on-startup", + "true", + "--data-recovery-on-startup", + f"{data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_4.log", + "data_directory": f"{temp_dir}/instance_4", + "setup_queries": [], + }, + "coordinator": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7690", + "--log-level=TRACE", + "--raft-server-id=1", + "--raft-server-port=10111", + ], + "log_file": "coordinator.log", + "setup_queries": [ + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001';", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002';", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003';", + "REGISTER INSTANCE instance_4 ON '127.0.0.1:10014' WITH '127.0.0.1:10004';", + "SET INSTANCE instance_3 TO MAIN", + ], + }, + } + + # 1 + + interactive_mg_runner.start_all(MEMGRAPH_INNER_INSTANCES_DESCRIPTION) + + # 2 + + main_cursor = connect(host="localhost", port=7687).cursor() + expected_data_on_main = [ + ( + "instance_1", + "127.0.0.1:10001", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ( + "instance_4", + "127.0.0.1:10004", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ] + + main_cursor = connect(host="localhost", port=7687).cursor() + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert_collection(expected_data_on_main, retrieve_data_show_replicas) + + coord_cursor = connect(host="localhost", port=7690).cursor() + + def retrieve_data_show_instances(): + return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) + + # TODO(antoniofilipovic) Before fixing durability, if this is removed we also have an issue. Check after fix + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", True, "replica"), + ("instance_2", "", "127.0.0.1:10012", True, "replica"), + ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 3 + + execute_and_fetch_all(main_cursor, "CREATE (:Epoch1Vertex {prop:1});") + execute_and_fetch_all(main_cursor, "CREATE (:Epoch1Vertex {prop:2});") + + # 4 + instance_1_cursor = connect(host="localhost", port=7688).cursor() + instance_2_cursor = connect(host="localhost", port=7689).cursor() + instance_4_cursor = connect(host="localhost", port=7691).cursor() + + assert execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + assert execute_and_fetch_all(instance_2_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + assert execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + + # 5 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + + # 6 + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") + + # 7 + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", False, "unknown"), + ("instance_2", "", "127.0.0.1:10012", True, "main"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 8 + + with pytest.raises(Exception) as e: + execute_and_fetch_all(instance_2_cursor, "CREATE (:Epoch2Vertex {prop:1});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + def get_vertex_count(): + return execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(3, get_vertex_count) + + assert execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n);")[0][0] == 3 + + # 9 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_4") + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", False, "unknown"), + ("instance_2", "", "127.0.0.1:10012", True, "main"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("instance_4", "", "127.0.0.1:10014", False, "unknown"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 10 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_2") + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_4") + + # 11 + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), + ("instance_1", "", "127.0.0.1:10011", True, "replica"), + ("instance_2", "", "127.0.0.1:10012", False, "unknown"), + ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("instance_4", "", "127.0.0.1:10014", True, "main"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 12 + instance_1_cursor = connect(host="localhost", port=7688).cursor() + instance_4_cursor = connect(host="localhost", port=7691).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(3, get_vertex_count) + + def get_vertex_count(): + return execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(3, get_vertex_count) + + +def test_replication_works_on_failover_simple(): + # Goal of this test is to check the replication works after failover command. + # 1. We start all replicas, main and coordinator manually # 2. We check that main has correct state # 3. We kill main # 4. We check that coordinator and new main have correct state # 5. We insert one vertex on new main # 6. We check that vertex appears on new replica + # 7. We bring back main up + # 8. Expect data to be copied to main safe_execute(shutil.rmtree, TEMP_DIR) # 1 @@ -164,33 +953,48 @@ def test_replication_works_on_failover(): ] mg_sleep_and_assert_collection(expected_data_on_new_main, retrieve_data_show_replicas) + # 5 + with pytest.raises(Exception) as e: + execute_and_fetch_all(new_main_cursor, "CREATE ();") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + # 6 + alive_replica_cursor = connect(host="localhost", port=7689).cursor() + res = execute_and_fetch_all(alive_replica_cursor, "MATCH (n) RETURN count(n) as count;")[0][0] + assert res == 1, "Vertex should be replicated" + + # 7 interactive_mg_runner.start(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(new_main_cursor, "SHOW REPLICAS;"))) + + new_main_cursor = connect(host="localhost", port=7688).cursor() + expected_data_on_new_main = [ ( "instance_2", "127.0.0.1:10002", "sync", {"ts": 0, "behind": None, "status": "ready"}, - {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + {"memgraph": {"ts": 2, "behind": 0, "status": "ready"}}, ), ( "instance_3", "127.0.0.1:10003", "sync", {"ts": 0, "behind": None, "status": "ready"}, - {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + {"memgraph": {"ts": 2, "behind": 0, "status": "ready"}}, ), ] - mg_sleep_and_assert_collection(expected_data_on_new_main, retrieve_data_show_replicas) + mg_sleep_and_assert(expected_data_on_new_main, retrieve_data_show_replicas) - # 5 - execute_and_fetch_all(new_main_cursor, "CREATE ();") + # 8 + alive_main = connect(host="localhost", port=7687).cursor() - # 6 - alive_replica_cursror = connect(host="localhost", port=7689).cursor() - res = execute_and_fetch_all(alive_replica_cursror, "MATCH (n) RETURN count(n) as count;")[0][0] - assert res == 1, "Vertex should be replicated" - interactive_mg_runner.stop_all(MEMGRAPH_INSTANCES_DESCRIPTION) + def retrieve_vertices_count(): + return execute_and_fetch_all(alive_main, "MATCH (n) RETURN count(n) as count;")[0][0] + + mg_sleep_and_assert(1, retrieve_vertices_count) def test_replication_works_on_replica_instance_restart(): diff --git a/tests/e2e/replication/show_while_creating_invalid_state.py b/tests/e2e/replication/show_while_creating_invalid_state.py index be7cd2b54..963aad7fd 100644 --- a/tests/e2e/replication/show_while_creating_invalid_state.py +++ b/tests/e2e/replication/show_while_creating_invalid_state.py @@ -923,7 +923,7 @@ def test_replication_role_recovery(connection): "--log-level=TRACE", "--replication-restore-state-on-startup", "true", - "--storage-recover-on-startup", + "--data-recovery-on-startup", "false", ], "log_file": "replica.log", @@ -934,7 +934,7 @@ def test_replication_role_recovery(connection): "--bolt-port", "7687", "--log-level=TRACE", - "--storage-recover-on-startup=true", + "--data-recovery-on-startup=true", "--replication-restore-state-on-startup=true", ], "log_file": "main.log", @@ -1105,7 +1105,7 @@ def test_basic_recovery_when_replica_is_kill_when_main_is_down(): "--bolt-port", "7687", "--log-level=TRACE", - "--storage-recover-on-startup=true", + "--data-recovery-on-startup=true", "--replication-restore-state-on-startup=true", ], "log_file": "main.log", @@ -1201,7 +1201,7 @@ def test_async_replication_when_main_is_killed(): "data_directory": f"{data_directory_replica.name}", }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [], "data_directory": f"{data_directory_main.name}", @@ -1284,7 +1284,7 @@ def test_sync_replication_when_main_is_killed(): "data_directory": f"{data_directory_replica.name}", }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [], "data_directory": f"{data_directory_main.name}", @@ -1340,7 +1340,7 @@ def test_attempt_to_write_data_on_main_when_async_replica_is_down(): "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [ "REGISTER REPLICA async_replica1 ASYNC TO '127.0.0.1:10001';", @@ -1443,7 +1443,7 @@ def test_attempt_to_write_data_on_main_when_sync_replica_is_down(connection): "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup", "true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup", "true"], "log_file": "main.log", # need to do it manually "setup_queries": [], @@ -1572,7 +1572,7 @@ def test_attempt_to_create_indexes_on_main_when_async_replica_is_down(): "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [ "REGISTER REPLICA async_replica1 ASYNC TO '127.0.0.1:10001';", @@ -1673,7 +1673,7 @@ def test_attempt_to_create_indexes_on_main_when_sync_replica_is_down(connection) "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", # Need to do it manually "setup_queries": [], @@ -1818,7 +1818,7 @@ def test_trigger_on_create_before_commit_with_offline_sync_replica(connection): "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", # Need to do it manually since we kill replica "setup_queries": [], @@ -1937,7 +1937,7 @@ def test_trigger_on_update_before_commit_with_offline_sync_replica(connection): "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [], }, @@ -2060,7 +2060,7 @@ def test_trigger_on_delete_before_commit_with_offline_sync_replica(connection): "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [], }, @@ -2187,7 +2187,7 @@ def test_trigger_on_create_before_and_after_commit_with_offline_sync_replica(con "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [], }, @@ -2310,7 +2310,7 @@ def test_triggers_on_create_before_commit_with_offline_sync_replica(connection): "setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"], }, "main": { - "args": ["--bolt-port", "7687", "--log-level=TRACE", "--storage-recover-on-startup=true"], + "args": ["--bolt-port", "7687", "--log-level=TRACE", "--data-recovery-on-startup=true"], "log_file": "main.log", "setup_queries": [], }, diff --git a/tests/unit/coordination_utils.cpp b/tests/unit/coordination_utils.cpp index 1346dce2c..2a595c19f 100644 --- a/tests/unit/coordination_utils.cpp +++ b/tests/unit/coordination_utils.cpp @@ -67,8 +67,8 @@ TEST_F(CoordinationUtils, MemgraphDbHistorySimple) { auto [instance_name, latest_epoch, latest_commit_timestamp] = instance.ChooseMostUpToDateInstance(instance_database_histories); ASSERT_TRUE(instance_name == "instance_1" || instance_name == "instance_2" || instance_name == "instance_3"); - ASSERT_TRUE(*latest_epoch == db_histories.back().first); - ASSERT_TRUE(*latest_commit_timestamp == db_histories.back().second); + ASSERT_TRUE(latest_epoch == db_histories.back().first); + ASSERT_TRUE(latest_commit_timestamp == db_histories.back().second); } TEST_F(CoordinationUtils, MemgraphDbHistoryLastEpochDifferent) { @@ -121,8 +121,8 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryLastEpochDifferent) { instance.ChooseMostUpToDateInstance(instance_database_histories); ASSERT_TRUE(instance_name == "instance_3"); - ASSERT_TRUE(*latest_epoch == db_histories.back().first); - ASSERT_TRUE(*latest_commit_timestamp == db_histories.back().second); + ASSERT_TRUE(latest_epoch == db_histories.back().first); + ASSERT_TRUE(latest_commit_timestamp == db_histories.back().second); } TEST_F(CoordinationUtils, MemgraphDbHistoryOneInstanceAheadFewEpochs) { @@ -179,8 +179,8 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryOneInstanceAheadFewEpochs) { instance.ChooseMostUpToDateInstance(instance_database_histories); ASSERT_TRUE(instance_name == "instance_3"); - ASSERT_TRUE(*latest_epoch == db_histories_longest.back().first); - ASSERT_TRUE(*latest_commit_timestamp == db_histories_longest.back().second); + ASSERT_TRUE(latest_epoch == db_histories_longest.back().first); + ASSERT_TRUE(latest_commit_timestamp == db_histories_longest.back().second); } TEST_F(CoordinationUtils, MemgraphDbHistoryInstancesHistoryDiverged) { @@ -241,6 +241,6 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryInstancesHistoryDiverged) { instance.ChooseMostUpToDateInstance(instance_database_histories); ASSERT_TRUE(instance_name == "instance_3"); - ASSERT_TRUE(*latest_epoch == std::string(newest_different_epoch)); - ASSERT_TRUE(*latest_commit_timestamp == oldest_commit_timestamp); + ASSERT_TRUE(latest_epoch == std::string(newest_different_epoch)); + ASSERT_TRUE(latest_commit_timestamp == oldest_commit_timestamp); } From 822183b62d94970ca88f8ba45ace12f232a864e6 Mon Sep 17 00:00:00 2001 From: Andi <andi8647@gmail.com> Date: Mon, 4 Mar 2024 08:24:18 +0100 Subject: [PATCH 08/18] Support failure of coordinators (#1728) --- src/coordination/CMakeLists.txt | 3 + src/coordination/coordinator_client.cpp | 13 +- .../coordinator_cluster_state.cpp | 166 +++++++ src/coordination/coordinator_instance.cpp | 413 +++++++++++------- src/coordination/coordinator_state.cpp | 12 +- .../coordinator_state_machine.cpp | 183 ++++++-- .../coordination/coordinator_client.hpp | 9 +- .../coordination/coordinator_config.hpp | 46 +- .../coordination/coordinator_exceptions.hpp | 11 + .../coordination/coordinator_instance.hpp | 28 +- .../coordination/coordinator_state.hpp | 10 +- .../include/coordination/instance_status.hpp | 2 +- .../include/coordination/raft_state.hpp | 30 +- ...gister_main_replica_coordinator_status.hpp | 17 +- .../coordination/replication_instance.hpp | 25 +- .../nuraft/coordinator_cluster_state.hpp | 82 ++++ .../nuraft/coordinator_state_machine.hpp | 42 +- .../include/nuraft/raft_log_action.hpp | 53 +++ src/coordination/raft_state.cpp | 140 +++++- src/coordination/replication_instance.cpp | 30 +- src/dbms/coordinator_handler.cpp | 16 +- src/dbms/coordinator_handler.hpp | 9 +- src/dbms/inmemory/replication_handlers.cpp | 3 +- src/io/network/endpoint.cpp | 28 +- src/io/network/endpoint.hpp | 12 +- src/query/interpreter.cpp | 50 ++- src/query/interpreter.hpp | 13 +- src/replication_coordination_glue/mode.hpp | 25 ++ src/utils/CMakeLists.txt | 1 + src/utils/string.hpp | 7 + src/utils/uuid.cpp | 4 +- src/utils/uuid.hpp | 1 + .../coord_cluster_registration.py | 214 ++++++--- tests/e2e/high_availability/coordinator.py | 8 +- .../disable_writing_on_main_after_restart.py | 24 +- .../high_availability/distributed_coords.py | 108 ++++- .../not_replicate_from_old_main.py | 19 +- .../high_availability/single_coordinator.py | 252 ++++++----- tests/e2e/mg_utils.py | 22 + 39 files changed, 1557 insertions(+), 574 deletions(-) create mode 100644 src/coordination/coordinator_cluster_state.cpp create mode 100644 src/coordination/include/nuraft/coordinator_cluster_state.hpp create mode 100644 src/coordination/include/nuraft/raft_log_action.hpp diff --git a/src/coordination/CMakeLists.txt b/src/coordination/CMakeLists.txt index 936d7a5c2..3e293e2e7 100644 --- a/src/coordination/CMakeLists.txt +++ b/src/coordination/CMakeLists.txt @@ -16,6 +16,8 @@ target_sources(mg-coordination include/coordination/raft_state.hpp include/coordination/rpc_errors.hpp + include/nuraft/raft_log_action.hpp + include/nuraft/coordinator_cluster_state.hpp include/nuraft/coordinator_log_store.hpp include/nuraft/coordinator_state_machine.hpp include/nuraft/coordinator_state_manager.hpp @@ -33,6 +35,7 @@ target_sources(mg-coordination coordinator_log_store.cpp coordinator_state_machine.cpp coordinator_state_manager.cpp + coordinator_cluster_state.cpp ) target_include_directories(mg-coordination PUBLIC include) diff --git a/src/coordination/coordinator_client.cpp b/src/coordination/coordinator_client.cpp index b4f82b60c..bc7f42eaa 100644 --- a/src/coordination/coordinator_client.cpp +++ b/src/coordination/coordinator_client.cpp @@ -41,7 +41,9 @@ CoordinatorClient::CoordinatorClient(CoordinatorInstance *coord_instance, Coordi fail_cb_{std::move(fail_cb)} {} auto CoordinatorClient::InstanceName() const -> std::string { return config_.instance_name; } -auto CoordinatorClient::SocketAddress() const -> std::string { return rpc_client_.Endpoint().SocketAddress(); } + +auto CoordinatorClient::CoordinatorSocketAddress() const -> std::string { return config_.CoordinatorSocketAddress(); } +auto CoordinatorClient::ReplicationSocketAddress() const -> std::string { return config_.ReplicationSocketAddress(); } auto CoordinatorClient::InstanceDownTimeoutSec() const -> std::chrono::seconds { return config_.instance_down_timeout_sec; @@ -64,7 +66,7 @@ void CoordinatorClient::StartFrequentCheck() { [this, instance_name = config_.instance_name] { try { spdlog::trace("Sending frequent heartbeat to machine {} on {}", instance_name, - rpc_client_.Endpoint().SocketAddress()); + config_.CoordinatorSocketAddress()); { // NOTE: This is intentionally scoped so that stream lock could get released. auto stream{rpc_client_.Stream<memgraph::replication_coordination_glue::FrequentHeartbeatRpc>()}; stream.AwaitResponse(); @@ -117,7 +119,7 @@ auto CoordinatorClient::DemoteToReplica() const -> bool { return false; } -auto CoordinatorClient::SendSwapMainUUIDRpc(const utils::UUID &uuid) const -> bool { +auto CoordinatorClient::SendSwapMainUUIDRpc(utils::UUID const &uuid) const -> bool { try { auto stream{rpc_client_.Stream<replication_coordination_glue::SwapMainUUIDRpc>(uuid)}; if (!stream.AwaitResponse().success) { @@ -131,9 +133,10 @@ auto CoordinatorClient::SendSwapMainUUIDRpc(const utils::UUID &uuid) const -> bo return false; } -auto CoordinatorClient::SendUnregisterReplicaRpc(std::string const &instance_name) const -> bool { +auto CoordinatorClient::SendUnregisterReplicaRpc(std::string_view instance_name) const -> bool { try { - auto stream{rpc_client_.Stream<UnregisterReplicaRpc>(instance_name)}; + auto stream{rpc_client_.Stream<UnregisterReplicaRpc>( + std::string(instance_name))}; // TODO: (andi) Try to change to stream string_view and do just one copy later if (!stream.AwaitResponse().success) { spdlog::error("Failed to receive successful RPC response for unregistering replica!"); return false; diff --git a/src/coordination/coordinator_cluster_state.cpp b/src/coordination/coordinator_cluster_state.cpp new file mode 100644 index 000000000..60f0ca622 --- /dev/null +++ b/src/coordination/coordinator_cluster_state.cpp @@ -0,0 +1,166 @@ +// 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 +// 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. + +#ifdef MG_ENTERPRISE + +#include "nuraft/coordinator_cluster_state.hpp" +#include "utils/logging.hpp" + +#include <shared_mutex> + +namespace memgraph::coordination { + +using replication_coordination_glue::ReplicationRole; + +CoordinatorClusterState::CoordinatorClusterState(CoordinatorClusterState const &other) + : instance_roles_{other.instance_roles_} {} + +CoordinatorClusterState &CoordinatorClusterState::operator=(CoordinatorClusterState const &other) { + if (this == &other) { + return *this; + } + instance_roles_ = other.instance_roles_; + return *this; +} + +CoordinatorClusterState::CoordinatorClusterState(CoordinatorClusterState &&other) noexcept + : instance_roles_{std::move(other.instance_roles_)} {} + +CoordinatorClusterState &CoordinatorClusterState::operator=(CoordinatorClusterState &&other) noexcept { + if (this == &other) { + return *this; + } + instance_roles_ = std::move(other.instance_roles_); + return *this; +} + +auto CoordinatorClusterState::MainExists() const -> bool { + auto lock = std::shared_lock{log_lock_}; + return std::ranges::any_of(instance_roles_, + [](auto const &entry) { return entry.second.role == ReplicationRole::MAIN; }); +} + +auto CoordinatorClusterState::IsMain(std::string_view instance_name) const -> bool { + auto lock = std::shared_lock{log_lock_}; + auto const it = instance_roles_.find(instance_name); + return it != instance_roles_.end() && it->second.role == ReplicationRole::MAIN; +} + +auto CoordinatorClusterState::IsReplica(std::string_view instance_name) const -> bool { + auto lock = std::shared_lock{log_lock_}; + auto const it = instance_roles_.find(instance_name); + return it != instance_roles_.end() && it->second.role == ReplicationRole::REPLICA; +} + +auto CoordinatorClusterState::InsertInstance(std::string_view instance_name, ReplicationRole role) -> void { + auto lock = std::unique_lock{log_lock_}; + instance_roles_[instance_name.data()].role = role; +} + +auto CoordinatorClusterState::DoAction(TRaftLog log_entry, RaftLogAction log_action) -> void { + auto lock = std::unique_lock{log_lock_}; + switch (log_action) { + case RaftLogAction::REGISTER_REPLICATION_INSTANCE: { + auto const &config = std::get<CoordinatorClientConfig>(log_entry); + instance_roles_[config.instance_name] = InstanceState{config, ReplicationRole::REPLICA}; + break; + } + case RaftLogAction::UNREGISTER_REPLICATION_INSTANCE: { + auto const instance_name = std::get<std::string>(log_entry); + instance_roles_.erase(instance_name); + break; + } + case RaftLogAction::SET_INSTANCE_AS_MAIN: { + auto const instance_name = std::get<std::string>(log_entry); + auto it = instance_roles_.find(instance_name); + MG_ASSERT(it != instance_roles_.end(), "Instance does not exist as part of raft state!"); + it->second.role = ReplicationRole::MAIN; + break; + } + case RaftLogAction::SET_INSTANCE_AS_REPLICA: { + auto const instance_name = std::get<std::string>(log_entry); + auto it = instance_roles_.find(instance_name); + MG_ASSERT(it != instance_roles_.end(), "Instance does not exist as part of raft state!"); + it->second.role = ReplicationRole::REPLICA; + break; + } + case RaftLogAction::UPDATE_UUID: { + uuid_ = std::get<utils::UUID>(log_entry); + break; + } + } +} + +// TODO: (andi) Improve based on Gareth's comments +auto CoordinatorClusterState::Serialize(ptr<buffer> &data) -> void { + auto lock = std::shared_lock{log_lock_}; + auto const role_to_string = [](auto const &role) -> std::string_view { + switch (role) { + case ReplicationRole::MAIN: + return "main"; + case ReplicationRole::REPLICA: + return "replica"; + } + }; + + auto const entry_to_string = [&role_to_string](auto const &entry) { + return fmt::format("{}_{}", entry.first, role_to_string(entry.second.role)); + }; + + auto instances_str_view = instance_roles_ | ranges::views::transform(entry_to_string); + uint32_t size = + std::accumulate(instances_str_view.begin(), instances_str_view.end(), 0, + [](uint32_t acc, auto const &entry) { return acc + sizeof(uint32_t) + entry.size(); }); + + data = buffer::alloc(size); + buffer_serializer bs(data); + std::for_each(instances_str_view.begin(), instances_str_view.end(), [&bs](auto const &entry) { bs.put_str(entry); }); +} + +auto CoordinatorClusterState::Deserialize(buffer &data) -> CoordinatorClusterState { + auto const str_to_role = [](auto const &str) -> ReplicationRole { + if (str == "main") { + return ReplicationRole::MAIN; + } + return ReplicationRole::REPLICA; + }; + + CoordinatorClusterState cluster_state; + buffer_serializer bs(data); + while (bs.size() > 0) { + auto const entry = bs.get_str(); + auto const first_dash = entry.find('_'); + auto const instance_name = entry.substr(0, first_dash); + auto const role_str = entry.substr(first_dash + 1); + cluster_state.InsertInstance(instance_name, str_to_role(role_str)); + } + return cluster_state; +} + +auto CoordinatorClusterState::GetInstances() const -> std::vector<InstanceState> { + auto lock = std::shared_lock{log_lock_}; + return instance_roles_ | ranges::views::values | ranges::to<std::vector<InstanceState>>; +} + +auto CoordinatorClusterState::GetUUID() const -> utils::UUID { return uuid_; } + +auto CoordinatorClusterState::FindCurrentMainInstanceName() const -> std::optional<std::string> { + auto lock = std::shared_lock{log_lock_}; + auto const it = std::ranges::find_if(instance_roles_, + [](auto const &entry) { return entry.second.role == ReplicationRole::MAIN; }); + if (it == instance_roles_.end()) { + return {}; + } + return it->first; +} + +} // namespace memgraph::coordination +#endif diff --git a/src/coordination/coordinator_instance.cpp b/src/coordination/coordinator_instance.cpp index daf1d7138..afbcaa7d8 100644 --- a/src/coordination/coordinator_instance.cpp +++ b/src/coordination/coordinator_instance.cpp @@ -32,8 +32,42 @@ using nuraft::srv_config; CoordinatorInstance::CoordinatorInstance() : raft_state_(RaftState::MakeRaftState( - [this] { std::ranges::for_each(repl_instances_, &ReplicationInstance::StartFrequentCheck); }, - [this] { std::ranges::for_each(repl_instances_, &ReplicationInstance::StopFrequentCheck); })) { + [this]() { + spdlog::info("Leader changed, starting all replication instances!"); + auto const instances = raft_state_.GetInstances(); + auto replicas = instances | ranges::views::filter([](auto const &instance) { + return instance.role == ReplicationRole::REPLICA; + }); + + std::ranges::for_each(replicas, [this](auto &replica) { + spdlog::info("Starting replication instance {}", replica.config.instance_name); + repl_instances_.emplace_back(this, replica.config, client_succ_cb_, client_fail_cb_, + &CoordinatorInstance::ReplicaSuccessCallback, + &CoordinatorInstance::ReplicaFailCallback); + }); + + auto main = instances | ranges::views::filter( + [](auto const &instance) { return instance.role == ReplicationRole::MAIN; }); + + // TODO: (andi) Add support for this + // MG_ASSERT(std::ranges::distance(main) == 1, "There should be exactly one main instance"); + + std::ranges::for_each(main, [this](auto &main_instance) { + spdlog::info("Starting main instance {}", main_instance.config.instance_name); + repl_instances_.emplace_back(this, main_instance.config, client_succ_cb_, client_fail_cb_, + &CoordinatorInstance::MainSuccessCallback, + &CoordinatorInstance::MainFailCallback); + }); + + std::ranges::for_each(repl_instances_, [this](auto &instance) { + instance.SetNewMainUUID(raft_state_.GetUUID()); // TODO: (andi) Rename + instance.StartFrequentCheck(); + }); + }, + [this]() { + spdlog::info("Leader changed, stopping all replication instances!"); + repl_instances_.clear(); + })) { client_succ_cb_ = [](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { auto lock = std::unique_lock{self->coord_instance_lock_}; auto &repl_instance = self->FindReplicationInstance(repl_instance_name); @@ -59,75 +93,98 @@ auto CoordinatorInstance::FindReplicationInstance(std::string_view replication_i } auto CoordinatorInstance::ShowInstances() const -> std::vector<InstanceStatus> { - auto const coord_instances = raft_state_.GetAllCoordinators(); - - auto const stringify_repl_role = [](ReplicationInstance const &instance) -> std::string { - if (!instance.IsAlive()) return "unknown"; - if (instance.IsMain()) return "main"; - return "replica"; - }; - - auto const repl_instance_to_status = [&stringify_repl_role](ReplicationInstance const &instance) -> InstanceStatus { - return {.instance_name = instance.InstanceName(), - .coord_socket_address = instance.SocketAddress(), - .cluster_role = stringify_repl_role(instance), - .is_alive = instance.IsAlive()}; - }; - auto const coord_instance_to_status = [](ptr<srv_config> const &instance) -> InstanceStatus { return {.instance_name = "coordinator_" + std::to_string(instance->get_id()), .raft_socket_address = instance->get_endpoint(), .cluster_role = "coordinator", - .is_alive = true}; // TODO: (andi) Get this info from RAFT and test it or when we will move - // CoordinatorState to every instance, we can be smarter about this using our RPC. + .health = "unknown"}; // TODO: (andi) Get this info from RAFT and test it or when we will move + // CoordinatorState to every instance, we can be smarter about this using our RPC. }; - auto instances_status = utils::fmap(coord_instance_to_status, coord_instances); - { - auto lock = std::shared_lock{coord_instance_lock_}; - std::ranges::transform(repl_instances_, std::back_inserter(instances_status), repl_instance_to_status); + auto instances_status = utils::fmap(coord_instance_to_status, raft_state_.GetAllCoordinators()); + + if (raft_state_.IsLeader()) { + auto const stringify_repl_role = [this](ReplicationInstance const &instance) -> std::string { + if (!instance.IsAlive()) return "unknown"; + if (raft_state_.IsMain(instance.InstanceName())) return "main"; + return "replica"; + }; + + auto const stringify_repl_health = [](ReplicationInstance const &instance) -> std::string { + return instance.IsAlive() ? "up" : "down"; + }; + + auto process_repl_instance_as_leader = + [&stringify_repl_role, &stringify_repl_health](ReplicationInstance const &instance) -> InstanceStatus { + return {.instance_name = instance.InstanceName(), + .coord_socket_address = instance.CoordinatorSocketAddress(), + .cluster_role = stringify_repl_role(instance), + .health = stringify_repl_health(instance)}; + }; + + { + auto lock = std::shared_lock{coord_instance_lock_}; + std::ranges::transform(repl_instances_, std::back_inserter(instances_status), process_repl_instance_as_leader); + } + } else { + auto const stringify_repl_role = [](ReplicationRole role) -> std::string { + return role == ReplicationRole::MAIN ? "main" : "replica"; + }; + + // TODO: (andi) Add capability that followers can also return socket addresses + auto process_repl_instance_as_follower = [&stringify_repl_role](auto const &instance) -> InstanceStatus { + return {.instance_name = instance.config.instance_name, + .cluster_role = stringify_repl_role(instance.role), + .health = "unknown"}; + }; + + std::ranges::transform(raft_state_.GetInstances(), std::back_inserter(instances_status), + process_repl_instance_as_follower); } return instances_status; } auto CoordinatorInstance::TryFailover() -> void { - auto alive_replicas = repl_instances_ | ranges::views::filter(&ReplicationInstance::IsReplica) | - ranges::views::filter(&ReplicationInstance::IsAlive); + auto const is_replica = [this](ReplicationInstance const &instance) { return IsReplica(instance.InstanceName()); }; + + auto alive_replicas = + repl_instances_ | ranges::views::filter(is_replica) | ranges::views::filter(&ReplicationInstance::IsAlive); if (ranges::empty(alive_replicas)) { spdlog::warn("Failover failed since all replicas are down!"); return; } - // for each DB in instance we get one DatabaseHistory - using DatabaseHistories = replication_coordination_glue::DatabaseHistories; - std::vector<std::pair<std::string, DatabaseHistories>> instance_database_histories; + if (!raft_state_.RequestLeadership()) { + spdlog::error("Failover failed since the instance is not the leader!"); + return; + } - bool success{true}; - std::ranges::for_each(alive_replicas, [&success, &instance_database_histories](ReplicationInstance &replica) { - if (!success) { - return; - } - auto res = replica.GetClient().SendGetInstanceTimestampsRpc(); - if (res.HasError()) { - spdlog::error("Could get per db history data for instance {}", replica.InstanceName()); - success = false; - return; - } - instance_database_histories.emplace_back(replica.InstanceName(), std::move(res.GetValue())); - }); + auto const get_ts = [](ReplicationInstance &replica) { return replica.GetClient().SendGetInstanceTimestampsRpc(); }; - if (!success) { + auto maybe_instance_db_histories = alive_replicas | ranges::views::transform(get_ts) | ranges::to<std::vector>(); + + auto const ts_has_error = [](auto const &res) -> bool { return res.HasError(); }; + + if (std::ranges::any_of(maybe_instance_db_histories, ts_has_error)) { spdlog::error("Aborting failover as at least one instance didn't provide per database history."); return; } + auto transform_to_pairs = ranges::views::transform([](auto const &zipped) { + auto &[replica, res] = zipped; + return std::make_pair(replica.InstanceName(), res.GetValue()); + }); + + auto instance_db_histories = + ranges::views::zip(alive_replicas, maybe_instance_db_histories) | transform_to_pairs | ranges::to<std::vector>(); + auto [most_up_to_date_instance, latest_epoch, latest_commit_timestamp] = - ChooseMostUpToDateInstance(instance_database_histories); + ChooseMostUpToDateInstance(instance_db_histories); spdlog::trace("The most up to date instance is {} with epoch {} and {} latest commit timestamp", - most_up_to_date_instance, latest_epoch, latest_commit_timestamp); + most_up_to_date_instance, latest_epoch, latest_commit_timestamp); // NOLINT auto *new_main = &FindReplicationInstance(most_up_to_date_instance); @@ -139,16 +196,17 @@ auto CoordinatorInstance::TryFailover() -> void { }; auto const new_main_uuid = utils::UUID{}; + + auto const failed_to_swap = [&new_main_uuid](ReplicationInstance &instance) { + return !instance.SendSwapAndUpdateUUID(new_main_uuid); + }; + // If for some replicas swap fails, for others on successful ping we will revert back on next change // or we will do failover first again and then it will be consistent again - for (auto &other_replica_instance : alive_replicas | ranges::views::filter(is_not_new_main)) { - if (!other_replica_instance.SendSwapAndUpdateUUID(new_main_uuid)) { - spdlog::error(fmt::format("Failed to swap uuid for instance {} which is alive, aborting failover", - other_replica_instance.InstanceName())); - return; - } + if (std::ranges::any_of(alive_replicas | ranges::views::filter(is_not_new_main), failed_to_swap)) { + spdlog::error("Failed to swap uuid for all instances"); + return; } - auto repl_clients_info = repl_instances_ | ranges::views::filter(is_not_new_main) | ranges::views::transform(&ReplicationInstance::ReplicationClientInfo) | ranges::to<ReplicationClientsInfo>(); @@ -158,23 +216,36 @@ auto CoordinatorInstance::TryFailover() -> void { spdlog::warn("Failover failed since promoting replica to main failed!"); return; } - // TODO: (andi) This should be replicated across all coordinator instances with Raft log - SetMainUUID(new_main_uuid); + + if (!raft_state_.AppendUpdateUUIDLog(new_main_uuid)) { + return; + } + + auto const new_main_instance_name = new_main->InstanceName(); + + if (!raft_state_.AppendSetInstanceAsMainLog(new_main_instance_name)) { + return; + } + spdlog::info("Failover successful! Instance {} promoted to main.", new_main->InstanceName()); } -// TODO: (andi) Make sure you cannot put coordinator instance to the main -auto CoordinatorInstance::SetReplicationInstanceToMain(std::string instance_name) +auto CoordinatorInstance::SetReplicationInstanceToMain(std::string_view instance_name) -> SetInstanceToMainCoordinatorStatus { auto lock = std::lock_guard{coord_instance_lock_}; - if (std::ranges::any_of(repl_instances_, &ReplicationInstance::IsMain)) { + if (raft_state_.MainExists()) { return SetInstanceToMainCoordinatorStatus::MAIN_ALREADY_EXISTS; } + if (!raft_state_.RequestLeadership()) { + return SetInstanceToMainCoordinatorStatus::NOT_LEADER; + } + auto const is_new_main = [&instance_name](ReplicationInstance const &instance) { return instance.InstanceName() == instance_name; }; + auto new_main = std::ranges::find_if(repl_instances_, is_new_main); if (new_main == repl_instances_.end()) { @@ -192,99 +263,149 @@ auto CoordinatorInstance::SetReplicationInstanceToMain(std::string instance_name auto const new_main_uuid = utils::UUID{}; - for (auto &other_instance : repl_instances_ | ranges::views::filter(is_not_new_main)) { - if (!other_instance.SendSwapAndUpdateUUID(new_main_uuid)) { - spdlog::error( - fmt::format("Failed to swap uuid for instance {}, aborting failover", other_instance.InstanceName())); - return SetInstanceToMainCoordinatorStatus::SWAP_UUID_FAILED; - } + auto const failed_to_swap = [&new_main_uuid](ReplicationInstance &instance) { + return !instance.SendSwapAndUpdateUUID(new_main_uuid); + }; + + if (std::ranges::any_of(repl_instances_ | ranges::views::filter(is_not_new_main), failed_to_swap)) { + spdlog::error("Failed to swap uuid for all instances"); + return SetInstanceToMainCoordinatorStatus::SWAP_UUID_FAILED; } - ReplicationClientsInfo repl_clients_info; - repl_clients_info.reserve(repl_instances_.size() - 1); - std::ranges::transform(repl_instances_ | ranges::views::filter(is_not_new_main), - std::back_inserter(repl_clients_info), &ReplicationInstance::ReplicationClientInfo); + auto repl_clients_info = repl_instances_ | ranges::views::filter(is_not_new_main) | + ranges::views::transform(&ReplicationInstance::ReplicationClientInfo) | + ranges::to<ReplicationClientsInfo>(); if (!new_main->PromoteToMain(new_main_uuid, std::move(repl_clients_info), &CoordinatorInstance::MainSuccessCallback, &CoordinatorInstance::MainFailCallback)) { return SetInstanceToMainCoordinatorStatus::COULD_NOT_PROMOTE_TO_MAIN; } - // TODO: (andi) This should be replicated across all coordinator instances with Raft log - SetMainUUID(new_main_uuid); - spdlog::info("Instance {} promoted to main", instance_name); + if (!raft_state_.AppendUpdateUUIDLog(new_main_uuid)) { + return SetInstanceToMainCoordinatorStatus::RAFT_LOG_ERROR; + } + + if (!raft_state_.AppendSetInstanceAsMainLog(instance_name)) { + return SetInstanceToMainCoordinatorStatus::RAFT_LOG_ERROR; + } + + spdlog::info("Instance {} promoted to main on leader", instance_name); return SetInstanceToMainCoordinatorStatus::SUCCESS; } -auto CoordinatorInstance::RegisterReplicationInstance(CoordinatorClientConfig config) +auto CoordinatorInstance::RegisterReplicationInstance(CoordinatorClientConfig const &config) -> RegisterInstanceCoordinatorStatus { auto lock = std::lock_guard{coord_instance_lock_}; - auto instance_name = config.instance_name; - - auto const name_matches = [&instance_name](ReplicationInstance const &instance) { - return instance.InstanceName() == instance_name; - }; - - if (std::ranges::any_of(repl_instances_, name_matches)) { + if (std::ranges::any_of(repl_instances_, [instance_name = config.instance_name](ReplicationInstance const &instance) { + return instance.InstanceName() == instance_name; + })) { return RegisterInstanceCoordinatorStatus::NAME_EXISTS; } - auto const socket_address_matches = [&config](ReplicationInstance const &instance) { - return instance.SocketAddress() == config.SocketAddress(); - }; + if (std::ranges::any_of(repl_instances_, [&config](ReplicationInstance const &instance) { + return instance.CoordinatorSocketAddress() == config.CoordinatorSocketAddress(); + })) { + return RegisterInstanceCoordinatorStatus::COORD_ENDPOINT_EXISTS; + } - if (std::ranges::any_of(repl_instances_, socket_address_matches)) { - return RegisterInstanceCoordinatorStatus::ENDPOINT_EXISTS; + if (std::ranges::any_of(repl_instances_, [&config](ReplicationInstance const &instance) { + return instance.ReplicationSocketAddress() == config.ReplicationSocketAddress(); + })) { + return RegisterInstanceCoordinatorStatus::REPL_ENDPOINT_EXISTS; } if (!raft_state_.RequestLeadership()) { return RegisterInstanceCoordinatorStatus::NOT_LEADER; } - auto const res = raft_state_.AppendRegisterReplicationInstance(instance_name); - if (!res->get_accepted()) { - spdlog::error( - "Failed to accept request for registering instance {}. Most likely the reason is that the instance is not " - "the " - "leader.", - config.instance_name); - return RegisterInstanceCoordinatorStatus::RAFT_COULD_NOT_ACCEPT; - } + auto *new_instance = &repl_instances_.emplace_back(this, config, client_succ_cb_, client_fail_cb_, + &CoordinatorInstance::ReplicaSuccessCallback, + &CoordinatorInstance::ReplicaFailCallback); - spdlog::info("Request for registering instance {} accepted", instance_name); - try { - repl_instances_.emplace_back(this, std::move(config), client_succ_cb_, client_fail_cb_, - &CoordinatorInstance::ReplicaSuccessCallback, - &CoordinatorInstance::ReplicaFailCallback); - } catch (CoordinatorRegisterInstanceException const &) { + if (!new_instance->SendDemoteToReplicaRpc()) { + spdlog::error("Failed to send demote to replica rpc for instance {}", config.instance_name); + repl_instances_.pop_back(); return RegisterInstanceCoordinatorStatus::RPC_FAILED; } - if (res->get_result_code() != nuraft::cmd_result_code::OK) { - spdlog::error("Failed to register instance {} with error code {}", instance_name, res->get_result_code()); - return RegisterInstanceCoordinatorStatus::RAFT_COULD_NOT_APPEND; + if (!raft_state_.AppendRegisterReplicationInstanceLog(config)) { + return RegisterInstanceCoordinatorStatus::RAFT_LOG_ERROR; } - spdlog::info("Instance {} registered", instance_name); + new_instance->StartFrequentCheck(); + + spdlog::info("Instance {} registered", config.instance_name); return RegisterInstanceCoordinatorStatus::SUCCESS; } +auto CoordinatorInstance::UnregisterReplicationInstance(std::string_view instance_name) + -> UnregisterInstanceCoordinatorStatus { + auto lock = std::lock_guard{coord_instance_lock_}; + + if (!raft_state_.RequestLeadership()) { + return UnregisterInstanceCoordinatorStatus::NOT_LEADER; + } + + auto const name_matches = [&instance_name](ReplicationInstance const &instance) { + return instance.InstanceName() == instance_name; + }; + + auto inst_to_remove = std::ranges::find_if(repl_instances_, name_matches); + if (inst_to_remove == repl_instances_.end()) { + return UnregisterInstanceCoordinatorStatus::NO_INSTANCE_WITH_NAME; + } + + // TODO: (andi) Change so that RaftLogState is the central place for asking who is main... + + auto const is_main = [this](ReplicationInstance const &instance) { return IsMain(instance.InstanceName()); }; + + if (is_main(*inst_to_remove) && inst_to_remove->IsAlive()) { + return UnregisterInstanceCoordinatorStatus::IS_MAIN; + } + + inst_to_remove->StopFrequentCheck(); + + auto curr_main = std::ranges::find_if(repl_instances_, is_main); + + if (curr_main != repl_instances_.end() && curr_main->IsAlive()) { + if (!curr_main->SendUnregisterReplicaRpc(instance_name)) { + inst_to_remove->StartFrequentCheck(); + return UnregisterInstanceCoordinatorStatus::RPC_FAILED; + } + } + + std::erase_if(repl_instances_, name_matches); + + if (!raft_state_.AppendUnregisterReplicationInstanceLog(instance_name)) { + return UnregisterInstanceCoordinatorStatus::RAFT_LOG_ERROR; + } + + return UnregisterInstanceCoordinatorStatus::SUCCESS; +} + +auto CoordinatorInstance::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, + std::string_view raft_address) -> void { + raft_state_.AddCoordinatorInstance(raft_server_id, raft_port, raft_address); +} + void CoordinatorInstance::MainFailCallback(std::string_view repl_instance_name) { + spdlog::trace("Instance {} performing main fail callback", repl_instance_name); auto &repl_instance = FindReplicationInstance(repl_instance_name); repl_instance.OnFailPing(); const auto &repl_instance_uuid = repl_instance.GetMainUUID(); - MG_ASSERT(repl_instance_uuid.has_value(), "Instance must have uuid set"); + MG_ASSERT(repl_instance_uuid.has_value(), "Replication instance must have uuid set"); - if (!repl_instance.IsAlive() && GetMainUUID() == repl_instance_uuid.value()) { + // NOLINTNEXTLINE + if (!repl_instance.IsAlive() && raft_state_.GetUUID() == repl_instance_uuid.value()) { spdlog::info("Cluster without main instance, trying automatic failover"); TryFailover(); } } void CoordinatorInstance::MainSuccessCallback(std::string_view repl_instance_name) { - auto &repl_instance = FindReplicationInstance(repl_instance_name); spdlog::trace("Instance {} performing main successful callback", repl_instance_name); + auto &repl_instance = FindReplicationInstance(repl_instance_name); if (repl_instance.IsAlive()) { repl_instance.OnSuccessPing(); @@ -294,8 +415,8 @@ void CoordinatorInstance::MainSuccessCallback(std::string_view repl_instance_nam const auto &repl_instance_uuid = repl_instance.GetMainUUID(); MG_ASSERT(repl_instance_uuid.has_value(), "Instance must have uuid set."); - auto const curr_main_uuid = GetMainUUID(); - if (curr_main_uuid == repl_instance_uuid.value()) { + // NOLINTNEXTLINE + if (raft_state_.GetUUID() == repl_instance_uuid.value()) { if (!repl_instance.EnableWritingOnMain()) { spdlog::error("Failed to enable writing on main instance {}", repl_instance_name); return; @@ -305,6 +426,12 @@ void CoordinatorInstance::MainSuccessCallback(std::string_view repl_instance_nam return; } + if (!raft_state_.RequestLeadership()) { + spdlog::error("Demoting main instance {} to replica failed since the instance is not the leader!", + repl_instance_name); + return; + } + if (repl_instance.DemoteToReplica(&CoordinatorInstance::ReplicaSuccessCallback, &CoordinatorInstance::ReplicaFailCallback)) { repl_instance.OnSuccessPing(); @@ -314,24 +441,29 @@ void CoordinatorInstance::MainSuccessCallback(std::string_view repl_instance_nam return; } - if (!repl_instance.SendSwapAndUpdateUUID(curr_main_uuid)) { - spdlog::error(fmt::format("Failed to swap uuid for demoted main instance {}", repl_instance.InstanceName())); + if (!repl_instance.SendSwapAndUpdateUUID(raft_state_.GetUUID())) { + spdlog::error("Failed to swap uuid for demoted main instance {}", repl_instance_name); + return; + } + + if (!raft_state_.AppendSetInstanceAsReplicaLog(repl_instance_name)) { return; } } void CoordinatorInstance::ReplicaSuccessCallback(std::string_view repl_instance_name) { + spdlog::trace("Instance {} performing replica successful callback", repl_instance_name); auto &repl_instance = FindReplicationInstance(repl_instance_name); - if (!repl_instance.IsReplica()) { + + if (!IsReplica(repl_instance_name)) { spdlog::error("Aborting replica callback since instance {} is not replica anymore", repl_instance_name); return; } - spdlog::trace("Instance {} performing replica successful callback", repl_instance_name); // We need to get replicas UUID from time to time to ensure replica is listening to correct main // and that it didn't go down for less time than we could notice // We need to get id of main replica is listening to // and swap if necessary - if (!repl_instance.EnsureReplicaHasCorrectMainUUID(GetMainUUID())) { + if (!repl_instance.EnsureReplicaHasCorrectMainUUID(raft_state_.GetUUID())) { spdlog::error("Failed to swap uuid for replica instance {} which is alive", repl_instance.InstanceName()); return; } @@ -340,57 +472,19 @@ void CoordinatorInstance::ReplicaSuccessCallback(std::string_view repl_instance_ } void CoordinatorInstance::ReplicaFailCallback(std::string_view repl_instance_name) { + spdlog::trace("Instance {} performing replica failure callback", repl_instance_name); auto &repl_instance = FindReplicationInstance(repl_instance_name); - if (!repl_instance.IsReplica()) { + + if (!IsReplica(repl_instance_name)) { spdlog::error("Aborting replica fail callback since instance {} is not replica anymore", repl_instance_name); return; } - spdlog::trace("Instance {} performing replica failure callback", repl_instance_name); + repl_instance.OnFailPing(); } -auto CoordinatorInstance::UnregisterReplicationInstance(std::string instance_name) - -> UnregisterInstanceCoordinatorStatus { - auto lock = std::lock_guard{coord_instance_lock_}; - - auto const name_matches = [&instance_name](ReplicationInstance const &instance) { - return instance.InstanceName() == instance_name; - }; - - auto inst_to_remove = std::ranges::find_if(repl_instances_, name_matches); - if (inst_to_remove == repl_instances_.end()) { - return UnregisterInstanceCoordinatorStatus::NO_INSTANCE_WITH_NAME; - } - - if (inst_to_remove->IsMain() && inst_to_remove->IsAlive()) { - return UnregisterInstanceCoordinatorStatus::IS_MAIN; - } - - inst_to_remove->StopFrequentCheck(); - auto curr_main = std::ranges::find_if(repl_instances_, &ReplicationInstance::IsMain); - MG_ASSERT(curr_main != repl_instances_.end(), "There must be a main instance when unregistering a replica"); - if (!curr_main->SendUnregisterReplicaRpc(instance_name)) { - inst_to_remove->StartFrequentCheck(); - return UnregisterInstanceCoordinatorStatus::RPC_FAILED; - } - std::erase_if(repl_instances_, name_matches); - - return UnregisterInstanceCoordinatorStatus::SUCCESS; -} - -auto CoordinatorInstance::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) - -> void { - raft_state_.AddCoordinatorInstance(raft_server_id, raft_port, std::move(raft_address)); -} - -auto CoordinatorInstance::GetMainUUID() const -> utils::UUID { return main_uuid_; } - -// TODO: (andi) Add to the RAFT log. -auto CoordinatorInstance::SetMainUUID(utils::UUID new_uuid) -> void { main_uuid_ = new_uuid; } - -auto CoordinatorInstance::ChooseMostUpToDateInstance( - const std::vector<std::pair<std::string, replication_coordination_glue::DatabaseHistories>> - &instance_database_histories) -> NewMainRes { +auto CoordinatorInstance::ChooseMostUpToDateInstance(std::span<InstanceNameDbHistories> instance_database_histories) + -> NewMainRes { std::optional<NewMainRes> new_main_res; std::for_each( instance_database_histories.begin(), instance_database_histories.end(), @@ -456,5 +550,14 @@ auto CoordinatorInstance::ChooseMostUpToDateInstance( return std::move(*new_main_res); } + +auto CoordinatorInstance::IsMain(std::string_view instance_name) const -> bool { + return raft_state_.IsMain(instance_name); +} + +auto CoordinatorInstance::IsReplica(std::string_view instance_name) const -> bool { + return raft_state_.IsReplica(instance_name); +} + } // namespace memgraph::coordination #endif diff --git a/src/coordination/coordinator_state.cpp b/src/coordination/coordinator_state.cpp index 28d6c604e..f429cd5a7 100644 --- a/src/coordination/coordinator_state.cpp +++ b/src/coordination/coordinator_state.cpp @@ -41,7 +41,7 @@ CoordinatorState::CoordinatorState() { } } -auto CoordinatorState::RegisterReplicationInstance(CoordinatorClientConfig config) +auto CoordinatorState::RegisterReplicationInstance(CoordinatorClientConfig const &config) -> RegisterInstanceCoordinatorStatus { MG_ASSERT(std::holds_alternative<CoordinatorInstance>(data_), "Coordinator cannot register replica since variant holds wrong alternative"); @@ -56,7 +56,8 @@ auto CoordinatorState::RegisterReplicationInstance(CoordinatorClientConfig confi data_); } -auto CoordinatorState::UnregisterReplicationInstance(std::string instance_name) -> UnregisterInstanceCoordinatorStatus { +auto CoordinatorState::UnregisterReplicationInstance(std::string_view instance_name) + -> UnregisterInstanceCoordinatorStatus { MG_ASSERT(std::holds_alternative<CoordinatorInstance>(data_), "Coordinator cannot unregister instance since variant holds wrong alternative"); @@ -70,7 +71,8 @@ auto CoordinatorState::UnregisterReplicationInstance(std::string instance_name) data_); } -auto CoordinatorState::SetReplicationInstanceToMain(std::string instance_name) -> SetInstanceToMainCoordinatorStatus { +auto CoordinatorState::SetReplicationInstanceToMain(std::string_view instance_name) + -> SetInstanceToMainCoordinatorStatus { MG_ASSERT(std::holds_alternative<CoordinatorInstance>(data_), "Coordinator cannot register replica since variant holds wrong alternative"); @@ -96,8 +98,8 @@ auto CoordinatorState::GetCoordinatorServer() const -> CoordinatorServer & { return *std::get<CoordinatorMainReplicaData>(data_).coordinator_server_; } -auto CoordinatorState::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) - -> void { +auto CoordinatorState::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, + std::string_view raft_address) -> void { MG_ASSERT(std::holds_alternative<CoordinatorInstance>(data_), "Coordinator cannot register replica since variant holds wrong alternative"); return std::get<CoordinatorInstance>(data_).AddCoordinatorInstance(raft_server_id, raft_port, raft_address); diff --git a/src/coordination/coordinator_state_machine.cpp b/src/coordination/coordinator_state_machine.cpp index b939bd304..6e2986ecf 100644 --- a/src/coordination/coordinator_state_machine.cpp +++ b/src/coordination/coordinator_state_machine.cpp @@ -12,37 +12,94 @@ #ifdef MG_ENTERPRISE #include "nuraft/coordinator_state_machine.hpp" +#include "utils/logging.hpp" namespace memgraph::coordination { -auto CoordinatorStateMachine::EncodeRegisterReplicationInstance(const std::string &name) -> ptr<buffer> { - std::string str_log = name + "_replica"; - ptr<buffer> log = buffer::alloc(sizeof(uint32_t) + str_log.size()); - buffer_serializer bs(log); - bs.put_str(str_log); - return log; +auto CoordinatorStateMachine::FindCurrentMainInstanceName() const -> std::optional<std::string> { + return cluster_state_.FindCurrentMainInstanceName(); } -auto CoordinatorStateMachine::DecodeRegisterReplicationInstance(buffer &data) -> std::string { +auto CoordinatorStateMachine::MainExists() const -> bool { return cluster_state_.MainExists(); } + +auto CoordinatorStateMachine::IsMain(std::string_view instance_name) const -> bool { + return cluster_state_.IsMain(instance_name); +} + +auto CoordinatorStateMachine::IsReplica(std::string_view instance_name) const -> bool { + return cluster_state_.IsReplica(instance_name); +} + +auto CoordinatorStateMachine::CreateLog(std::string_view log) -> ptr<buffer> { + ptr<buffer> log_buf = buffer::alloc(sizeof(uint32_t) + log.size()); + buffer_serializer bs(log_buf); + bs.put_str(log.data()); + return log_buf; +} + +auto CoordinatorStateMachine::SerializeRegisterInstance(CoordinatorClientConfig const &config) -> ptr<buffer> { + auto const str_log = fmt::format("{}*register", config.ToString()); + return CreateLog(str_log); +} + +auto CoordinatorStateMachine::SerializeUnregisterInstance(std::string_view instance_name) -> ptr<buffer> { + auto const str_log = fmt::format("{}*unregister", instance_name); + return CreateLog(str_log); +} + +auto CoordinatorStateMachine::SerializeSetInstanceAsMain(std::string_view instance_name) -> ptr<buffer> { + auto const str_log = fmt::format("{}*promote", instance_name); + return CreateLog(str_log); +} + +auto CoordinatorStateMachine::SerializeSetInstanceAsReplica(std::string_view instance_name) -> ptr<buffer> { + auto const str_log = fmt::format("{}*demote", instance_name); + return CreateLog(str_log); +} + +auto CoordinatorStateMachine::SerializeUpdateUUID(utils::UUID const &uuid) -> ptr<buffer> { + auto const str_log = fmt::format("{}*update_uuid", nlohmann::json{{"uuid", uuid}}.dump()); + return CreateLog(str_log); +} + +auto CoordinatorStateMachine::DecodeLog(buffer &data) -> std::pair<TRaftLog, RaftLogAction> { buffer_serializer bs(data); - return bs.get_str(); + + auto const log_str = bs.get_str(); + auto const sep = log_str.find('*'); + auto const action = log_str.substr(sep + 1); + auto const info = log_str.substr(0, sep); + + if (action == "register") { + return {CoordinatorClientConfig::FromString(info), RaftLogAction::REGISTER_REPLICATION_INSTANCE}; + } + if (action == "unregister") { + return {info, RaftLogAction::UNREGISTER_REPLICATION_INSTANCE}; + } + if (action == "promote") { + return {info, RaftLogAction::SET_INSTANCE_AS_MAIN}; + } + if (action == "demote") { + return {info, RaftLogAction::SET_INSTANCE_AS_REPLICA}; + } + if (action == "update_uuid") { + auto const json = nlohmann::json::parse(info); + return {json.at("uuid").get<utils::UUID>(), RaftLogAction::UPDATE_UUID}; + } + + throw std::runtime_error("Unknown action"); } -auto CoordinatorStateMachine::pre_commit(ulong const log_idx, buffer &data) -> ptr<buffer> { - buffer_serializer bs(data); - std::string str = bs.get_str(); - - spdlog::info("pre_commit {} : {}", log_idx, str); - return nullptr; -} +auto CoordinatorStateMachine::pre_commit(ulong const /*log_idx*/, buffer & /*data*/) -> ptr<buffer> { return nullptr; } auto CoordinatorStateMachine::commit(ulong const log_idx, buffer &data) -> ptr<buffer> { buffer_serializer bs(data); - std::string str = bs.get_str(); - spdlog::info("commit {} : {}", log_idx, str); + auto const [parsed_data, log_action] = DecodeLog(data); + cluster_state_.DoAction(parsed_data, log_action); last_committed_idx_ = log_idx; + // TODO: (andi) Don't return nullptr return nullptr; } @@ -51,61 +108,95 @@ auto CoordinatorStateMachine::commit_config(ulong const log_idx, ptr<cluster_con } auto CoordinatorStateMachine::rollback(ulong const log_idx, buffer &data) -> void { - buffer_serializer bs(data); - std::string str = bs.get_str(); - - spdlog::info("rollback {} : {}", log_idx, str); + // NOTE: Nothing since we don't do anything in pre_commit } -auto CoordinatorStateMachine::read_logical_snp_obj(snapshot & /*snapshot*/, void *& /*user_snp_ctx*/, ulong /*obj_id*/, +auto CoordinatorStateMachine::read_logical_snp_obj(snapshot &snapshot, void *& /*user_snp_ctx*/, ulong obj_id, ptr<buffer> &data_out, bool &is_last_obj) -> int { - // Put dummy data. - data_out = buffer::alloc(sizeof(int32)); - buffer_serializer bs(data_out); - bs.put_i32(0); + spdlog::info("read logical snapshot object, obj_id: {}", obj_id); + ptr<SnapshotCtx> ctx = nullptr; + { + auto ll = std::lock_guard{snapshots_lock_}; + auto entry = snapshots_.find(snapshot.get_last_log_idx()); + if (entry == snapshots_.end()) { + data_out = nullptr; + is_last_obj = true; + return 0; + } + ctx = entry->second; + } + ctx->cluster_state_.Serialize(data_out); is_last_obj = true; return 0; } -auto CoordinatorStateMachine::save_logical_snp_obj(snapshot &s, ulong &obj_id, buffer & /*data*/, bool /*is_first_obj*/, - bool /*is_last_obj*/) -> void { - spdlog::info("save snapshot {} term {} object ID", s.get_last_log_idx(), s.get_last_log_term(), obj_id); - // Request next object. - obj_id++; +auto CoordinatorStateMachine::save_logical_snp_obj(snapshot &snapshot, ulong &obj_id, buffer &data, bool is_first_obj, + bool is_last_obj) -> void { + spdlog::info("save logical snapshot object, obj_id: {}, is_first_obj: {}, is_last_obj: {}", obj_id, is_first_obj, + is_last_obj); + + buffer_serializer bs(data); + auto cluster_state = CoordinatorClusterState::Deserialize(data); + + { + auto ll = std::lock_guard{snapshots_lock_}; + auto entry = snapshots_.find(snapshot.get_last_log_idx()); + DMG_ASSERT(entry != snapshots_.end()); + entry->second->cluster_state_ = cluster_state; + } } auto CoordinatorStateMachine::apply_snapshot(snapshot &s) -> bool { - spdlog::info("apply snapshot {} term {}", s.get_last_log_idx(), s.get_last_log_term()); - { - auto lock = std::lock_guard{last_snapshot_lock_}; - ptr<buffer> snp_buf = s.serialize(); - last_snapshot_ = snapshot::deserialize(*snp_buf); - } + auto ll = std::lock_guard{snapshots_lock_}; + + auto entry = snapshots_.find(s.get_last_log_idx()); + if (entry == snapshots_.end()) return false; + + cluster_state_ = entry->second->cluster_state_; return true; } auto CoordinatorStateMachine::free_user_snp_ctx(void *&user_snp_ctx) -> void {} auto CoordinatorStateMachine::last_snapshot() -> ptr<snapshot> { - auto lock = std::lock_guard{last_snapshot_lock_}; - return last_snapshot_; + auto ll = std::lock_guard{snapshots_lock_}; + auto entry = snapshots_.rbegin(); + if (entry == snapshots_.rend()) return nullptr; + + ptr<SnapshotCtx> ctx = entry->second; + return ctx->snapshot_; } auto CoordinatorStateMachine::last_commit_index() -> ulong { return last_committed_idx_; } auto CoordinatorStateMachine::create_snapshot(snapshot &s, async_result<bool>::handler_type &when_done) -> void { - spdlog::info("create snapshot {} term {}", s.get_last_log_idx(), s.get_last_log_term()); - // Clone snapshot from `s`. - { - auto lock = std::lock_guard{last_snapshot_lock_}; - ptr<buffer> snp_buf = s.serialize(); - last_snapshot_ = snapshot::deserialize(*snp_buf); - } + ptr<buffer> snp_buf = s.serialize(); + ptr<snapshot> ss = snapshot::deserialize(*snp_buf); + create_snapshot_internal(ss); + ptr<std::exception> except(nullptr); bool ret = true; when_done(ret, except); } +auto CoordinatorStateMachine::create_snapshot_internal(ptr<snapshot> snapshot) -> void { + auto ll = std::lock_guard{snapshots_lock_}; + + auto ctx = cs_new<SnapshotCtx>(snapshot, cluster_state_); + snapshots_[snapshot->get_last_log_idx()] = ctx; + + constexpr int MAX_SNAPSHOTS = 3; + while (snapshots_.size() > MAX_SNAPSHOTS) { + snapshots_.erase(snapshots_.begin()); + } +} + +auto CoordinatorStateMachine::GetInstances() const -> std::vector<InstanceState> { + return cluster_state_.GetInstances(); +} + +auto CoordinatorStateMachine::GetUUID() const -> utils::UUID { return cluster_state_.GetUUID(); } + } // namespace memgraph::coordination #endif diff --git a/src/coordination/include/coordination/coordinator_client.hpp b/src/coordination/include/coordination/coordinator_client.hpp index 994c78d18..5d4795f81 100644 --- a/src/coordination/include/coordination/coordinator_client.hpp +++ b/src/coordination/include/coordination/coordinator_client.hpp @@ -46,16 +46,17 @@ class CoordinatorClient { void ResumeFrequentCheck(); auto InstanceName() const -> std::string; - auto SocketAddress() const -> std::string; + auto CoordinatorSocketAddress() const -> std::string; + auto ReplicationSocketAddress() const -> std::string; [[nodiscard]] auto DemoteToReplica() const -> bool; - auto SendPromoteReplicaToMainRpc(const utils::UUID &uuid, ReplicationClientsInfo replication_clients_info) const + auto SendPromoteReplicaToMainRpc(utils::UUID const &uuid, ReplicationClientsInfo replication_clients_info) const -> bool; - auto SendSwapMainUUIDRpc(const utils::UUID &uuid) const -> bool; + auto SendSwapMainUUIDRpc(utils::UUID const &uuid) const -> bool; - auto SendUnregisterReplicaRpc(std::string const &instance_name) const -> bool; + auto SendUnregisterReplicaRpc(std::string_view instance_name) const -> bool; auto SendEnableWritingOnMainRpc() const -> bool; diff --git a/src/coordination/include/coordination/coordinator_config.hpp b/src/coordination/include/coordination/coordinator_config.hpp index df7a5f94f..b0d7118ff 100644 --- a/src/coordination/include/coordination/coordinator_config.hpp +++ b/src/coordination/include/coordination/coordinator_config.hpp @@ -14,16 +14,20 @@ #ifdef MG_ENTERPRISE #include "replication_coordination_glue/mode.hpp" +#include "utils/string.hpp" #include <chrono> #include <cstdint> #include <optional> #include <string> +#include <fmt/format.h> + namespace memgraph::coordination { inline constexpr auto *kDefaultReplicationServerIp = "0.0.0.0"; +// TODO: (andi) JSON serialization for RAFT log. struct CoordinatorClientConfig { std::string instance_name; std::string ip_address; @@ -32,14 +36,35 @@ struct CoordinatorClientConfig { std::chrono::seconds instance_down_timeout_sec{5}; std::chrono::seconds instance_get_uuid_frequency_sec{10}; - auto SocketAddress() const -> std::string { return ip_address + ":" + std::to_string(port); } + auto CoordinatorSocketAddress() const -> std::string { return fmt::format("{}:{}", ip_address, port); } + auto ReplicationSocketAddress() const -> std::string { + return fmt::format("{}:{}", replication_client_info.replication_ip_address, + replication_client_info.replication_port); + } struct ReplicationClientInfo { + // TODO: (andi) Do we even need here instance_name for this struct? std::string instance_name; replication_coordination_glue::ReplicationMode replication_mode{}; std::string replication_ip_address; uint16_t replication_port{}; + auto ToString() const -> std::string { + return fmt::format("{}#{}#{}#{}", instance_name, replication_ip_address, replication_port, + replication_coordination_glue::ReplicationModeToString(replication_mode)); + } + + // TODO: (andi) How can I make use of monadic parsers here? + static auto FromString(std::string_view log) -> ReplicationClientInfo { + ReplicationClientInfo replication_client_info; + auto splitted = utils::Split(log, "#"); + replication_client_info.instance_name = splitted[0]; + replication_client_info.replication_ip_address = splitted[1]; + replication_client_info.replication_port = std::stoi(splitted[2]); + replication_client_info.replication_mode = replication_coordination_glue::ReplicationModeFromString(splitted[3]); + return replication_client_info; + } + friend bool operator==(ReplicationClientInfo const &, ReplicationClientInfo const &) = default; }; @@ -54,6 +79,25 @@ struct CoordinatorClientConfig { std::optional<SSL> ssl; + auto ToString() const -> std::string { + return fmt::format("{}|{}|{}|{}|{}|{}|{}", instance_name, ip_address, port, + instance_health_check_frequency_sec.count(), instance_down_timeout_sec.count(), + instance_get_uuid_frequency_sec.count(), replication_client_info.ToString()); + } + + static auto FromString(std::string_view log) -> CoordinatorClientConfig { + CoordinatorClientConfig config; + auto splitted = utils::Split(log, "|"); + config.instance_name = splitted[0]; + config.ip_address = splitted[1]; + config.port = std::stoi(splitted[2]); + config.instance_health_check_frequency_sec = std::chrono::seconds(std::stoi(splitted[3])); + config.instance_down_timeout_sec = std::chrono::seconds(std::stoi(splitted[4])); + config.instance_get_uuid_frequency_sec = std::chrono::seconds(std::stoi(splitted[5])); + config.replication_client_info = ReplicationClientInfo::FromString(splitted[6]); + return config; + } + friend bool operator==(CoordinatorClientConfig const &, CoordinatorClientConfig const &) = default; }; diff --git a/src/coordination/include/coordination/coordinator_exceptions.hpp b/src/coordination/include/coordination/coordinator_exceptions.hpp index 59a2e89d8..7a967f80b 100644 --- a/src/coordination/include/coordination/coordinator_exceptions.hpp +++ b/src/coordination/include/coordination/coordinator_exceptions.hpp @@ -83,5 +83,16 @@ class RaftCouldNotParseFlagsException final : public utils::BasicException { SPECIALIZE_GET_EXCEPTION_NAME(RaftCouldNotParseFlagsException) }; +class InvalidRaftLogActionException final : public utils::BasicException { + public: + explicit InvalidRaftLogActionException(std::string_view what) noexcept : BasicException(what) {} + + template <class... Args> + explicit InvalidRaftLogActionException(fmt::format_string<Args...> fmt, Args &&...args) noexcept + : InvalidRaftLogActionException(fmt::format(fmt, std::forward<Args>(args)...)) {} + + SPECIALIZE_GET_EXCEPTION_NAME(InvalidRaftLogActionException) +}; + } // namespace memgraph::coordination #endif diff --git a/src/coordination/include/coordination/coordinator_instance.hpp b/src/coordination/include/coordination/coordinator_instance.hpp index ee5a6fb6e..10549f468 100644 --- a/src/coordination/include/coordination/coordinator_instance.hpp +++ b/src/coordination/include/coordination/coordinator_instance.hpp @@ -37,20 +37,25 @@ class CoordinatorInstance { public: CoordinatorInstance(); - [[nodiscard]] auto RegisterReplicationInstance(CoordinatorClientConfig config) -> RegisterInstanceCoordinatorStatus; - [[nodiscard]] auto UnregisterReplicationInstance(std::string instance_name) -> UnregisterInstanceCoordinatorStatus; + [[nodiscard]] auto RegisterReplicationInstance(CoordinatorClientConfig const &config) + -> RegisterInstanceCoordinatorStatus; + [[nodiscard]] auto UnregisterReplicationInstance(std::string_view instance_name) + -> UnregisterInstanceCoordinatorStatus; - [[nodiscard]] auto SetReplicationInstanceToMain(std::string instance_name) -> SetInstanceToMainCoordinatorStatus; + [[nodiscard]] auto SetReplicationInstanceToMain(std::string_view instance_name) -> SetInstanceToMainCoordinatorStatus; auto ShowInstances() const -> std::vector<InstanceStatus>; auto TryFailover() -> void; - auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) -> void; + auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string_view raft_address) -> void; - auto GetMainUUID() const -> utils::UUID; + static auto ChooseMostUpToDateInstance(std::span<InstanceNameDbHistories> histories) -> NewMainRes; - auto SetMainUUID(utils::UUID new_uuid) -> void; + private: + HealthCheckClientCallback client_succ_cb_, client_fail_cb_; + + auto OnRaftCommitCallback(TRaftLog const &log_entry, RaftLogAction log_action) -> void; auto FindReplicationInstance(std::string_view replication_instance_name) -> ReplicationInstance &; @@ -62,17 +67,14 @@ class CoordinatorInstance { void ReplicaFailCallback(std::string_view); - static auto ChooseMostUpToDateInstance(const std::vector<InstanceNameDbHistories> &) -> NewMainRes; + auto IsMain(std::string_view instance_name) const -> bool; + auto IsReplica(std::string_view instance_name) const -> bool; - private: - HealthCheckClientCallback client_succ_cb_, client_fail_cb_; - - // NOTE: Must be std::list because we rely on pointer stability + // NOTE: Must be std::list because we rely on pointer stability. + // Leader and followers should both have same view on repl_instances_ std::list<ReplicationInstance> repl_instances_; mutable utils::ResourceLock coord_instance_lock_{}; - utils::UUID main_uuid_; - RaftState raft_state_; }; diff --git a/src/coordination/include/coordination/coordinator_state.hpp b/src/coordination/include/coordination/coordinator_state.hpp index 256af66f9..400c36940 100644 --- a/src/coordination/include/coordination/coordinator_state.hpp +++ b/src/coordination/include/coordination/coordinator_state.hpp @@ -33,14 +33,16 @@ class CoordinatorState { CoordinatorState(CoordinatorState &&) noexcept = delete; CoordinatorState &operator=(CoordinatorState &&) noexcept = delete; - [[nodiscard]] auto RegisterReplicationInstance(CoordinatorClientConfig config) -> RegisterInstanceCoordinatorStatus; - [[nodiscard]] auto UnregisterReplicationInstance(std::string instance_name) -> UnregisterInstanceCoordinatorStatus; + [[nodiscard]] auto RegisterReplicationInstance(CoordinatorClientConfig const &config) + -> RegisterInstanceCoordinatorStatus; + [[nodiscard]] auto UnregisterReplicationInstance(std::string_view instance_name) + -> UnregisterInstanceCoordinatorStatus; - [[nodiscard]] auto SetReplicationInstanceToMain(std::string instance_name) -> SetInstanceToMainCoordinatorStatus; + [[nodiscard]] auto SetReplicationInstanceToMain(std::string_view instance_name) -> SetInstanceToMainCoordinatorStatus; auto ShowInstances() const -> std::vector<InstanceStatus>; - auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) -> void; + auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string_view raft_address) -> void; // NOTE: The client code must check that the server exists before calling this method. auto GetCoordinatorServer() const -> CoordinatorServer &; diff --git a/src/coordination/include/coordination/instance_status.hpp b/src/coordination/include/coordination/instance_status.hpp index 492410061..da6fd8828 100644 --- a/src/coordination/include/coordination/instance_status.hpp +++ b/src/coordination/include/coordination/instance_status.hpp @@ -26,7 +26,7 @@ struct InstanceStatus { std::string raft_socket_address; std::string coord_socket_address; std::string cluster_role; - bool is_alive; + std::string health; }; } // namespace memgraph::coordination diff --git a/src/coordination/include/coordination/raft_state.hpp b/src/coordination/include/coordination/raft_state.hpp index b6ef06008..d702697f1 100644 --- a/src/coordination/include/coordination/raft_state.hpp +++ b/src/coordination/include/coordination/raft_state.hpp @@ -14,11 +14,16 @@ #ifdef MG_ENTERPRISE #include <flags/replication.hpp> +#include "nuraft/coordinator_state_machine.hpp" +#include "nuraft/coordinator_state_manager.hpp" #include <libnuraft/nuraft.hxx> namespace memgraph::coordination { +class CoordinatorInstance; +struct CoordinatorClientConfig; + using BecomeLeaderCb = std::function<void()>; using BecomeFollowerCb = std::function<void()>; @@ -47,26 +52,39 @@ class RaftState { RaftState &operator=(RaftState &&other) noexcept = default; ~RaftState(); - static auto MakeRaftState(BecomeLeaderCb become_leader_cb, BecomeFollowerCb become_follower_cb) -> RaftState; + static auto MakeRaftState(BecomeLeaderCb &&become_leader_cb, BecomeFollowerCb &&become_follower_cb) -> RaftState; auto InstanceName() const -> std::string; auto RaftSocketAddress() const -> std::string; - auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) -> void; + auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string_view raft_address) -> void; auto GetAllCoordinators() const -> std::vector<ptr<srv_config>>; auto RequestLeadership() -> bool; auto IsLeader() const -> bool; - auto AppendRegisterReplicationInstance(std::string const &instance) -> ptr<raft_result>; + auto FindCurrentMainInstanceName() const -> std::optional<std::string>; + auto MainExists() const -> bool; + auto IsMain(std::string_view instance_name) const -> bool; + auto IsReplica(std::string_view instance_name) const -> bool; - // TODO: (andi) I think variables below can be abstracted + auto AppendRegisterReplicationInstanceLog(CoordinatorClientConfig const &config) -> bool; + auto AppendUnregisterReplicationInstanceLog(std::string_view instance_name) -> bool; + auto AppendSetInstanceAsMainLog(std::string_view instance_name) -> bool; + auto AppendSetInstanceAsReplicaLog(std::string_view instance_name) -> bool; + auto AppendUpdateUUIDLog(utils::UUID const &uuid) -> bool; + + auto GetInstances() const -> std::vector<InstanceState>; + auto GetUUID() const -> utils::UUID; + + private: + // TODO: (andi) I think variables below can be abstracted/clean them. uint32_t raft_server_id_; uint32_t raft_port_; std::string raft_address_; - ptr<state_machine> state_machine_; - ptr<state_mgr> state_manager_; + ptr<CoordinatorStateMachine> state_machine_; + ptr<CoordinatorStateManager> state_manager_; ptr<raft_server> raft_server_; ptr<logger> logger_; raft_launcher launcher_; diff --git a/src/coordination/include/coordination/register_main_replica_coordinator_status.hpp b/src/coordination/include/coordination/register_main_replica_coordinator_status.hpp index 3aa7e3ca1..13b58ff9f 100644 --- a/src/coordination/include/coordination/register_main_replica_coordinator_status.hpp +++ b/src/coordination/include/coordination/register_main_replica_coordinator_status.hpp @@ -19,12 +19,12 @@ namespace memgraph::coordination { enum class RegisterInstanceCoordinatorStatus : uint8_t { NAME_EXISTS, - ENDPOINT_EXISTS, + COORD_ENDPOINT_EXISTS, + REPL_ENDPOINT_EXISTS, NOT_COORDINATOR, - RPC_FAILED, NOT_LEADER, - RAFT_COULD_NOT_ACCEPT, - RAFT_COULD_NOT_APPEND, + RPC_FAILED, + RAFT_LOG_ERROR, SUCCESS }; @@ -32,8 +32,9 @@ enum class UnregisterInstanceCoordinatorStatus : uint8_t { NO_INSTANCE_WITH_NAME, IS_MAIN, NOT_COORDINATOR, - NOT_LEADER, RPC_FAILED, + NOT_LEADER, + RAFT_LOG_ERROR, SUCCESS, }; @@ -41,9 +42,11 @@ enum class SetInstanceToMainCoordinatorStatus : uint8_t { NO_INSTANCE_WITH_NAME, MAIN_ALREADY_EXISTS, NOT_COORDINATOR, - SUCCESS, + NOT_LEADER, + RAFT_LOG_ERROR, COULD_NOT_PROMOTE_TO_MAIN, - SWAP_UUID_FAILED + SWAP_UUID_FAILED, + SUCCESS, }; } // namespace memgraph::coordination diff --git a/src/coordination/include/coordination/replication_instance.hpp b/src/coordination/include/coordination/replication_instance.hpp index e8e00a0a8..7b5d73b81 100644 --- a/src/coordination/include/coordination/replication_instance.hpp +++ b/src/coordination/include/coordination/replication_instance.hpp @@ -17,11 +17,12 @@ #include "coordination/coordinator_exceptions.hpp" #include "replication_coordination_glue/role.hpp" -#include <libnuraft/nuraft.hxx> #include "utils/resource_lock.hpp" #include "utils/result.hpp" #include "utils/uuid.hpp" +#include <libnuraft/nuraft.hxx> + namespace memgraph::coordination { class CoordinatorInstance; @@ -50,13 +51,14 @@ class ReplicationInstance { auto IsAlive() const -> bool; auto InstanceName() const -> std::string; - auto SocketAddress() const -> std::string; + auto CoordinatorSocketAddress() const -> std::string; + auto ReplicationSocketAddress() const -> std::string; - auto IsReplica() const -> bool; - auto IsMain() const -> bool; - - auto PromoteToMain(utils::UUID uuid, ReplicationClientsInfo repl_clients_info, + auto PromoteToMain(utils::UUID const &uuid, ReplicationClientsInfo repl_clients_info, HealthCheckInstanceCallback main_succ_cb, HealthCheckInstanceCallback main_fail_cb) -> bool; + + auto SendDemoteToReplicaRpc() -> bool; + auto DemoteToReplica(HealthCheckInstanceCallback replica_succ_cb, HealthCheckInstanceCallback replica_fail_cb) -> bool; @@ -69,8 +71,8 @@ class ReplicationInstance { auto EnsureReplicaHasCorrectMainUUID(utils::UUID const &curr_main_uuid) -> bool; - auto SendSwapAndUpdateUUID(const utils::UUID &new_main_uuid) -> bool; - auto SendUnregisterReplicaRpc(std::string const &instance_name) -> bool; + auto SendSwapAndUpdateUUID(utils::UUID const &new_main_uuid) -> bool; + auto SendUnregisterReplicaRpc(std::string_view instance_name) -> bool; auto SendGetInstanceUUID() -> utils::BasicResult<coordination::GetInstanceUUIDError, std::optional<utils::UUID>>; auto GetClient() -> CoordinatorClient &; @@ -78,14 +80,14 @@ class ReplicationInstance { auto EnableWritingOnMain() -> bool; auto SetNewMainUUID(utils::UUID const &main_uuid) -> void; - auto GetMainUUID() const -> const std::optional<utils::UUID> &; + auto ResetMainUUID() -> void; + auto GetMainUUID() const -> std::optional<utils::UUID> const &; auto GetSuccessCallback() -> HealthCheckInstanceCallback &; auto GetFailCallback() -> HealthCheckInstanceCallback &; private: CoordinatorClient client_; - replication_coordination_glue::ReplicationRole replication_role_; std::chrono::system_clock::time_point last_response_time_{}; bool is_alive_{false}; std::chrono::system_clock::time_point last_check_of_uuid_{}; @@ -101,7 +103,8 @@ class ReplicationInstance { HealthCheckInstanceCallback fail_cb_; friend bool operator==(ReplicationInstance const &first, ReplicationInstance const &second) { - return first.client_ == second.client_ && first.replication_role_ == second.replication_role_; + return first.client_ == second.client_ && first.last_response_time_ == second.last_response_time_ && + first.is_alive_ == second.is_alive_ && first.main_uuid_ == second.main_uuid_; } }; diff --git a/src/coordination/include/nuraft/coordinator_cluster_state.hpp b/src/coordination/include/nuraft/coordinator_cluster_state.hpp new file mode 100644 index 000000000..f38d00073 --- /dev/null +++ b/src/coordination/include/nuraft/coordinator_cluster_state.hpp @@ -0,0 +1,82 @@ +// 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 +// 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. + +#pragma once + +#ifdef MG_ENTERPRISE + +#include "coordination/coordinator_config.hpp" +#include "nuraft/raft_log_action.hpp" +#include "replication_coordination_glue/role.hpp" +#include "utils/resource_lock.hpp" +#include "utils/uuid.hpp" + +#include <libnuraft/nuraft.hxx> +#include <range/v3/view.hpp> + +#include <map> +#include <numeric> +#include <string> +#include <variant> + +namespace memgraph::coordination { + +using replication_coordination_glue::ReplicationRole; + +struct InstanceState { + CoordinatorClientConfig config; + ReplicationRole role; +}; + +using TRaftLog = std::variant<CoordinatorClientConfig, std::string, utils::UUID>; + +using nuraft::buffer; +using nuraft::buffer_serializer; +using nuraft::ptr; + +class CoordinatorClusterState { + public: + CoordinatorClusterState() = default; + CoordinatorClusterState(CoordinatorClusterState const &); + CoordinatorClusterState &operator=(CoordinatorClusterState const &); + + CoordinatorClusterState(CoordinatorClusterState &&other) noexcept; + CoordinatorClusterState &operator=(CoordinatorClusterState &&other) noexcept; + ~CoordinatorClusterState() = default; + + auto FindCurrentMainInstanceName() const -> std::optional<std::string>; + + auto MainExists() const -> bool; + + auto IsMain(std::string_view instance_name) const -> bool; + + auto IsReplica(std::string_view instance_name) const -> bool; + + auto InsertInstance(std::string_view instance_name, ReplicationRole role) -> void; + + auto DoAction(TRaftLog log_entry, RaftLogAction log_action) -> void; + + auto Serialize(ptr<buffer> &data) -> void; + + static auto Deserialize(buffer &data) -> CoordinatorClusterState; + + auto GetInstances() const -> std::vector<InstanceState>; + + auto GetUUID() const -> utils::UUID; + + private: + std::map<std::string, InstanceState, std::less<>> instance_roles_; + utils::UUID uuid_{}; + mutable utils::ResourceLock log_lock_{}; +}; + +} // namespace memgraph::coordination +#endif diff --git a/src/coordination/include/nuraft/coordinator_state_machine.hpp b/src/coordination/include/nuraft/coordinator_state_machine.hpp index 5b5f37b48..aea21ab4e 100644 --- a/src/coordination/include/nuraft/coordinator_state_machine.hpp +++ b/src/coordination/include/nuraft/coordinator_state_machine.hpp @@ -13,9 +13,15 @@ #ifdef MG_ENTERPRISE +#include "coordination/coordinator_config.hpp" +#include "nuraft/coordinator_cluster_state.hpp" +#include "nuraft/raft_log_action.hpp" + #include <spdlog/spdlog.h> #include <libnuraft/nuraft.hxx> +#include <variant> + namespace memgraph::coordination { using nuraft::async_result; @@ -36,9 +42,19 @@ class CoordinatorStateMachine : public state_machine { CoordinatorStateMachine &operator=(CoordinatorStateMachine &&) = delete; ~CoordinatorStateMachine() override {} - static auto EncodeRegisterReplicationInstance(const std::string &name) -> ptr<buffer>; + auto FindCurrentMainInstanceName() const -> std::optional<std::string>; + auto MainExists() const -> bool; + auto IsMain(std::string_view instance_name) const -> bool; + auto IsReplica(std::string_view instance_name) const -> bool; - static auto DecodeRegisterReplicationInstance(buffer &data) -> std::string; + static auto CreateLog(std::string_view log) -> ptr<buffer>; + static auto SerializeRegisterInstance(CoordinatorClientConfig const &config) -> ptr<buffer>; + static auto SerializeUnregisterInstance(std::string_view instance_name) -> ptr<buffer>; + static auto SerializeSetInstanceAsMain(std::string_view instance_name) -> ptr<buffer>; + static auto SerializeSetInstanceAsReplica(std::string_view instance_name) -> ptr<buffer>; + static auto SerializeUpdateUUID(utils::UUID const &uuid) -> ptr<buffer>; + + static auto DecodeLog(buffer &data) -> std::pair<TRaftLog, RaftLogAction>; auto pre_commit(ulong log_idx, buffer &data) -> ptr<buffer> override; @@ -64,11 +80,31 @@ class CoordinatorStateMachine : public state_machine { auto create_snapshot(snapshot &s, async_result<bool>::handler_type &when_done) -> void override; + auto GetInstances() const -> std::vector<InstanceState>; + auto GetUUID() const -> utils::UUID; + private: + struct SnapshotCtx { + SnapshotCtx(ptr<snapshot> &snapshot, CoordinatorClusterState const &cluster_state) + : snapshot_(snapshot), cluster_state_(cluster_state) {} + + ptr<snapshot> snapshot_; + CoordinatorClusterState cluster_state_; + }; + + auto create_snapshot_internal(ptr<snapshot> snapshot) -> void; + + CoordinatorClusterState cluster_state_; + + // mutable utils::RWLock lock{utils::RWLock::Priority::READ}; + std::atomic<uint64_t> last_committed_idx_{0}; - ptr<snapshot> last_snapshot_; + // TODO: (andi) Maybe not needed, remove it + std::map<uint64_t, ptr<SnapshotCtx>> snapshots_; + std::mutex snapshots_lock_; + ptr<snapshot> last_snapshot_; std::mutex last_snapshot_lock_; }; diff --git a/src/coordination/include/nuraft/raft_log_action.hpp b/src/coordination/include/nuraft/raft_log_action.hpp new file mode 100644 index 000000000..399d33150 --- /dev/null +++ b/src/coordination/include/nuraft/raft_log_action.hpp @@ -0,0 +1,53 @@ +// 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 +// 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. + +#pragma once + +#ifdef MG_ENTERPRISE + +#include "coordination/coordinator_exceptions.hpp" + +#include <cstdint> +#include <string> + +namespace memgraph::coordination { + +enum class RaftLogAction : uint8_t { + REGISTER_REPLICATION_INSTANCE, + UNREGISTER_REPLICATION_INSTANCE, + SET_INSTANCE_AS_MAIN, + SET_INSTANCE_AS_REPLICA, + UPDATE_UUID +}; + +inline auto ParseRaftLogAction(std::string_view action) -> RaftLogAction { + if (action == "register") { + return RaftLogAction::REGISTER_REPLICATION_INSTANCE; + } + if (action == "unregister") { + return RaftLogAction::UNREGISTER_REPLICATION_INSTANCE; + } + if (action == "promote") { + return RaftLogAction::SET_INSTANCE_AS_MAIN; + } + if (action == "demote") { + return RaftLogAction::SET_INSTANCE_AS_REPLICA; + } + + if (action == "update_uuid") { + return RaftLogAction::UPDATE_UUID; + } + + throw InvalidRaftLogActionException("Invalid Raft log action: {}.", action); +} + +} // namespace memgraph::coordination +#endif diff --git a/src/coordination/raft_state.cpp b/src/coordination/raft_state.cpp index d171a6b3d..365388b06 100644 --- a/src/coordination/raft_state.cpp +++ b/src/coordination/raft_state.cpp @@ -13,9 +13,8 @@ #include "coordination/raft_state.hpp" +#include "coordination/coordinator_config.hpp" #include "coordination/coordinator_exceptions.hpp" -#include "nuraft/coordinator_state_machine.hpp" -#include "nuraft/coordinator_state_manager.hpp" #include "utils/counter.hpp" namespace memgraph::coordination { @@ -90,18 +89,13 @@ auto RaftState::InitRaftServer() -> void { throw RaftServerStartException("Failed to initialize raft server on {}:{}", raft_address_, raft_port_); } -auto RaftState::MakeRaftState(BecomeLeaderCb become_leader_cb, BecomeFollowerCb become_follower_cb) -> RaftState { - uint32_t raft_server_id{0}; - uint32_t raft_port{0}; - try { - raft_server_id = FLAGS_raft_server_id; - raft_port = FLAGS_raft_server_port; - } catch (std::exception const &e) { - throw RaftCouldNotParseFlagsException("Failed to parse flags: {}", e.what()); - } +auto RaftState::MakeRaftState(BecomeLeaderCb &&become_leader_cb, BecomeFollowerCb &&become_follower_cb) -> RaftState { + uint32_t raft_server_id = FLAGS_raft_server_id; + uint32_t raft_port = FLAGS_raft_server_port; auto raft_state = RaftState(std::move(become_leader_cb), std::move(become_follower_cb), raft_server_id, raft_port, "127.0.0.1"); + raft_state.InitRaftServer(); return raft_state; } @@ -112,8 +106,9 @@ auto RaftState::InstanceName() const -> std::string { return "coordinator_" + st auto RaftState::RaftSocketAddress() const -> std::string { return raft_address_ + ":" + std::to_string(raft_port_); } -auto RaftState::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) -> void { - auto const endpoint = raft_address + ":" + std::to_string(raft_port); +auto RaftState::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string_view raft_address) + -> void { + auto const endpoint = fmt::format("{}:{}", raft_address, raft_port); srv_config const srv_config_to_add(static_cast<int>(raft_server_id), endpoint); if (!raft_server_->add_srv(srv_config_to_add)->get_accepted()) { throw RaftAddServerException("Failed to add server {} to the cluster", endpoint); @@ -131,10 +126,123 @@ auto RaftState::IsLeader() const -> bool { return raft_server_->is_leader(); } auto RaftState::RequestLeadership() -> bool { return raft_server_->is_leader() || raft_server_->request_leadership(); } -auto RaftState::AppendRegisterReplicationInstance(std::string const &instance) -> ptr<raft_result> { - auto new_log = CoordinatorStateMachine::EncodeRegisterReplicationInstance(instance); - return raft_server_->append_entries({new_log}); +auto RaftState::AppendRegisterReplicationInstanceLog(CoordinatorClientConfig const &config) -> bool { + auto new_log = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto const res = raft_server_->append_entries({new_log}); + + if (!res->get_accepted()) { + spdlog::error( + "Failed to accept request for registering instance {}. Most likely the reason is that the instance is not " + "the " + "leader.", + config.instance_name); + return false; + } + + spdlog::info("Request for registering instance {} accepted", config.instance_name); + + if (res->get_result_code() != nuraft::cmd_result_code::OK) { + spdlog::error("Failed to register instance {} with error code {}", config.instance_name, res->get_result_code()); + return false; + } + + return true; } +auto RaftState::AppendUnregisterReplicationInstanceLog(std::string_view instance_name) -> bool { + auto new_log = CoordinatorStateMachine::SerializeUnregisterInstance(instance_name); + auto const res = raft_server_->append_entries({new_log}); + if (!res->get_accepted()) { + spdlog::error( + "Failed to accept request for unregistering instance {}. Most likely the reason is that the instance is not " + "the leader.", + instance_name); + return false; + } + + spdlog::info("Request for unregistering instance {} accepted", instance_name); + + if (res->get_result_code() != nuraft::cmd_result_code::OK) { + spdlog::error("Failed to unregister instance {} with error code {}", instance_name, res->get_result_code()); + return false; + } + return true; +} + +auto RaftState::AppendSetInstanceAsMainLog(std::string_view instance_name) -> bool { + auto new_log = CoordinatorStateMachine::SerializeSetInstanceAsMain(instance_name); + auto const res = raft_server_->append_entries({new_log}); + if (!res->get_accepted()) { + spdlog::error( + "Failed to accept request for promoting instance {}. Most likely the reason is that the instance is not " + "the leader.", + instance_name); + return false; + } + + spdlog::info("Request for promoting instance {} accepted", instance_name); + + if (res->get_result_code() != nuraft::cmd_result_code::OK) { + spdlog::error("Failed to promote instance {} with error code {}", instance_name, res->get_result_code()); + return false; + } + return true; +} + +auto RaftState::AppendSetInstanceAsReplicaLog(std::string_view instance_name) -> bool { + auto new_log = CoordinatorStateMachine::SerializeSetInstanceAsReplica(instance_name); + auto const res = raft_server_->append_entries({new_log}); + if (!res->get_accepted()) { + spdlog::error( + "Failed to accept request for demoting instance {}. Most likely the reason is that the instance is not " + "the leader.", + instance_name); + return false; + } + spdlog::info("Request for demoting instance {} accepted", instance_name); + + if (res->get_result_code() != nuraft::cmd_result_code::OK) { + spdlog::error("Failed to promote instance {} with error code {}", instance_name, res->get_result_code()); + return false; + } + + return true; +} + +auto RaftState::AppendUpdateUUIDLog(utils::UUID const &uuid) -> bool { + auto new_log = CoordinatorStateMachine::SerializeUpdateUUID(uuid); + auto const res = raft_server_->append_entries({new_log}); + if (!res->get_accepted()) { + spdlog::error( + "Failed to accept request for updating UUID. Most likely the reason is that the instance is not " + "the leader."); + return false; + } + spdlog::info("Request for updating UUID accepted"); + + if (res->get_result_code() != nuraft::cmd_result_code::OK) { + spdlog::error("Failed to update UUID with error code {}", res->get_result_code()); + return false; + } + + return true; +} + +auto RaftState::FindCurrentMainInstanceName() const -> std::optional<std::string> { + return state_machine_->FindCurrentMainInstanceName(); +} + +auto RaftState::MainExists() const -> bool { return state_machine_->MainExists(); } + +auto RaftState::IsMain(std::string_view instance_name) const -> bool { return state_machine_->IsMain(instance_name); } + +auto RaftState::IsReplica(std::string_view instance_name) const -> bool { + return state_machine_->IsReplica(instance_name); +} + +auto RaftState::GetInstances() const -> std::vector<InstanceState> { return state_machine_->GetInstances(); } + +auto RaftState::GetUUID() const -> utils::UUID { return state_machine_->GetUUID(); } + } // namespace memgraph::coordination #endif diff --git a/src/coordination/replication_instance.cpp b/src/coordination/replication_instance.cpp index 50f1be468..ca7572ea7 100644 --- a/src/coordination/replication_instance.cpp +++ b/src/coordination/replication_instance.cpp @@ -25,15 +25,8 @@ ReplicationInstance::ReplicationInstance(CoordinatorInstance *peer, CoordinatorC HealthCheckInstanceCallback succ_instance_cb, HealthCheckInstanceCallback fail_instance_cb) : client_(peer, std::move(config), std::move(succ_cb), std::move(fail_cb)), - replication_role_(replication_coordination_glue::ReplicationRole::REPLICA), succ_cb_(succ_instance_cb), - fail_cb_(fail_instance_cb) { - if (!client_.DemoteToReplica()) { - throw CoordinatorRegisterInstanceException("Failed to demote instance {} to replica", client_.InstanceName()); - } - - client_.StartFrequentCheck(); -} + fail_cb_(fail_instance_cb) {} auto ReplicationInstance::OnSuccessPing() -> void { last_response_time_ = std::chrono::system_clock::now(); @@ -52,24 +45,17 @@ auto ReplicationInstance::IsReadyForUUIDPing() -> bool { } auto ReplicationInstance::InstanceName() const -> std::string { return client_.InstanceName(); } -auto ReplicationInstance::SocketAddress() const -> std::string { return client_.SocketAddress(); } +auto ReplicationInstance::CoordinatorSocketAddress() const -> std::string { return client_.CoordinatorSocketAddress(); } +auto ReplicationInstance::ReplicationSocketAddress() const -> std::string { return client_.ReplicationSocketAddress(); } auto ReplicationInstance::IsAlive() const -> bool { return is_alive_; } -auto ReplicationInstance::IsReplica() const -> bool { - return replication_role_ == replication_coordination_glue::ReplicationRole::REPLICA; -} -auto ReplicationInstance::IsMain() const -> bool { - return replication_role_ == replication_coordination_glue::ReplicationRole::MAIN; -} - -auto ReplicationInstance::PromoteToMain(utils::UUID new_uuid, ReplicationClientsInfo repl_clients_info, +auto ReplicationInstance::PromoteToMain(utils::UUID const &new_uuid, ReplicationClientsInfo repl_clients_info, HealthCheckInstanceCallback main_succ_cb, HealthCheckInstanceCallback main_fail_cb) -> bool { if (!client_.SendPromoteReplicaToMainRpc(new_uuid, std::move(repl_clients_info))) { return false; } - replication_role_ = replication_coordination_glue::ReplicationRole::MAIN; main_uuid_ = new_uuid; succ_cb_ = main_succ_cb; fail_cb_ = main_fail_cb; @@ -77,13 +63,14 @@ auto ReplicationInstance::PromoteToMain(utils::UUID new_uuid, ReplicationClients return true; } +auto ReplicationInstance::SendDemoteToReplicaRpc() -> bool { return client_.DemoteToReplica(); } + auto ReplicationInstance::DemoteToReplica(HealthCheckInstanceCallback replica_succ_cb, HealthCheckInstanceCallback replica_fail_cb) -> bool { if (!client_.DemoteToReplica()) { return false; } - replication_role_ = replication_coordination_glue::ReplicationRole::REPLICA; succ_cb_ = replica_succ_cb; fail_cb_ = replica_fail_cb; @@ -117,6 +104,7 @@ auto ReplicationInstance::EnsureReplicaHasCorrectMainUUID(utils::UUID const &cur } UpdateReplicaLastResponseUUID(); + // NOLINTNEXTLINE if (res.GetValue().has_value() && res.GetValue().value() == curr_main_uuid) { return true; } @@ -124,7 +112,7 @@ auto ReplicationInstance::EnsureReplicaHasCorrectMainUUID(utils::UUID const &cur return SendSwapAndUpdateUUID(curr_main_uuid); } -auto ReplicationInstance::SendSwapAndUpdateUUID(const utils::UUID &new_main_uuid) -> bool { +auto ReplicationInstance::SendSwapAndUpdateUUID(utils::UUID const &new_main_uuid) -> bool { if (!replication_coordination_glue::SendSwapMainUUIDRpc(client_.RpcClient(), new_main_uuid)) { return false; } @@ -132,7 +120,7 @@ auto ReplicationInstance::SendSwapAndUpdateUUID(const utils::UUID &new_main_uuid return true; } -auto ReplicationInstance::SendUnregisterReplicaRpc(std::string const &instance_name) -> bool { +auto ReplicationInstance::SendUnregisterReplicaRpc(std::string_view instance_name) -> bool { return client_.SendUnregisterReplicaRpc(instance_name); } diff --git a/src/dbms/coordinator_handler.cpp b/src/dbms/coordinator_handler.cpp index f8e14e2a0..292d50d3d 100644 --- a/src/dbms/coordinator_handler.cpp +++ b/src/dbms/coordinator_handler.cpp @@ -20,28 +20,28 @@ namespace memgraph::dbms { CoordinatorHandler::CoordinatorHandler(coordination::CoordinatorState &coordinator_state) : coordinator_state_(coordinator_state) {} -auto CoordinatorHandler::RegisterReplicationInstance(memgraph::coordination::CoordinatorClientConfig config) +auto CoordinatorHandler::RegisterReplicationInstance(coordination::CoordinatorClientConfig const &config) -> coordination::RegisterInstanceCoordinatorStatus { return coordinator_state_.RegisterReplicationInstance(config); } -auto CoordinatorHandler::UnregisterReplicationInstance(std::string instance_name) +auto CoordinatorHandler::UnregisterReplicationInstance(std::string_view instance_name) -> coordination::UnregisterInstanceCoordinatorStatus { - return coordinator_state_.UnregisterReplicationInstance(std::move(instance_name)); + return coordinator_state_.UnregisterReplicationInstance(instance_name); } -auto CoordinatorHandler::SetReplicationInstanceToMain(std::string instance_name) +auto CoordinatorHandler::SetReplicationInstanceToMain(std::string_view instance_name) -> coordination::SetInstanceToMainCoordinatorStatus { - return coordinator_state_.SetReplicationInstanceToMain(std::move(instance_name)); + return coordinator_state_.SetReplicationInstanceToMain(instance_name); } auto CoordinatorHandler::ShowInstances() const -> std::vector<coordination::InstanceStatus> { return coordinator_state_.ShowInstances(); } -auto CoordinatorHandler::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) - -> void { - coordinator_state_.AddCoordinatorInstance(raft_server_id, raft_port, std::move(raft_address)); +auto CoordinatorHandler::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, + std::string_view raft_address) -> void { + coordinator_state_.AddCoordinatorInstance(raft_server_id, raft_port, raft_address); } } // namespace memgraph::dbms diff --git a/src/dbms/coordinator_handler.hpp b/src/dbms/coordinator_handler.hpp index d06e70676..1c456134d 100644 --- a/src/dbms/coordinator_handler.hpp +++ b/src/dbms/coordinator_handler.hpp @@ -30,16 +30,17 @@ class CoordinatorHandler { // TODO: (andi) When moving coordinator state on same instances, rename from RegisterReplicationInstance to // RegisterInstance - auto RegisterReplicationInstance(coordination::CoordinatorClientConfig config) + auto RegisterReplicationInstance(coordination::CoordinatorClientConfig const &config) -> coordination::RegisterInstanceCoordinatorStatus; - auto UnregisterReplicationInstance(std::string instance_name) -> coordination::UnregisterInstanceCoordinatorStatus; + auto UnregisterReplicationInstance(std::string_view instance_name) + -> coordination::UnregisterInstanceCoordinatorStatus; - auto SetReplicationInstanceToMain(std::string instance_name) -> coordination::SetInstanceToMainCoordinatorStatus; + auto SetReplicationInstanceToMain(std::string_view instance_name) -> coordination::SetInstanceToMainCoordinatorStatus; auto ShowInstances() const -> std::vector<coordination::InstanceStatus>; - auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) -> void; + auto AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string_view raft_address) -> void; private: coordination::CoordinatorState &coordinator_state_; diff --git a/src/dbms/inmemory/replication_handlers.cpp b/src/dbms/inmemory/replication_handlers.cpp index 3fc174d3c..8339b65b4 100644 --- a/src/dbms/inmemory/replication_handlers.cpp +++ b/src/dbms/inmemory/replication_handlers.cpp @@ -19,7 +19,6 @@ #include "storage/v2/durability/durability.hpp" #include "storage/v2/durability/snapshot.hpp" #include "storage/v2/durability/version.hpp" -#include "storage/v2/fmt.hpp" #include "storage/v2/indices/label_index_stats.hpp" #include "storage/v2/inmemory/storage.hpp" #include "storage/v2/inmemory/unique_constraints.hpp" @@ -135,7 +134,7 @@ void InMemoryReplicationHandlers::SwapMainUUIDHandler(dbms::DbmsHandler *dbms_ha replication_coordination_glue::SwapMainUUIDReq req; slk::Load(&req, req_reader); - spdlog::info(fmt::format("Set replica data UUID to main uuid {}", std::string(req.uuid))); + spdlog::info("Set replica data UUID to main uuid {}", std::string(req.uuid)); dbms_handler->ReplicationState().TryPersistRoleReplica(role_replica_data.config, req.uuid); role_replica_data.uuid_ = req.uuid; diff --git a/src/io/network/endpoint.cpp b/src/io/network/endpoint.cpp index 44123db6b..bb6dcfd10 100644 --- a/src/io/network/endpoint.cpp +++ b/src/io/network/endpoint.cpp @@ -24,22 +24,22 @@ namespace memgraph::io::network { -Endpoint::IpFamily Endpoint::GetIpFamily(const std::string &address) { +Endpoint::IpFamily Endpoint::GetIpFamily(std::string_view address) { in_addr addr4; in6_addr addr6; - int ipv4_result = inet_pton(AF_INET, address.c_str(), &addr4); - int ipv6_result = inet_pton(AF_INET6, address.c_str(), &addr6); + int ipv4_result = inet_pton(AF_INET, address.data(), &addr4); + int ipv6_result = inet_pton(AF_INET6, address.data(), &addr6); if (ipv4_result == 1) { return IpFamily::IP4; - } else if (ipv6_result == 1) { - return IpFamily::IP6; - } else { - return IpFamily::NONE; } + if (ipv6_result == 1) { + return IpFamily::IP6; + } + return IpFamily::NONE; } std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrIpAddress( - const std::string &address, const std::optional<uint16_t> default_port) { + std::string_view address, const std::optional<uint16_t> default_port) { /// expected address format: /// - "ip_address:port_number" /// - "ip_address" @@ -56,7 +56,7 @@ std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrIpAddress if (GetIpFamily(address) == IpFamily::NONE) { return std::nullopt; } - return std::pair{address, *default_port}; + return std::pair{std::string(address), *default_port}; // TODO: (andi) Optimize throughout the code } } else if (parts.size() == 2) { ip_address = std::move(parts[0]); @@ -88,7 +88,7 @@ std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrIpAddress } std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseHostname( - const std::string &address, const std::optional<uint16_t> default_port = {}) { + std::string_view address, const std::optional<uint16_t> default_port = {}) { const std::string delimiter = ":"; std::string ip_address; std::vector<std::string> parts = utils::Split(address, delimiter); @@ -97,7 +97,7 @@ std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseHostname( if (!IsResolvableAddress(address, *default_port)) { return std::nullopt; } - return std::pair{address, *default_port}; + return std::pair{std::string(address), *default_port}; // TODO: (andi) Optimize throughout the code } } else if (parts.size() == 2) { int64_t int_port{0}; @@ -153,20 +153,20 @@ std::ostream &operator<<(std::ostream &os, const Endpoint &endpoint) { return os << endpoint.address << ":" << endpoint.port; } -bool Endpoint::IsResolvableAddress(const std::string &address, uint16_t port) { +bool Endpoint::IsResolvableAddress(std::string_view address, uint16_t port) { addrinfo hints{ .ai_flags = AI_PASSIVE, .ai_family = AF_UNSPEC, // IPv4 and IPv6 .ai_socktype = SOCK_STREAM // TCP socket }; addrinfo *info = nullptr; - auto status = getaddrinfo(address.c_str(), std::to_string(port).c_str(), &hints, &info); + auto status = getaddrinfo(address.data(), std::to_string(port).c_str(), &hints, &info); if (info) freeaddrinfo(info); return status == 0; } std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrAddress( - const std::string &address, const std::optional<uint16_t> default_port) { + std::string_view address, const std::optional<uint16_t> default_port) { const std::string delimiter = ":"; std::vector<std::string> parts = utils::Split(address, delimiter); if (parts.size() == 1) { diff --git a/src/io/network/endpoint.hpp b/src/io/network/endpoint.hpp index 16d70e080..b0201240b 100644 --- a/src/io/network/endpoint.hpp +++ b/src/io/network/endpoint.hpp @@ -48,8 +48,8 @@ struct Endpoint { uint16_t port{0}; IpFamily family{IpFamily::NONE}; - static std::optional<std::pair<std::string, uint16_t>> ParseSocketOrAddress(const std::string &address, - std::optional<uint16_t> default_port); + static std::optional<std::pair<std::string, uint16_t>> ParseSocketOrAddress( + std::string_view address, std::optional<uint16_t> default_port = {}); /** * Tries to parse the given string as either a socket address or ip address. @@ -62,7 +62,7 @@ struct Endpoint { * it won't be used, as we expect that it is given in the address string. */ static std::optional<std::pair<std::string, uint16_t>> ParseSocketOrIpAddress( - const std::string &address, std::optional<uint16_t> default_port = {}); + std::string_view address, std::optional<uint16_t> default_port = {}); /** * Tries to parse given string as either socket address or hostname. @@ -71,12 +71,12 @@ struct Endpoint { * - "hostname" * After we parse hostname and port we try to resolve the hostname into an ip_address. */ - static std::optional<std::pair<std::string, uint16_t>> ParseHostname(const std::string &address, + static std::optional<std::pair<std::string, uint16_t>> ParseHostname(std::string_view address, std::optional<uint16_t> default_port); - static IpFamily GetIpFamily(const std::string &address); + static IpFamily GetIpFamily(std::string_view address); - static bool IsResolvableAddress(const std::string &address, uint16_t port); + static bool IsResolvableAddress(std::string_view address, uint16_t port); /** * Tries to resolve hostname to its corresponding IP address. diff --git a/src/query/interpreter.cpp b/src/query/interpreter.cpp index 0cc0f182a..f7213bed1 100644 --- a/src/query/interpreter.cpp +++ b/src/query/interpreter.cpp @@ -410,7 +410,7 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { : coordinator_handler_(coordinator_state) {} - void UnregisterInstance(std::string const &instance_name) override { + void UnregisterInstance(std::string_view instance_name) override { auto status = coordinator_handler_.UnregisterReplicationInstance(instance_name); switch (status) { using enum memgraph::coordination::UnregisterInstanceCoordinatorStatus; @@ -423,6 +423,8 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { throw QueryRuntimeException("UNREGISTER INSTANCE query can only be run on a coordinator!"); case NOT_LEADER: throw QueryRuntimeException("Couldn't unregister replica instance since coordinator is not a leader!"); + case RAFT_LOG_ERROR: + throw QueryRuntimeException("Couldn't unregister replica instance since raft server couldn't append the log!"); case RPC_FAILED: throw QueryRuntimeException( "Couldn't unregister replica instance because current main instance couldn't unregister replica!"); @@ -431,20 +433,18 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { } } - void RegisterReplicationInstance(std::string const &coordinator_socket_address, - std::string const &replication_socket_address, + void RegisterReplicationInstance(std::string_view coordinator_socket_address, + std::string_view replication_socket_address, std::chrono::seconds const &instance_check_frequency, std::chrono::seconds const &instance_down_timeout, std::chrono::seconds const &instance_get_uuid_frequency, - std::string const &instance_name, CoordinatorQuery::SyncMode sync_mode) override { - const auto maybe_replication_ip_port = - io::network::Endpoint::ParseSocketOrAddress(replication_socket_address, std::nullopt); + std::string_view instance_name, CoordinatorQuery::SyncMode sync_mode) override { + const auto maybe_replication_ip_port = io::network::Endpoint::ParseSocketOrAddress(replication_socket_address); if (!maybe_replication_ip_port) { throw QueryRuntimeException("Invalid replication socket address!"); } - const auto maybe_coordinator_ip_port = - io::network::Endpoint::ParseSocketOrAddress(coordinator_socket_address, std::nullopt); + const auto maybe_coordinator_ip_port = io::network::Endpoint::ParseSocketOrAddress(coordinator_socket_address); if (!maybe_replication_ip_port) { throw QueryRuntimeException("Invalid replication socket address!"); } @@ -452,13 +452,13 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { const auto [replication_ip, replication_port] = *maybe_replication_ip_port; const auto [coordinator_server_ip, coordinator_server_port] = *maybe_coordinator_ip_port; const auto repl_config = coordination::CoordinatorClientConfig::ReplicationClientInfo{ - .instance_name = instance_name, + .instance_name = std::string(instance_name), .replication_mode = convertFromCoordinatorToReplicationMode(sync_mode), .replication_ip_address = replication_ip, .replication_port = replication_port}; auto coordinator_client_config = - coordination::CoordinatorClientConfig{.instance_name = instance_name, + coordination::CoordinatorClientConfig{.instance_name = std::string(instance_name), .ip_address = coordinator_server_ip, .port = coordinator_server_port, .instance_health_check_frequency_sec = instance_check_frequency, @@ -472,18 +472,17 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { using enum memgraph::coordination::RegisterInstanceCoordinatorStatus; case NAME_EXISTS: throw QueryRuntimeException("Couldn't register replica instance since instance with such name already exists!"); - case ENDPOINT_EXISTS: + case COORD_ENDPOINT_EXISTS: throw QueryRuntimeException( - "Couldn't register replica instance since instance with such endpoint already exists!"); + "Couldn't register replica instance since instance with such coordinator endpoint already exists!"); + case REPL_ENDPOINT_EXISTS: + throw QueryRuntimeException( + "Couldn't register replica instance since instance with such replication endpoint already exists!"); case NOT_COORDINATOR: throw QueryRuntimeException("REGISTER INSTANCE query can only be run on a coordinator!"); case NOT_LEADER: throw QueryRuntimeException("Couldn't register replica instance since coordinator is not a leader!"); - case RAFT_COULD_NOT_ACCEPT: - throw QueryRuntimeException( - "Couldn't register replica instance since raft server couldn't accept the log! Most likely the raft " - "instance is not a leader!"); - case RAFT_COULD_NOT_APPEND: + case RAFT_LOG_ERROR: throw QueryRuntimeException("Couldn't register replica instance since raft server couldn't append the log!"); case RPC_FAILED: throw QueryRuntimeException( @@ -494,8 +493,8 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { } } - auto AddCoordinatorInstance(uint32_t raft_server_id, std::string const &raft_socket_address) -> void override { - auto const maybe_ip_and_port = io::network::Endpoint::ParseSocketOrIpAddress(raft_socket_address); + auto AddCoordinatorInstance(uint32_t raft_server_id, std::string_view raft_socket_address) -> void override { + auto const maybe_ip_and_port = io::network::Endpoint::ParseSocketOrAddress(raft_socket_address); if (maybe_ip_and_port) { auto const [ip, port] = *maybe_ip_and_port; spdlog::info("Adding instance {} with raft socket address {}:{}.", raft_server_id, port, ip); @@ -505,8 +504,8 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { } } - void SetReplicationInstanceToMain(const std::string &instance_name) override { - auto status = coordinator_handler_.SetReplicationInstanceToMain(instance_name); + void SetReplicationInstanceToMain(std::string_view instance_name) override { + auto const status = coordinator_handler_.SetReplicationInstanceToMain(instance_name); switch (status) { using enum memgraph::coordination::SetInstanceToMainCoordinatorStatus; case NO_INSTANCE_WITH_NAME: @@ -515,6 +514,10 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { throw QueryRuntimeException("Couldn't set instance to main since there is already a main instance in cluster!"); case NOT_COORDINATOR: throw QueryRuntimeException("SET INSTANCE TO MAIN query can only be run on a coordinator!"); + case NOT_LEADER: + throw QueryRuntimeException("Couldn't set instance to main since coordinator is not a leader!"); + case RAFT_LOG_ERROR: + throw QueryRuntimeException("Couldn't promote instance since raft server couldn't append the log!"); case COULD_NOT_PROMOTE_TO_MAIN: throw QueryRuntimeException( "Couldn't set replica instance to main! Check coordinator and replica for more logs"); @@ -1251,14 +1254,13 @@ Callback HandleCoordinatorQuery(CoordinatorQuery *coordinator_query, const Param throw QueryRuntimeException("Only coordinator can run SHOW INSTANCES."); } - callback.header = {"name", "raft_socket_address", "coordinator_socket_address", "alive", "role"}; + callback.header = {"name", "raft_socket_address", "coordinator_socket_address", "health", "role"}; callback.fn = [handler = CoordQueryHandler{*coordinator_state}, replica_nfields = callback.header.size()]() mutable { auto const instances = handler.ShowInstances(); auto const converter = [](const auto &status) -> std::vector<TypedValue> { return {TypedValue{status.instance_name}, TypedValue{status.raft_socket_address}, - TypedValue{status.coord_socket_address}, TypedValue{status.is_alive}, - TypedValue{status.cluster_role}}; + TypedValue{status.coord_socket_address}, TypedValue{status.health}, TypedValue{status.cluster_role}}; }; return utils::fmap(converter, instances); diff --git a/src/query/interpreter.hpp b/src/query/interpreter.hpp index 01a443d6d..f18bd6721 100644 --- a/src/query/interpreter.hpp +++ b/src/query/interpreter.hpp @@ -95,25 +95,24 @@ class CoordinatorQueryHandler { }; /// @throw QueryRuntimeException if an error ocurred. - virtual void RegisterReplicationInstance(std::string const &coordinator_socket_address, - std::string const &replication_socket_address, + virtual void RegisterReplicationInstance(std::string_view coordinator_socket_address, + std::string_view replication_socket_address, std::chrono::seconds const &instance_health_check_frequency, std::chrono::seconds const &instance_down_timeout, std::chrono::seconds const &instance_get_uuid_frequency, - std::string const &instance_name, CoordinatorQuery::SyncMode sync_mode) = 0; + std::string_view instance_name, CoordinatorQuery::SyncMode sync_mode) = 0; /// @throw QueryRuntimeException if an error ocurred. - virtual void UnregisterInstance(std::string const &instance_name) = 0; + virtual void UnregisterInstance(std::string_view instance_name) = 0; /// @throw QueryRuntimeException if an error ocurred. - virtual void SetReplicationInstanceToMain(const std::string &instance_name) = 0; + virtual void SetReplicationInstanceToMain(std::string_view instance_name) = 0; /// @throw QueryRuntimeException if an error ocurred. virtual std::vector<coordination::InstanceStatus> ShowInstances() const = 0; /// @throw QueryRuntimeException if an error ocurred. - virtual auto AddCoordinatorInstance(uint32_t raft_server_id, std::string const &coordinator_socket_address) - -> void = 0; + virtual auto AddCoordinatorInstance(uint32_t raft_server_id, std::string_view coordinator_socket_address) -> void = 0; }; #endif diff --git a/src/replication_coordination_glue/mode.hpp b/src/replication_coordination_glue/mode.hpp index d0b415733..3f27afb05 100644 --- a/src/replication_coordination_glue/mode.hpp +++ b/src/replication_coordination_glue/mode.hpp @@ -12,7 +12,32 @@ #pragma once #include <cstdint> +#include <map> +#include <stdexcept> +#include <string> namespace memgraph::replication_coordination_glue { + enum class ReplicationMode : std::uint8_t { SYNC, ASYNC }; + +inline auto ReplicationModeToString(ReplicationMode mode) -> std::string { + switch (mode) { + case ReplicationMode::SYNC: + return "SYNC"; + case ReplicationMode::ASYNC: + return "ASYNC"; + } + throw std::invalid_argument("Invalid replication mode"); +} + +inline auto ReplicationModeFromString(std::string_view mode) -> ReplicationMode { + if (mode == "SYNC") { + return ReplicationMode::SYNC; + } + if (mode == "ASYNC") { + return ReplicationMode::ASYNC; + } + throw std::invalid_argument("Invalid replication mode"); +} + } // namespace memgraph::replication_coordination_glue diff --git a/src/utils/CMakeLists.txt b/src/utils/CMakeLists.txt index bac3e78f3..802b8ff6f 100644 --- a/src/utils/CMakeLists.txt +++ b/src/utils/CMakeLists.txt @@ -24,6 +24,7 @@ find_package(Threads REQUIRED) add_library(mg-utils STATIC ${utils_src_files}) add_library(mg::utils ALIAS mg-utils) + target_link_libraries(mg-utils PUBLIC Boost::headers fmt::fmt spdlog::spdlog json) target_link_libraries(mg-utils PRIVATE librdtsc stdc++fs Threads::Threads gflags uuid rt) diff --git a/src/utils/string.hpp b/src/utils/string.hpp index 31723bb65..26709439f 100644 --- a/src/utils/string.hpp +++ b/src/utils/string.hpp @@ -229,6 +229,13 @@ inline std::vector<std::string> Split(const std::string_view src, const std::str return res; } +inline std::vector<std::string_view> SplitView(const std::string_view src, const std::string_view delimiter, + int splits = -1) { + std::vector<std::string_view> res; + Split(&res, src, delimiter, splits); + return res; +} + /** * Split a string by whitespace into a vector. * Runs of consecutive whitespace are regarded as a single delimiter. diff --git a/src/utils/uuid.cpp b/src/utils/uuid.cpp index fbcf662de..d55cfaba3 100644 --- a/src/utils/uuid.cpp +++ b/src/utils/uuid.cpp @@ -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 @@ -10,7 +10,7 @@ // licenses/APL.txt. #include "utils/uuid.hpp" -#include <uuid/uuid.h> + #include "slk/serialization.hpp" namespace memgraph::utils { diff --git a/src/utils/uuid.hpp b/src/utils/uuid.hpp index f83811d10..bbfec6228 100644 --- a/src/utils/uuid.hpp +++ b/src/utils/uuid.hpp @@ -12,6 +12,7 @@ #pragma once #include <uuid/uuid.h> + #include <array> #include <json/json.hpp> #include <string> diff --git a/tests/e2e/high_availability/coord_cluster_registration.py b/tests/e2e/high_availability/coord_cluster_registration.py index 68a387281..774c6dca1 100644 --- a/tests/e2e/high_availability/coord_cluster_registration.py +++ b/tests/e2e/high_availability/coord_cluster_registration.py @@ -133,12 +133,12 @@ def test_register_repl_instances_then_coordinators(): return sorted(list(execute_and_fetch_all(coordinator3_cursor, "SHOW INSTANCES"))) expected_cluster_coord3 = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_cluster_coord3, check_coordinator3) @@ -147,21 +147,23 @@ def test_register_repl_instances_then_coordinators(): def check_coordinator1(): return sorted(list(execute_and_fetch_all(coordinator1_cursor, "SHOW INSTANCES"))) - # TODO: (andi) This should be solved eventually - expected_cluster_not_shared = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), ] - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) coordinator2_cursor = connect(host="localhost", port=7691).cursor() def check_coordinator2(): return sorted(list(execute_and_fetch_all(coordinator2_cursor, "SHOW INSTANCES"))) - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator2) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) def test_register_coordinator_then_repl_instances(): @@ -187,12 +189,12 @@ def test_register_coordinator_then_repl_instances(): return sorted(list(execute_and_fetch_all(coordinator3_cursor, "SHOW INSTANCES"))) expected_cluster_coord3 = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_cluster_coord3, check_coordinator3) @@ -201,21 +203,23 @@ def test_register_coordinator_then_repl_instances(): def check_coordinator1(): return sorted(list(execute_and_fetch_all(coordinator1_cursor, "SHOW INSTANCES"))) - # TODO: (andi) This should be solved eventually - expected_cluster_not_shared = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), ] - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) coordinator2_cursor = connect(host="localhost", port=7691).cursor() def check_coordinator2(): return sorted(list(execute_and_fetch_all(coordinator2_cursor, "SHOW INSTANCES"))) - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator2) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) def test_coordinators_communication_with_restarts(): @@ -237,10 +241,13 @@ def test_coordinators_communication_with_restarts(): ) execute_and_fetch_all(coordinator3_cursor, "SET INSTANCE instance_3 TO MAIN") - expected_cluster_not_shared = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), ] coordinator1_cursor = connect(host="localhost", port=7690).cursor() @@ -248,20 +255,20 @@ def test_coordinators_communication_with_restarts(): def check_coordinator1(): return sorted(list(execute_and_fetch_all(coordinator1_cursor, "SHOW INSTANCES"))) - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) coordinator2_cursor = connect(host="localhost", port=7691).cursor() def check_coordinator2(): return sorted(list(execute_and_fetch_all(coordinator2_cursor, "SHOW INSTANCES"))) - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator2) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_1") interactive_mg_runner.start(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_1") coordinator1_cursor = connect(host="localhost", port=7690).cursor() - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_1") interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_2") @@ -271,11 +278,11 @@ def test_coordinators_communication_with_restarts(): coordinator1_cursor = connect(host="localhost", port=7690).cursor() coordinator2_cursor = connect(host="localhost", port=7691).cursor() - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator1) - mg_sleep_and_assert(expected_cluster_not_shared, check_coordinator2) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) -# TODO: (andi) Test when dealing with distributed coordinators that you can register on one coordinator and unregister from any other coordinator +# # TODO: (andi) Test when dealing with distributed coordinators that you can register on one coordinator and unregister from any other coordinator @pytest.mark.parametrize( "kill_instance", [True, False], @@ -284,7 +291,12 @@ def test_unregister_replicas(kill_instance): safe_execute(shutil.rmtree, TEMP_DIR) interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION) + coordinator1_cursor = connect(host="localhost", port=7690).cursor() + coordinator2_cursor = connect(host="localhost", port=7691).cursor() coordinator3_cursor = connect(host="localhost", port=7692).cursor() + + assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 1 ON '127.0.0.1:10111'") + assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 2 ON '127.0.0.1:10112'") execute_and_fetch_all( coordinator3_cursor, "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'" ) @@ -296,6 +308,12 @@ def test_unregister_replicas(kill_instance): ) execute_and_fetch_all(coordinator3_cursor, "SET INSTANCE instance_3 TO MAIN") + def check_coordinator1(): + return sorted(list(execute_and_fetch_all(coordinator1_cursor, "SHOW INSTANCES"))) + + def check_coordinator2(): + return sorted(list(execute_and_fetch_all(coordinator2_cursor, "SHOW INSTANCES"))) + def check_coordinator3(): return sorted(list(execute_and_fetch_all(coordinator3_cursor, "SHOW INSTANCES"))) @@ -305,10 +323,21 @@ def test_unregister_replicas(kill_instance): return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS"))) expected_cluster = [ - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), ] expected_replicas = [ @@ -328,6 +357,8 @@ def test_unregister_replicas(kill_instance): ), ] + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) mg_sleep_and_assert(expected_cluster, check_coordinator3) mg_sleep_and_assert(expected_replicas, check_main) @@ -336,9 +367,19 @@ def test_unregister_replicas(kill_instance): execute_and_fetch_all(coordinator3_cursor, "UNREGISTER INSTANCE instance_1") expected_cluster = [ - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), ] expected_replicas = [ @@ -351,6 +392,8 @@ def test_unregister_replicas(kill_instance): ), ] + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) mg_sleep_and_assert(expected_cluster, check_coordinator3) mg_sleep_and_assert(expected_replicas, check_main) @@ -359,11 +402,22 @@ def test_unregister_replicas(kill_instance): execute_and_fetch_all(coordinator3_cursor, "UNREGISTER INSTANCE instance_2") expected_cluster = [ - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_3", "", "", "unknown", "main"), ] expected_replicas = [] + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) mg_sleep_and_assert(expected_cluster, check_coordinator3) mg_sleep_and_assert(expected_replicas, check_main) @@ -372,7 +426,11 @@ def test_unregister_main(): safe_execute(shutil.rmtree, TEMP_DIR) interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION) + coordinator1_cursor = connect(host="localhost", port=7690).cursor() + coordinator2_cursor = connect(host="localhost", port=7691).cursor() coordinator3_cursor = connect(host="localhost", port=7692).cursor() + assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 1 ON '127.0.0.1:10111'") + assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 2 ON '127.0.0.1:10112'") execute_and_fetch_all( coordinator3_cursor, "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'" ) @@ -384,16 +442,35 @@ def test_unregister_main(): ) execute_and_fetch_all(coordinator3_cursor, "SET INSTANCE instance_3 TO MAIN") + def check_coordinator1(): + return sorted(list(execute_and_fetch_all(coordinator1_cursor, "SHOW INSTANCES"))) + + def check_coordinator2(): + return sorted(list(execute_and_fetch_all(coordinator2_cursor, "SHOW INSTANCES"))) + def check_coordinator3(): return sorted(list(execute_and_fetch_all(coordinator3_cursor, "SHOW INSTANCES"))) expected_cluster = [ - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), + ] + + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) mg_sleep_and_assert(expected_cluster, check_coordinator3) try: @@ -407,20 +484,43 @@ def test_unregister_main(): interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") expected_cluster = [ - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "main"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), + ] + + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) mg_sleep_and_assert(expected_cluster, check_coordinator3) execute_and_fetch_all(coordinator3_cursor, "UNREGISTER INSTANCE instance_3") expected_cluster = [ - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ] + + expected_cluster_shared = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "main"), + ("instance_2", "", "", "unknown", "replica"), ] expected_replicas = [ @@ -438,6 +538,8 @@ def test_unregister_main(): def check_main(): return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS"))) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator1) + mg_sleep_and_assert(expected_cluster_shared, check_coordinator2) mg_sleep_and_assert(expected_cluster, check_coordinator3) mg_sleep_and_assert(expected_replicas, check_main) diff --git a/tests/e2e/high_availability/coordinator.py b/tests/e2e/high_availability/coordinator.py index 4330c2194..ed55dff9e 100644 --- a/tests/e2e/high_availability/coordinator.py +++ b/tests/e2e/high_availability/coordinator.py @@ -44,10 +44,10 @@ def test_coordinator_show_instances(): return sorted(list(execute_and_fetch_all(cursor, "SHOW INSTANCES;"))) expected_data = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data, retrieve_data) diff --git a/tests/e2e/high_availability/disable_writing_on_main_after_restart.py b/tests/e2e/high_availability/disable_writing_on_main_after_restart.py index 53d570a6d..363ce1c41 100644 --- a/tests/e2e/high_availability/disable_writing_on_main_after_restart.py +++ b/tests/e2e/high_availability/disable_writing_on_main_after_restart.py @@ -143,20 +143,20 @@ def test_writing_disabled_on_main_restart(): return sorted(list(execute_and_fetch_all(coordinator3_cursor, "SHOW INSTANCES"))) expected_cluster_coord3 = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_cluster_coord3, check_coordinator3) interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") expected_cluster_coord3 = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(expected_cluster_coord3, check_coordinator3) @@ -173,10 +173,10 @@ def test_writing_disabled_on_main_restart(): ) expected_cluster_coord3 = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_cluster_coord3, check_coordinator3) diff --git a/tests/e2e/high_availability/distributed_coords.py b/tests/e2e/high_availability/distributed_coords.py index 07b6eefe0..9fa654d68 100644 --- a/tests/e2e/high_availability/distributed_coords.py +++ b/tests/e2e/high_availability/distributed_coords.py @@ -17,7 +17,11 @@ import tempfile import interactive_mg_runner import pytest from common import connect, execute_and_fetch_all, safe_execute -from mg_utils import mg_sleep_and_assert, mg_sleep_and_assert_collection +from mg_utils import ( + mg_sleep_and_assert, + mg_sleep_and_assert_any_function, + mg_sleep_and_assert_collection, +) interactive_mg_runner.SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) interactive_mg_runner.PROJECT_DIR = os.path.normpath( @@ -138,8 +142,11 @@ def test_distributed_automatic_failover(): {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, ), ] - actual_data_on_main = sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) - assert actual_data_on_main == sorted(expected_data_on_main) + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert_collection(expected_data_on_main, retrieve_data_show_replicas) interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") @@ -149,12 +156,12 @@ def test_distributed_automatic_failover(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("coordinator_2", "127.0.0.1:10112", "", True, "coordinator"), - ("coordinator_3", "127.0.0.1:10113", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) @@ -202,5 +209,88 @@ def test_distributed_automatic_failover(): mg_sleep_and_assert_collection(expected_data_on_new_main_old_alive, retrieve_data_show_replicas) +def test_distributed_automatic_failover_after_coord_dies(): + safe_execute(shutil.rmtree, TEMP_DIR) + interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION) + + interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_3") + + interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") + + coord_cursor_1 = connect(host="localhost", port=7690).cursor() + + def show_instances_coord1(): + return sorted(list(execute_and_fetch_all(coord_cursor_1, "SHOW INSTANCES;"))) + + coord_cursor_2 = connect(host="localhost", port=7691).cursor() + + def show_instances_coord2(): + return sorted(list(execute_and_fetch_all(coord_cursor_2, "SHOW INSTANCES;"))) + + leader_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ] + mg_sleep_and_assert_any_function(leader_data, [show_instances_coord1, show_instances_coord2]) + + follower_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "main"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), # TODO: (andi) Will become unknown. + ] + mg_sleep_and_assert_any_function(leader_data, [show_instances_coord1, show_instances_coord2]) + mg_sleep_and_assert_any_function(follower_data, [show_instances_coord1, show_instances_coord2]) + + new_main_cursor = connect(host="localhost", port=7687).cursor() + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(new_main_cursor, "SHOW REPLICAS;"))) + + expected_data_on_new_main = [ + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + ), + ( + "instance_3", + "127.0.0.1:10003", + "sync", + {"ts": 0, "behind": None, "status": "invalid"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "invalid"}}, + ), + ] + mg_sleep_and_assert_collection(expected_data_on_new_main, retrieve_data_show_replicas) + + interactive_mg_runner.start(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") + expected_data_on_new_main_old_alive = [ + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + ), + ( + "instance_3", + "127.0.0.1:10003", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + ), + ] + + mg_sleep_and_assert_collection(expected_data_on_new_main_old_alive, retrieve_data_show_replicas) + + if __name__ == "__main__": sys.exit(pytest.main([__file__, "-rA"])) diff --git a/tests/e2e/high_availability/not_replicate_from_old_main.py b/tests/e2e/high_availability/not_replicate_from_old_main.py index c2cc93cb1..7ffffc04a 100644 --- a/tests/e2e/high_availability/not_replicate_from_old_main.py +++ b/tests/e2e/high_availability/not_replicate_from_old_main.py @@ -75,8 +75,11 @@ def test_replication_works_on_failover(): {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, ), ] - actual_data_on_main = sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) - assert actual_data_on_main == expected_data_on_main + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert(expected_data_on_main, retrieve_data_show_replicas) # 3 interactive_mg_runner.start_all_keep_others(MEMGRAPH_SECOND_CLUSTER_DESCRIPTION) @@ -200,9 +203,9 @@ def test_not_replicate_old_main_register_new_cluster(): return sorted(list(execute_and_fetch_all(first_cluster_coord_cursor, "SHOW INSTANCES;"))) expected_data_up_first_cluster = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_2", "", "127.0.0.1:10012", True, "main"), - ("shared_instance", "", "127.0.0.1:10011", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("shared_instance", "", "127.0.0.1:10011", "up", "replica"), ] mg_sleep_and_assert(expected_data_up_first_cluster, show_repl_cluster) @@ -254,9 +257,9 @@ def test_not_replicate_old_main_register_new_cluster(): return sorted(list(execute_and_fetch_all(second_cluster_coord_cursor, "SHOW INSTANCES;"))) expected_data_up_second_cluster = [ - ("coordinator_1", "127.0.0.1:10112", "", True, "coordinator"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), - ("shared_instance", "", "127.0.0.1:10011", True, "replica"), + ("coordinator_1", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ("shared_instance", "", "127.0.0.1:10011", "up", "replica"), ] mg_sleep_and_assert(expected_data_up_second_cluster, show_repl_cluster) diff --git a/tests/e2e/high_availability/single_coordinator.py b/tests/e2e/high_availability/single_coordinator.py index 2d81298b2..f9bf789b8 100644 --- a/tests/e2e/high_availability/single_coordinator.py +++ b/tests/e2e/high_availability/single_coordinator.py @@ -252,10 +252,10 @@ def test_replication_works_on_failover_replica_1_epoch_2_commits_away(data_recov return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", True, "main"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -269,10 +269,10 @@ def test_replication_works_on_failover_replica_1_epoch_2_commits_away(data_recov interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") new_expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "main"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(new_expected_data_on_coord, retrieve_data_show_instances) @@ -289,10 +289,10 @@ def test_replication_works_on_failover_replica_1_epoch_2_commits_away(data_recov interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") new_expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "main"), - ("instance_3", "", "127.0.0.1:10013", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "up", "replica"), ] mg_sleep_and_assert(new_expected_data_on_coord, retrieve_data_show_instances) @@ -482,11 +482,11 @@ def test_replication_works_on_failover_replica_2_epochs_more_commits_away(data_r return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), - ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ("instance_4", "", "127.0.0.1:10014", "up", "replica"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -506,11 +506,11 @@ def test_replication_works_on_failover_replica_2_epochs_more_commits_away(data_r # 6 expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), - ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "up", "replica"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -531,11 +531,11 @@ def test_replication_works_on_failover_replica_2_epochs_more_commits_away(data_r # 10 expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), - ("instance_4", "", "127.0.0.1:10014", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "up", "main"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -550,11 +550,11 @@ def test_replication_works_on_failover_replica_2_epochs_more_commits_away(data_r interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_2") expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), - ("instance_4", "", "127.0.0.1:10014", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "up", "main"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -573,11 +573,11 @@ def test_replication_works_on_failover_replica_2_epochs_more_commits_away(data_r interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "replica"), - ("instance_4", "", "127.0.0.1:10014", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "replica"), + ("instance_4", "", "127.0.0.1:10014", "up", "main"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -779,11 +779,11 @@ def test_replication_correct_replica_chosen_up_to_date_data(data_recovery): # TODO(antoniofilipovic) Before fixing durability, if this is removed we also have an issue. Check after fix expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), - ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ("instance_4", "", "127.0.0.1:10014", "up", "replica"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -811,11 +811,11 @@ def test_replication_correct_replica_chosen_up_to_date_data(data_recovery): # 7 expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", True, "main"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), - ("instance_4", "", "127.0.0.1:10014", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "up", "replica"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -837,11 +837,11 @@ def test_replication_correct_replica_chosen_up_to_date_data(data_recovery): interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_4") expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", True, "main"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), - ("instance_4", "", "127.0.0.1:10014", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "down", "unknown"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -854,11 +854,11 @@ def test_replication_correct_replica_chosen_up_to_date_data(data_recovery): # 11 expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), - ("instance_4", "", "127.0.0.1:10014", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "up", "main"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) @@ -910,8 +910,11 @@ def test_replication_works_on_failover_simple(): {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, ), ] - actual_data_on_main = sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) - assert actual_data_on_main == expected_data_on_main + + def main_cursor_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert_collection(expected_data_on_main, main_cursor_show_replicas) # 3 interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") @@ -923,10 +926,10 @@ def test_replication_works_on_failover_simple(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) @@ -1028,8 +1031,11 @@ def test_replication_works_on_replica_instance_restart(): {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, ), ] - actual_data_on_main = sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) - assert actual_data_on_main == expected_data_on_main + + def main_cursor_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert_collection(expected_data_on_main, main_cursor_show_replicas) # 3 coord_cursor = connect(host="localhost", port=7690).cursor() @@ -1040,10 +1046,10 @@ def test_replication_works_on_replica_instance_restart(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert_collection(expected_data_on_coord, retrieve_data_show_repl_cluster) @@ -1106,10 +1112,10 @@ def test_replication_works_on_replica_instance_restart(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) @@ -1154,10 +1160,10 @@ def test_show_instances(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data, show_repl_cluster) @@ -1177,20 +1183,20 @@ def test_show_instances(): interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_1") expected_data = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data, show_repl_cluster) interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_2") expected_data = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data, show_repl_cluster) @@ -1216,8 +1222,11 @@ def test_simple_automatic_failover(): {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, ), ] - actual_data_on_main = sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) - assert actual_data_on_main == sorted(expected_data_on_main) + + def main_cursor_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert_collection(expected_data_on_main, main_cursor_show_replicas) interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") @@ -1227,10 +1236,10 @@ def test_simple_automatic_failover(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_on_coord = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) @@ -1302,7 +1311,10 @@ def test_registering_replica_fails_endpoint_exists(): coord_cursor, "REGISTER INSTANCE instance_5 ON '127.0.0.1:10011' WITH '127.0.0.1:10005';", ) - assert str(e.value) == "Couldn't register replica instance since instance with such endpoint already exists!" + assert ( + str(e.value) + == "Couldn't register replica instance since instance with such coordinator endpoint already exists!" + ) def test_replica_instance_restarts(): @@ -1315,20 +1327,20 @@ def test_replica_instance_restarts(): return sorted(list(execute_and_fetch_all(cursor, "SHOW INSTANCES;"))) expected_data_up = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data_up, show_repl_cluster) interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_1") expected_data_down = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data_down, show_repl_cluster) @@ -1357,18 +1369,18 @@ def test_automatic_failover_main_back_as_replica(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_after_failover = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(expected_data_after_failover, retrieve_data_show_repl_cluster) expected_data_after_main_coming_back = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "main"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "replica"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "replica"), ] interactive_mg_runner.start(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") @@ -1396,20 +1408,20 @@ def test_automatic_failover_main_back_as_main(): return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) expected_data_all_down = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", False, "unknown"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] mg_sleep_and_assert(expected_data_all_down, retrieve_data_show_repl_cluster) interactive_mg_runner.start(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") expected_data_main_back = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", False, "unknown"), - ("instance_2", "", "127.0.0.1:10012", False, "unknown"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data_main_back, retrieve_data_show_repl_cluster) @@ -1424,10 +1436,10 @@ def test_automatic_failover_main_back_as_main(): interactive_mg_runner.start(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_2") expected_data_replicas_back = [ - ("coordinator_1", "127.0.0.1:10111", "", True, "coordinator"), - ("instance_1", "", "127.0.0.1:10011", True, "replica"), - ("instance_2", "", "127.0.0.1:10012", True, "replica"), - ("instance_3", "", "127.0.0.1:10013", True, "main"), + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), ] mg_sleep_and_assert(expected_data_replicas_back, retrieve_data_show_repl_cluster) diff --git a/tests/e2e/mg_utils.py b/tests/e2e/mg_utils.py index 3a475bf3c..7279f25f2 100644 --- a/tests/e2e/mg_utils.py +++ b/tests/e2e/mg_utils.py @@ -17,6 +17,28 @@ def mg_sleep_and_assert(expected_value, function_to_retrieve_data, max_duration= return result +def mg_sleep_and_assert_any_function( + expected_value, functions_to_retrieve_data, max_duration=20, time_between_attempt=0.2 +): + result = [f() for f in functions_to_retrieve_data] + if any((x == expected_value for x in result)): + return result + start_time = time.time() + while result != expected_value: + duration = time.time() - start_time + if duration > max_duration: + assert ( + False + ), f" mg_sleep_and_assert has tried for too long and did not get the expected result! Last result was: {result}" + + time.sleep(time_between_attempt) + result = [f() for f in functions_to_retrieve_data] + if any((x == expected_value for x in result)): + return result + + return result + + def mg_sleep_and_assert_collection( expected_value, function_to_retrieve_data, max_duration=20, time_between_attempt=0.2 ): From 1802dc93d12bc904bb239145fa3909c3a5ccb6e0 Mon Sep 17 00:00:00 2001 From: Andi <andi8647@gmail.com> Date: Tue, 5 Mar 2024 08:33:13 +0100 Subject: [PATCH 09/18] Improve Raft log serialization (#1778) --- src/coordination/CMakeLists.txt | 1 + src/coordination/coordinator_config.cpp | 54 +++++++ .../coordinator_state_machine.cpp | 66 ++++---- .../coordination/coordinator_config.hpp | 44 +---- .../nuraft/coordinator_state_machine.hpp | 2 +- .../include/nuraft/raft_log_action.hpp | 10 ++ src/replication_coordination_glue/mode.hpp | 25 +-- tests/unit/CMakeLists.txt | 7 + tests/unit/raft_log_serialization.cpp | 151 ++++++++++++++++++ 9 files changed, 265 insertions(+), 95 deletions(-) create mode 100644 src/coordination/coordinator_config.cpp create mode 100644 tests/unit/raft_log_serialization.cpp diff --git a/src/coordination/CMakeLists.txt b/src/coordination/CMakeLists.txt index 3e293e2e7..ef9376a70 100644 --- a/src/coordination/CMakeLists.txt +++ b/src/coordination/CMakeLists.txt @@ -23,6 +23,7 @@ target_sources(mg-coordination include/nuraft/coordinator_state_manager.hpp PRIVATE + coordinator_config.cpp coordinator_client.cpp coordinator_state.cpp coordinator_rpc.cpp diff --git a/src/coordination/coordinator_config.cpp b/src/coordination/coordinator_config.cpp new file mode 100644 index 000000000..a1147d3b6 --- /dev/null +++ b/src/coordination/coordinator_config.cpp @@ -0,0 +1,54 @@ +// 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 +// 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. + +#ifdef MG_ENTERPRISE + +#include "coordination/coordinator_config.hpp" + +namespace memgraph::coordination { + +void to_json(nlohmann::json &j, ReplClientInfo const &config) { + j = nlohmann::json{{"instance_name", config.instance_name}, + {"replication_mode", config.replication_mode}, + {"replication_ip_address", config.replication_ip_address}, + {"replication_port", config.replication_port}}; +} + +void from_json(nlohmann::json const &j, ReplClientInfo &config) { + config.instance_name = j.at("instance_name").get<std::string>(); + config.replication_mode = j.at("replication_mode").get<replication_coordination_glue::ReplicationMode>(); + config.replication_ip_address = j.at("replication_ip_address").get<std::string>(); + config.replication_port = j.at("replication_port").get<uint16_t>(); +} + +void to_json(nlohmann::json &j, CoordinatorClientConfig const &config) { + j = nlohmann::json{{"instance_name", config.instance_name}, + {"ip_address", config.ip_address}, + {"port", config.port}, + {"instance_health_check_frequency_sec", config.instance_health_check_frequency_sec.count()}, + {"instance_down_timeout_sec", config.instance_down_timeout_sec.count()}, + {"instance_get_uuid_frequency_sec", config.instance_get_uuid_frequency_sec.count()}, + {"replication_client_info", config.replication_client_info}}; +} + +void from_json(nlohmann::json const &j, CoordinatorClientConfig &config) { + config.instance_name = j.at("instance_name").get<std::string>(); + config.ip_address = j.at("ip_address").get<std::string>(); + config.port = j.at("port").get<uint16_t>(); + config.instance_health_check_frequency_sec = + std::chrono::seconds{j.at("instance_health_check_frequency_sec").get<int>()}; + config.instance_down_timeout_sec = std::chrono::seconds{j.at("instance_down_timeout_sec").get<int>()}; + config.instance_get_uuid_frequency_sec = std::chrono::seconds{j.at("instance_get_uuid_frequency_sec").get<int>()}; + config.replication_client_info = j.at("replication_client_info").get<ReplClientInfo>(); +} + +} // namespace memgraph::coordination +#endif diff --git a/src/coordination/coordinator_state_machine.cpp b/src/coordination/coordinator_state_machine.cpp index 6e2986ecf..564303f22 100644 --- a/src/coordination/coordinator_state_machine.cpp +++ b/src/coordination/coordinator_state_machine.cpp @@ -30,77 +30,67 @@ auto CoordinatorStateMachine::IsReplica(std::string_view instance_name) const -> return cluster_state_.IsReplica(instance_name); } -auto CoordinatorStateMachine::CreateLog(std::string_view log) -> ptr<buffer> { - ptr<buffer> log_buf = buffer::alloc(sizeof(uint32_t) + log.size()); +auto CoordinatorStateMachine::CreateLog(nlohmann::json &&log) -> ptr<buffer> { + auto const log_dump = log.dump(); + ptr<buffer> log_buf = buffer::alloc(sizeof(uint32_t) + log_dump.size()); buffer_serializer bs(log_buf); - bs.put_str(log.data()); + bs.put_str(log_dump); return log_buf; } auto CoordinatorStateMachine::SerializeRegisterInstance(CoordinatorClientConfig const &config) -> ptr<buffer> { - auto const str_log = fmt::format("{}*register", config.ToString()); - return CreateLog(str_log); + return CreateLog({{"action", RaftLogAction::REGISTER_REPLICATION_INSTANCE}, {"info", config}}); } auto CoordinatorStateMachine::SerializeUnregisterInstance(std::string_view instance_name) -> ptr<buffer> { - auto const str_log = fmt::format("{}*unregister", instance_name); - return CreateLog(str_log); + return CreateLog({{"action", RaftLogAction::UNREGISTER_REPLICATION_INSTANCE}, {"info", instance_name}}); } auto CoordinatorStateMachine::SerializeSetInstanceAsMain(std::string_view instance_name) -> ptr<buffer> { - auto const str_log = fmt::format("{}*promote", instance_name); - return CreateLog(str_log); + return CreateLog({{"action", RaftLogAction::SET_INSTANCE_AS_MAIN}, {"info", instance_name}}); } auto CoordinatorStateMachine::SerializeSetInstanceAsReplica(std::string_view instance_name) -> ptr<buffer> { - auto const str_log = fmt::format("{}*demote", instance_name); - return CreateLog(str_log); + return CreateLog({{"action", RaftLogAction::SET_INSTANCE_AS_REPLICA}, {"info", instance_name}}); } auto CoordinatorStateMachine::SerializeUpdateUUID(utils::UUID const &uuid) -> ptr<buffer> { - auto const str_log = fmt::format("{}*update_uuid", nlohmann::json{{"uuid", uuid}}.dump()); - return CreateLog(str_log); + return CreateLog({{"action", RaftLogAction::UPDATE_UUID}, {"info", uuid}}); } auto CoordinatorStateMachine::DecodeLog(buffer &data) -> std::pair<TRaftLog, RaftLogAction> { buffer_serializer bs(data); + auto const json = nlohmann::json::parse(bs.get_str()); - auto const log_str = bs.get_str(); - auto const sep = log_str.find('*'); - auto const action = log_str.substr(sep + 1); - auto const info = log_str.substr(0, sep); + auto const action = json["action"].get<RaftLogAction>(); + auto const &info = json["info"]; - if (action == "register") { - return {CoordinatorClientConfig::FromString(info), RaftLogAction::REGISTER_REPLICATION_INSTANCE}; + switch (action) { + case RaftLogAction::REGISTER_REPLICATION_INSTANCE: + return {info.get<CoordinatorClientConfig>(), action}; + case RaftLogAction::UPDATE_UUID: + return {info.get<utils::UUID>(), action}; + case RaftLogAction::UNREGISTER_REPLICATION_INSTANCE: + case RaftLogAction::SET_INSTANCE_AS_MAIN: + [[fallthrough]]; + case RaftLogAction::SET_INSTANCE_AS_REPLICA: + return {info.get<std::string>(), action}; } - if (action == "unregister") { - return {info, RaftLogAction::UNREGISTER_REPLICATION_INSTANCE}; - } - if (action == "promote") { - return {info, RaftLogAction::SET_INSTANCE_AS_MAIN}; - } - if (action == "demote") { - return {info, RaftLogAction::SET_INSTANCE_AS_REPLICA}; - } - if (action == "update_uuid") { - auto const json = nlohmann::json::parse(info); - return {json.at("uuid").get<utils::UUID>(), RaftLogAction::UPDATE_UUID}; - } - throw std::runtime_error("Unknown action"); } auto CoordinatorStateMachine::pre_commit(ulong const /*log_idx*/, buffer & /*data*/) -> ptr<buffer> { return nullptr; } auto CoordinatorStateMachine::commit(ulong const log_idx, buffer &data) -> ptr<buffer> { - buffer_serializer bs(data); - auto const [parsed_data, log_action] = DecodeLog(data); cluster_state_.DoAction(parsed_data, log_action); - last_committed_idx_ = log_idx; - // TODO: (andi) Don't return nullptr - return nullptr; + + // Return raft log number + ptr<buffer> ret = buffer::alloc(sizeof(log_idx)); + buffer_serializer bs_ret(ret); + bs_ret.put_u64(log_idx); + return ret; } auto CoordinatorStateMachine::commit_config(ulong const log_idx, ptr<cluster_config> & /*new_conf*/) -> void { diff --git a/src/coordination/include/coordination/coordinator_config.hpp b/src/coordination/include/coordination/coordinator_config.hpp index b0d7118ff..127a365eb 100644 --- a/src/coordination/include/coordination/coordinator_config.hpp +++ b/src/coordination/include/coordination/coordinator_config.hpp @@ -22,12 +22,12 @@ #include <string> #include <fmt/format.h> +#include "json/json.hpp" namespace memgraph::coordination { inline constexpr auto *kDefaultReplicationServerIp = "0.0.0.0"; -// TODO: (andi) JSON serialization for RAFT log. struct CoordinatorClientConfig { std::string instance_name; std::string ip_address; @@ -43,28 +43,11 @@ struct CoordinatorClientConfig { } struct ReplicationClientInfo { - // TODO: (andi) Do we even need here instance_name for this struct? std::string instance_name; replication_coordination_glue::ReplicationMode replication_mode{}; std::string replication_ip_address; uint16_t replication_port{}; - auto ToString() const -> std::string { - return fmt::format("{}#{}#{}#{}", instance_name, replication_ip_address, replication_port, - replication_coordination_glue::ReplicationModeToString(replication_mode)); - } - - // TODO: (andi) How can I make use of monadic parsers here? - static auto FromString(std::string_view log) -> ReplicationClientInfo { - ReplicationClientInfo replication_client_info; - auto splitted = utils::Split(log, "#"); - replication_client_info.instance_name = splitted[0]; - replication_client_info.replication_ip_address = splitted[1]; - replication_client_info.replication_port = std::stoi(splitted[2]); - replication_client_info.replication_mode = replication_coordination_glue::ReplicationModeFromString(splitted[3]); - return replication_client_info; - } - friend bool operator==(ReplicationClientInfo const &, ReplicationClientInfo const &) = default; }; @@ -79,25 +62,6 @@ struct CoordinatorClientConfig { std::optional<SSL> ssl; - auto ToString() const -> std::string { - return fmt::format("{}|{}|{}|{}|{}|{}|{}", instance_name, ip_address, port, - instance_health_check_frequency_sec.count(), instance_down_timeout_sec.count(), - instance_get_uuid_frequency_sec.count(), replication_client_info.ToString()); - } - - static auto FromString(std::string_view log) -> CoordinatorClientConfig { - CoordinatorClientConfig config; - auto splitted = utils::Split(log, "|"); - config.instance_name = splitted[0]; - config.ip_address = splitted[1]; - config.port = std::stoi(splitted[2]); - config.instance_health_check_frequency_sec = std::chrono::seconds(std::stoi(splitted[3])); - config.instance_down_timeout_sec = std::chrono::seconds(std::stoi(splitted[4])); - config.instance_get_uuid_frequency_sec = std::chrono::seconds(std::stoi(splitted[5])); - config.replication_client_info = ReplicationClientInfo::FromString(splitted[6]); - return config; - } - friend bool operator==(CoordinatorClientConfig const &, CoordinatorClientConfig const &) = default; }; @@ -119,5 +83,11 @@ struct CoordinatorServerConfig { friend bool operator==(CoordinatorServerConfig const &, CoordinatorServerConfig const &) = default; }; +void to_json(nlohmann::json &j, CoordinatorClientConfig const &config); +void from_json(nlohmann::json const &j, CoordinatorClientConfig &config); + +void to_json(nlohmann::json &j, ReplClientInfo const &config); +void from_json(nlohmann::json const &j, ReplClientInfo &config); + } // namespace memgraph::coordination #endif diff --git a/src/coordination/include/nuraft/coordinator_state_machine.hpp b/src/coordination/include/nuraft/coordinator_state_machine.hpp index aea21ab4e..516b8efc5 100644 --- a/src/coordination/include/nuraft/coordinator_state_machine.hpp +++ b/src/coordination/include/nuraft/coordinator_state_machine.hpp @@ -47,7 +47,7 @@ class CoordinatorStateMachine : public state_machine { auto IsMain(std::string_view instance_name) const -> bool; auto IsReplica(std::string_view instance_name) const -> bool; - static auto CreateLog(std::string_view log) -> ptr<buffer>; + static auto CreateLog(nlohmann::json &&log) -> ptr<buffer>; static auto SerializeRegisterInstance(CoordinatorClientConfig const &config) -> ptr<buffer>; static auto SerializeUnregisterInstance(std::string_view instance_name) -> ptr<buffer>; static auto SerializeSetInstanceAsMain(std::string_view instance_name) -> ptr<buffer>; diff --git a/src/coordination/include/nuraft/raft_log_action.hpp b/src/coordination/include/nuraft/raft_log_action.hpp index 399d33150..953049038 100644 --- a/src/coordination/include/nuraft/raft_log_action.hpp +++ b/src/coordination/include/nuraft/raft_log_action.hpp @@ -18,6 +18,8 @@ #include <cstdint> #include <string> +#include "json/json.hpp" + namespace memgraph::coordination { enum class RaftLogAction : uint8_t { @@ -28,6 +30,14 @@ enum class RaftLogAction : uint8_t { UPDATE_UUID }; +NLOHMANN_JSON_SERIALIZE_ENUM(RaftLogAction, { + {RaftLogAction::REGISTER_REPLICATION_INSTANCE, "register"}, + {RaftLogAction::UNREGISTER_REPLICATION_INSTANCE, "unregister"}, + {RaftLogAction::SET_INSTANCE_AS_MAIN, "promote"}, + {RaftLogAction::SET_INSTANCE_AS_REPLICA, "demote"}, + {RaftLogAction::UPDATE_UUID, "update_uuid"}, + }) + inline auto ParseRaftLogAction(std::string_view action) -> RaftLogAction { if (action == "register") { return RaftLogAction::REGISTER_REPLICATION_INSTANCE; diff --git a/src/replication_coordination_glue/mode.hpp b/src/replication_coordination_glue/mode.hpp index 3f27afb05..4ca98b3a0 100644 --- a/src/replication_coordination_glue/mode.hpp +++ b/src/replication_coordination_glue/mode.hpp @@ -16,28 +16,15 @@ #include <stdexcept> #include <string> +#include "json/json.hpp" + namespace memgraph::replication_coordination_glue { enum class ReplicationMode : std::uint8_t { SYNC, ASYNC }; -inline auto ReplicationModeToString(ReplicationMode mode) -> std::string { - switch (mode) { - case ReplicationMode::SYNC: - return "SYNC"; - case ReplicationMode::ASYNC: - return "ASYNC"; - } - throw std::invalid_argument("Invalid replication mode"); -} - -inline auto ReplicationModeFromString(std::string_view mode) -> ReplicationMode { - if (mode == "SYNC") { - return ReplicationMode::SYNC; - } - if (mode == "ASYNC") { - return ReplicationMode::ASYNC; - } - throw std::invalid_argument("Invalid replication mode"); -} +NLOHMANN_JSON_SERIALIZE_ENUM(ReplicationMode, { + {ReplicationMode::SYNC, "sync"}, + {ReplicationMode::ASYNC, "async"}, + }) } // namespace memgraph::replication_coordination_glue diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt index b92989f4e..f1afcdf15 100644 --- a/tests/unit/CMakeLists.txt +++ b/tests/unit/CMakeLists.txt @@ -438,3 +438,10 @@ add_unit_test(coordination_utils.cpp) target_link_libraries(${test_prefix}coordination_utils gflags mg-coordination mg-repl_coord_glue) target_include_directories(${test_prefix}coordination_utils PRIVATE ${CMAKE_SOURCE_DIR}/include) endif() + +# Test Raft log serialization +if(MG_ENTERPRISE) +add_unit_test(raft_log_serialization.cpp) +target_link_libraries(${test_prefix}raft_log_serialization gflags mg-coordination mg-repl_coord_glue) +target_include_directories(${test_prefix}raft_log_serialization PRIVATE ${CMAKE_SOURCE_DIR}/include) +endif() diff --git a/tests/unit/raft_log_serialization.cpp b/tests/unit/raft_log_serialization.cpp new file mode 100644 index 000000000..8550cf5b8 --- /dev/null +++ b/tests/unit/raft_log_serialization.cpp @@ -0,0 +1,151 @@ +// 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 +// 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 "coordination/coordinator_config.hpp" +#include "nuraft/coordinator_state_machine.hpp" +#include "nuraft/raft_log_action.hpp" +#include "utils/file.hpp" +#include "utils/uuid.hpp" + +#include <gflags/gflags.h> +#include <gtest/gtest.h> +#include "json/json.hpp" + +using memgraph::coordination::CoordinatorClientConfig; +using memgraph::coordination::CoordinatorStateMachine; +using memgraph::coordination::RaftLogAction; +using memgraph::coordination::ReplClientInfo; +using memgraph::replication_coordination_glue::ReplicationMode; +using memgraph::utils::UUID; + +class RaftLogSerialization : public ::testing::Test { + protected: + void SetUp() override {} + + void TearDown() override {} + + std::filesystem::path test_folder_{std::filesystem::temp_directory_path() / "MG_tests_unit_raft_log_serialization"}; +}; + +TEST_F(RaftLogSerialization, ReplClientInfo) { + ReplClientInfo info{"instance_name", ReplicationMode::SYNC, "127.0.0.1", 10111}; + + nlohmann::json j = info; + ReplClientInfo info2 = j.get<memgraph::coordination::ReplClientInfo>(); + + ASSERT_EQ(info, info2); +} + +TEST_F(RaftLogSerialization, CoordinatorClientConfig) { + CoordinatorClientConfig config{"instance3", + "127.0.0.1", + 10112, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10001}, + .ssl = std::nullopt}; + + nlohmann::json j = config; + CoordinatorClientConfig config2 = j.get<memgraph::coordination::CoordinatorClientConfig>(); + + ASSERT_EQ(config, config2); +} + +TEST_F(RaftLogSerialization, RaftLogActionRegister) { + auto action = RaftLogAction::REGISTER_REPLICATION_INSTANCE; + + nlohmann::json j = action; + RaftLogAction action2 = j.get<memgraph::coordination::RaftLogAction>(); + + ASSERT_EQ(action, action2); +} + +TEST_F(RaftLogSerialization, RaftLogActionUnregister) { + auto action = RaftLogAction::UNREGISTER_REPLICATION_INSTANCE; + + nlohmann::json j = action; + RaftLogAction action2 = j.get<memgraph::coordination::RaftLogAction>(); + + ASSERT_EQ(action, action2); +} + +TEST_F(RaftLogSerialization, RaftLogActionPromote) { + auto action = RaftLogAction::SET_INSTANCE_AS_MAIN; + + nlohmann::json j = action; + RaftLogAction action2 = j.get<memgraph::coordination::RaftLogAction>(); + + ASSERT_EQ(action, action2); +} + +TEST_F(RaftLogSerialization, RaftLogActionDemote) { + auto action = RaftLogAction::SET_INSTANCE_AS_REPLICA; + + nlohmann::json j = action; + RaftLogAction action2 = j.get<memgraph::coordination::RaftLogAction>(); + + ASSERT_EQ(action, action2); +} + +TEST_F(RaftLogSerialization, RaftLogActionUpdateUUID) { + auto action = RaftLogAction::UPDATE_UUID; + + nlohmann::json j = action; + RaftLogAction action2 = j.get<memgraph::coordination::RaftLogAction>(); + + ASSERT_EQ(action, action2); +} + +TEST_F(RaftLogSerialization, RegisterInstance) { + CoordinatorClientConfig config{"instance3", + "127.0.0.1", + 10112, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10001}, + .ssl = std::nullopt}; + + auto buffer = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + ASSERT_EQ(action, RaftLogAction::REGISTER_REPLICATION_INSTANCE); + ASSERT_EQ(config, std::get<CoordinatorClientConfig>(payload)); +} + +TEST_F(RaftLogSerialization, UnregisterInstance) { + auto buffer = CoordinatorStateMachine::SerializeUnregisterInstance("instance3"); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + ASSERT_EQ(action, RaftLogAction::UNREGISTER_REPLICATION_INSTANCE); + ASSERT_EQ("instance3", std::get<std::string>(payload)); +} + +TEST_F(RaftLogSerialization, SetInstanceAsMain) { + auto buffer = CoordinatorStateMachine::SerializeSetInstanceAsMain("instance3"); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + ASSERT_EQ(action, RaftLogAction::SET_INSTANCE_AS_MAIN); + ASSERT_EQ("instance3", std::get<std::string>(payload)); +} + +TEST_F(RaftLogSerialization, SetInstanceAsReplica) { + auto buffer = CoordinatorStateMachine::SerializeSetInstanceAsReplica("instance3"); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + ASSERT_EQ(action, RaftLogAction::SET_INSTANCE_AS_REPLICA); + ASSERT_EQ("instance3", std::get<std::string>(payload)); +} + +TEST_F(RaftLogSerialization, UpdateUUID) { + UUID uuid; + auto buffer = CoordinatorStateMachine::SerializeUpdateUUID(uuid); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + ASSERT_EQ(action, RaftLogAction::UPDATE_UUID); + ASSERT_EQ(uuid, std::get<UUID>(payload)); +} From d4d4660af01823422b906e58f0dd261991797705 Mon Sep 17 00:00:00 2001 From: Antonio Filipovic <61245998+antoniofilipovic@users.noreply.github.com> Date: Tue, 5 Mar 2024 17:51:14 +0100 Subject: [PATCH 10/18] Add force sync REPLICA with MAIN (#1777) --- src/coordination/coordinator_instance.cpp | 3 +- src/dbms/inmemory/replication_handlers.cpp | 79 ++++- src/dbms/inmemory/replication_handlers.hpp | 3 + src/query/interpreter.cpp | 2 +- .../replication_handler.hpp | 5 + .../replication_handler.cpp | 3 +- src/storage/v2/durability/durability.cpp | 23 +- src/storage/v2/inmemory/storage.cpp | 2 +- .../v2/replication/replication_client.cpp | 55 +++- .../v2/replication/replication_client.hpp | 7 + src/storage/v2/replication/rpc.cpp | 43 +++ src/storage/v2/replication/rpc.hpp | 38 +++ src/utils/functional.hpp | 4 +- src/utils/typeinfo.hpp | 2 + .../high_availability/single_coordinator.py | 276 +++++++++++++++++- tests/unit/coordination_utils.cpp | 46 ++- 16 files changed, 525 insertions(+), 66 deletions(-) diff --git a/src/coordination/coordinator_instance.cpp b/src/coordination/coordinator_instance.cpp index afbcaa7d8..9a00ca87c 100644 --- a/src/coordination/coordinator_instance.cpp +++ b/src/coordination/coordinator_instance.cpp @@ -100,8 +100,7 @@ auto CoordinatorInstance::ShowInstances() const -> std::vector<InstanceStatus> { .health = "unknown"}; // TODO: (andi) Get this info from RAFT and test it or when we will move // CoordinatorState to every instance, we can be smarter about this using our RPC. }; - - auto instances_status = utils::fmap(coord_instance_to_status, raft_state_.GetAllCoordinators()); + auto instances_status = utils::fmap(raft_state_.GetAllCoordinators(), coord_instance_to_status); if (raft_state_.IsLeader()) { auto const stringify_repl_role = [this](ReplicationInstance const &instance) -> std::string { diff --git a/src/dbms/inmemory/replication_handlers.cpp b/src/dbms/inmemory/replication_handlers.cpp index 8339b65b4..6a78977bb 100644 --- a/src/dbms/inmemory/replication_handlers.cpp +++ b/src/dbms/inmemory/replication_handlers.cpp @@ -118,9 +118,14 @@ void InMemoryReplicationHandlers::Register(dbms::DbmsHandler *dbms_handler, repl }); server.rpc_server_.Register<replication_coordination_glue::SwapMainUUIDRpc>( [&data, dbms_handler](auto *req_reader, auto *res_builder) { - spdlog::debug("Received SwapMainUUIDHandler"); + spdlog::debug("Received SwapMainUUIDRpc"); InMemoryReplicationHandlers::SwapMainUUIDHandler(dbms_handler, data, req_reader, res_builder); }); + server.rpc_server_.Register<storage::replication::ForceResetStorageRpc>( + [&data, dbms_handler](auto *req_reader, auto *res_builder) { + spdlog::debug("Received ForceResetStorageRpc"); + InMemoryReplicationHandlers::ForceResetStorageHandler(dbms_handler, data.uuid_, req_reader, res_builder); + }); } void InMemoryReplicationHandlers::SwapMainUUIDHandler(dbms::DbmsHandler *dbms_handler, @@ -329,6 +334,78 @@ void InMemoryReplicationHandlers::SnapshotHandler(dbms::DbmsHandler *dbms_handle spdlog::debug("Replication recovery from snapshot finished!"); } +void InMemoryReplicationHandlers::ForceResetStorageHandler(dbms::DbmsHandler *dbms_handler, + const std::optional<utils::UUID> ¤t_main_uuid, + slk::Reader *req_reader, slk::Builder *res_builder) { + storage::replication::ForceResetStorageReq req; + slk::Load(&req, req_reader); + auto db_acc = GetDatabaseAccessor(dbms_handler, req.db_uuid); + if (!db_acc) { + storage::replication::ForceResetStorageRes res{false, 0}; + slk::Save(res, res_builder); + return; + } + if (!current_main_uuid.has_value() || req.main_uuid != current_main_uuid) [[unlikely]] { + LogWrongMain(current_main_uuid, req.main_uuid, storage::replication::SnapshotReq::kType.name); + storage::replication::ForceResetStorageRes res{false, 0}; + slk::Save(res, res_builder); + return; + } + + storage::replication::Decoder decoder(req_reader); + + auto *storage = static_cast<storage::InMemoryStorage *>(db_acc->get()->storage()); + + auto storage_guard = std::unique_lock{storage->main_lock_}; + + // Clear the database + storage->vertices_.clear(); + storage->edges_.clear(); + storage->commit_log_.reset(); + storage->commit_log_.emplace(); + + storage->constraints_.existence_constraints_ = std::make_unique<storage::ExistenceConstraints>(); + storage->constraints_.unique_constraints_ = std::make_unique<storage::InMemoryUniqueConstraints>(); + storage->indices_.label_index_ = std::make_unique<storage::InMemoryLabelIndex>(); + storage->indices_.label_property_index_ = std::make_unique<storage::InMemoryLabelPropertyIndex>(); + + // Fine since we will force push when reading from WAL just random epoch with 0 timestamp, as it should be if it + // acted as MAIN before + storage->repl_storage_state_.epoch_.SetEpoch(std::string(utils::UUID{})); + storage->repl_storage_state_.last_commit_timestamp_ = 0; + + storage->repl_storage_state_.history.clear(); + storage->vertex_id_ = 0; + storage->edge_id_ = 0; + storage->timestamp_ = storage::kTimestampInitialId; + + storage->CollectGarbage<true>(std::move(storage_guard), false); + storage->vertices_.run_gc(); + storage->edges_.run_gc(); + + storage::replication::ForceResetStorageRes res{true, storage->repl_storage_state_.last_commit_timestamp_.load()}; + slk::Save(res, res_builder); + + spdlog::trace("Deleting old snapshot files."); + // Delete other durability files + auto snapshot_files = storage::durability::GetSnapshotFiles(storage->recovery_.snapshot_directory_, storage->uuid_); + for (const auto &[path, uuid, _] : snapshot_files) { + spdlog::trace("Deleting snapshot file {}", path); + storage->file_retainer_.DeleteFile(path); + } + + spdlog::trace("Deleting old WAL files."); + auto wal_files = storage::durability::GetWalFiles(storage->recovery_.wal_directory_, storage->uuid_); + if (wal_files) { + for (const auto &wal_file : *wal_files) { + spdlog::trace("Deleting WAL file {}", wal_file.path); + storage->file_retainer_.DeleteFile(wal_file.path); + } + + storage->wal_file_.reset(); + } +} + void InMemoryReplicationHandlers::WalFilesHandler(dbms::DbmsHandler *dbms_handler, const std::optional<utils::UUID> ¤t_main_uuid, slk::Reader *req_reader, slk::Builder *res_builder) { diff --git a/src/dbms/inmemory/replication_handlers.hpp b/src/dbms/inmemory/replication_handlers.hpp index 4406b8338..aaa2d0755 100644 --- a/src/dbms/inmemory/replication_handlers.hpp +++ b/src/dbms/inmemory/replication_handlers.hpp @@ -48,6 +48,9 @@ class InMemoryReplicationHandlers { static void SwapMainUUIDHandler(dbms::DbmsHandler *dbms_handler, replication::RoleReplicaData &role_replica_data, slk::Reader *req_reader, slk::Builder *res_builder); + static void ForceResetStorageHandler(dbms::DbmsHandler *dbms_handler, + const std::optional<utils::UUID> ¤t_main_uuid, slk::Reader *req_reader, + slk::Builder *res_builder); static void LoadWal(storage::InMemoryStorage *storage, storage::replication::Decoder *decoder); diff --git a/src/query/interpreter.cpp b/src/query/interpreter.cpp index f7213bed1..e6d39ab9a 100644 --- a/src/query/interpreter.cpp +++ b/src/query/interpreter.cpp @@ -1263,7 +1263,7 @@ Callback HandleCoordinatorQuery(CoordinatorQuery *coordinator_query, const Param TypedValue{status.coord_socket_address}, TypedValue{status.health}, TypedValue{status.cluster_role}}; }; - return utils::fmap(converter, instances); + return utils::fmap(instances, converter); }; return callback; } diff --git a/src/replication_handler/include/replication_handler/replication_handler.hpp b/src/replication_handler/include/replication_handler/replication_handler.hpp index d5c2bfa71..e1da19bfa 100644 --- a/src/replication_handler/include/replication_handler/replication_handler.hpp +++ b/src/replication_handler/include/replication_handler/replication_handler.hpp @@ -210,8 +210,13 @@ struct ReplicationHandler : public memgraph::query::ReplicationQueryHandler { auto client = std::make_unique<storage::ReplicationStorageClient>(*instance_client_ptr, main_uuid); client->Start(storage, std::move(db_acc)); bool const success = std::invoke([state = client->State()]() { + // We force sync replicas in other situation if (state == storage::replication::ReplicaState::DIVERGED_FROM_MAIN) { +#ifdef MG_ENTERPRISE + return FLAGS_coordinator_server_port != 0; +#else return false; +#endif } return true; }); diff --git a/src/replication_handler/replication_handler.cpp b/src/replication_handler/replication_handler.cpp index fc3dd3da4..34ccdfc99 100644 --- a/src/replication_handler/replication_handler.cpp +++ b/src/replication_handler/replication_handler.cpp @@ -271,8 +271,7 @@ auto ReplicationHandler::GetDatabasesHistories() -> replication_coordination_glu dbms_handler_.ForEach([&results](memgraph::dbms::DatabaseAccess db_acc) { auto &repl_storage_state = db_acc->storage()->repl_storage_state_; - std::vector<std::pair<std::string, uint64_t>> history = utils::fmap( - [](const auto &elem) { return std::make_pair(elem.first, elem.second); }, repl_storage_state.history); + std::vector<std::pair<std::string, uint64_t>> history = utils::fmap(repl_storage_state.history); history.emplace_back(std::string(repl_storage_state.epoch_.id()), repl_storage_state.last_commit_timestamp_.load()); replication_coordination_glue::DatabaseHistory repl{ diff --git a/src/storage/v2/durability/durability.cpp b/src/storage/v2/durability/durability.cpp index b81357902..a83313820 100644 --- a/src/storage/v2/durability/durability.cpp +++ b/src/storage/v2/durability/durability.cpp @@ -358,7 +358,6 @@ std::optional<RecoveryInfo> Recovery::RecoverData(std::string *uuid, Replication spdlog::warn(utils::MessageWithLink("No snapshot or WAL file found.", "https://memgr.ph/durability")); return std::nullopt; } - // TODO(antoniofilipovic) What is the logic here? std::sort(wal_files.begin(), wal_files.end()); // UUID used for durability is the UUID of the last WAL file. // Same for the epoch id. @@ -437,17 +436,13 @@ std::optional<RecoveryInfo> Recovery::RecoverData(std::string *uuid, Replication last_loaded_timestamp.emplace(recovery_info.next_timestamp - 1); } - bool epoch_history_empty = epoch_history->empty(); - bool epoch_not_recorded = !epoch_history_empty && epoch_history->back().first != wal_file.epoch_id; auto last_loaded_timestamp_value = last_loaded_timestamp.value_or(0); - - if (epoch_history_empty || epoch_not_recorded) { - epoch_history->emplace_back(std::string(wal_file.epoch_id), last_loaded_timestamp_value); - } - - auto last_epoch_updated = !epoch_history_empty && epoch_history->back().first == wal_file.epoch_id && - epoch_history->back().second < last_loaded_timestamp_value; - if (last_epoch_updated) { + if (epoch_history->empty() || epoch_history->back().first != wal_file.epoch_id) { + // no history or new epoch, add it + epoch_history->emplace_back(wal_file.epoch_id, last_loaded_timestamp_value); + repl_storage_state.epoch_.SetEpoch(wal_file.epoch_id); + } else if (epoch_history->back().second < last_loaded_timestamp_value) { + // existing epoch, update with newer timestamp epoch_history->back().second = last_loaded_timestamp_value; } @@ -469,11 +464,11 @@ std::optional<RecoveryInfo> Recovery::RecoverData(std::string *uuid, Replication memgraph::metrics::Measure(memgraph::metrics::SnapshotRecoveryLatency_us, std::chrono::duration_cast<std::chrono::microseconds>(timer.Elapsed()).count()); - spdlog::info("Set epoch id: {} with commit timestamp {}", std::string(repl_storage_state.epoch_.id()), - repl_storage_state.last_commit_timestamp_); + spdlog::trace("Set epoch id: {} with commit timestamp {}", std::string(repl_storage_state.epoch_.id()), + repl_storage_state.last_commit_timestamp_); std::for_each(repl_storage_state.history.begin(), repl_storage_state.history.end(), [](auto &history) { - spdlog::info("epoch id: {} with commit timestamp {}", std::string(history.first), history.second); + spdlog::trace("epoch id: {} with commit timestamp {}", std::string(history.first), history.second); }); return recovery_info; } diff --git a/src/storage/v2/inmemory/storage.cpp b/src/storage/v2/inmemory/storage.cpp index 1437524d6..3a4fa9b91 100644 --- a/src/storage/v2/inmemory/storage.cpp +++ b/src/storage/v2/inmemory/storage.cpp @@ -109,7 +109,7 @@ InMemoryStorage::InMemoryStorage(Config config) timestamp_ = std::max(timestamp_, info->next_timestamp); if (info->last_commit_timestamp) { repl_storage_state_.last_commit_timestamp_ = *info->last_commit_timestamp; - spdlog::info("Recovering last commit timestamp {}", *info->last_commit_timestamp); + spdlog::trace("Recovering last commit timestamp {}", *info->last_commit_timestamp); } } } else if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED || diff --git a/src/storage/v2/replication/replication_client.cpp b/src/storage/v2/replication/replication_client.cpp index bd2c2cc7d..fb332672a 100644 --- a/src/storage/v2/replication/replication_client.cpp +++ b/src/storage/v2/replication/replication_client.cpp @@ -76,13 +76,37 @@ void ReplicationStorageClient::UpdateReplicaState(Storage *storage, DatabaseAcce } } if (branching_point) { - spdlog::error( - "You cannot register Replica {} to this Main because at one point " - "Replica {} acted as the Main instance. Both the Main and Replica {} " - "now hold unique data. Please resolve data conflicts and start the " - "replication on a clean instance.", - client_.name_, client_.name_, client_.name_); - replica_state_.WithLock([](auto &val) { val = replication::ReplicaState::DIVERGED_FROM_MAIN; }); + auto replica_state = replica_state_.Lock(); + if (*replica_state == replication::ReplicaState::DIVERGED_FROM_MAIN) { + return; + } + *replica_state = replication::ReplicaState::DIVERGED_FROM_MAIN; + + auto log_error = [client_name = client_.name_]() { + spdlog::error( + "You cannot register Replica {} to this Main because at one point " + "Replica {} acted as the Main instance. Both the Main and Replica {} " + "now hold unique data. Please resolve data conflicts and start the " + "replication on a clean instance.", + client_name, client_name, client_name); + }; +#ifdef MG_ENTERPRISE + if (!FLAGS_coordinator_server_port) { + log_error(); + return; + } + client_.thread_pool_.AddTask([storage, gk = std::move(db_acc), this] { + const auto [success, timestamp] = this->ForceResetStorage(storage); + if (success) { + spdlog::info("Successfully reset storage of REPLICA {} to timestamp {}.", client_.name_, timestamp); + return; + } + spdlog::error("You cannot register REPLICA {} to this MAIN because MAIN couldn't reset REPLICA's storage.", + client_.name_); + }); +#else + log_error(); +#endif return; } @@ -265,8 +289,6 @@ void ReplicationStorageClient::RecoverReplica(uint64_t replica_commit, memgraph: spdlog::debug("Starting replica recovery"); auto *mem_storage = static_cast<InMemoryStorage *>(storage); - // TODO(antoniofilipovic): Can we get stuck here in while loop if replica commit timestamp is not updated when using - // only snapshot while (true) { auto file_locker = mem_storage->file_retainer_.AddLocker(); @@ -335,6 +357,21 @@ void ReplicationStorageClient::RecoverReplica(uint64_t replica_commit, memgraph: } } +std::pair<bool, uint64_t> ReplicationStorageClient::ForceResetStorage(memgraph::storage::Storage *storage) { + utils::OnScopeExit set_to_maybe_behind{ + [this]() { replica_state_.WithLock([](auto &state) { state = replication::ReplicaState::MAYBE_BEHIND; }); }}; + try { + auto stream{client_.rpc_client_.Stream<replication::ForceResetStorageRpc>(main_uuid_, storage->uuid())}; + const auto res = stream.AwaitResponse(); + return std::pair{res.success, res.current_commit_timestamp}; + } catch (const rpc::RpcFailedException &) { + spdlog::error( + utils::MessageWithLink("Couldn't ForceReset data to {}.", client_.name_, "https://memgr.ph/replication")); + } + + return {false, 0}; +} + ////// ReplicaStream ////// ReplicaStream::ReplicaStream(Storage *storage, rpc::Client &rpc_client, const uint64_t current_seq_num, utils::UUID main_uuid) diff --git a/src/storage/v2/replication/replication_client.hpp b/src/storage/v2/replication/replication_client.hpp index 3352bab65..063501111 100644 --- a/src/storage/v2/replication/replication_client.hpp +++ b/src/storage/v2/replication/replication_client.hpp @@ -188,6 +188,13 @@ class ReplicationStorageClient { */ void UpdateReplicaState(Storage *storage, DatabaseAccessProtector db_acc); + /** + * @brief Forcefully reset storage to as it is when started from scratch. + * + * @param storage pointer to the storage associated with the client + */ + std::pair<bool, uint64_t> ForceResetStorage(Storage *storage); + void LogRpcFailure(); /** diff --git a/src/storage/v2/replication/rpc.cpp b/src/storage/v2/replication/rpc.cpp index f523bb5d7..71a9ca65c 100644 --- a/src/storage/v2/replication/rpc.cpp +++ b/src/storage/v2/replication/rpc.cpp @@ -59,6 +59,19 @@ void TimestampRes::Save(const TimestampRes &self, memgraph::slk::Builder *builde memgraph::slk::Save(self, builder); } void TimestampRes::Load(TimestampRes *self, memgraph::slk::Reader *reader) { memgraph::slk::Load(self, reader); } + +void ForceResetStorageReq::Save(const ForceResetStorageReq &self, memgraph::slk::Builder *builder) { + memgraph::slk::Save(self, builder); +} +void ForceResetStorageReq::Load(ForceResetStorageReq *self, memgraph::slk::Reader *reader) { + memgraph::slk::Load(self, reader); +} +void ForceResetStorageRes::Save(const ForceResetStorageRes &self, memgraph::slk::Builder *builder) { + memgraph::slk::Save(self, builder); +} +void ForceResetStorageRes::Load(ForceResetStorageRes *self, memgraph::slk::Reader *reader) { + memgraph::slk::Load(self, reader); +} } // namespace storage::replication constexpr utils::TypeInfo storage::replication::AppendDeltasReq::kType{utils::TypeId::REP_APPEND_DELTAS_REQ, @@ -97,6 +110,12 @@ constexpr utils::TypeInfo storage::replication::TimestampReq::kType{utils::TypeI constexpr utils::TypeInfo storage::replication::TimestampRes::kType{utils::TypeId::REP_TIMESTAMP_RES, "TimestampRes", nullptr}; +constexpr utils::TypeInfo storage::replication::ForceResetStorageReq::kType{utils::TypeId::REP_FORCE_RESET_STORAGE_REQ, + "ForceResetStorageReq", nullptr}; + +constexpr utils::TypeInfo storage::replication::ForceResetStorageRes::kType{utils::TypeId::REP_FORCE_RESET_STORAGE_RES, + "ForceResetStorageRes", nullptr}; + // Autogenerated SLK serialization code namespace slk { // Serialize code for TimestampRes @@ -255,6 +274,30 @@ void Load(memgraph::storage::replication::AppendDeltasReq *self, memgraph::slk:: memgraph::slk::Load(&self->seq_num, reader); } +// Serialize code for ForceResetStorageReq + +void Save(const memgraph::storage::replication::ForceResetStorageReq &self, memgraph::slk::Builder *builder) { + memgraph::slk::Save(self.main_uuid, builder); + memgraph::slk::Save(self.db_uuid, builder); +} + +void Load(memgraph::storage::replication::ForceResetStorageReq *self, memgraph::slk::Reader *reader) { + memgraph::slk::Load(&self->main_uuid, reader); + memgraph::slk::Load(&self->db_uuid, reader); +} + +// Serialize code for ForceResetStorageRes + +void Save(const memgraph::storage::replication::ForceResetStorageRes &self, memgraph::slk::Builder *builder) { + memgraph::slk::Save(self.success, builder); + memgraph::slk::Save(self.current_commit_timestamp, builder); +} + +void Load(memgraph::storage::replication::ForceResetStorageRes *self, memgraph::slk::Reader *reader) { + memgraph::slk::Load(&self->success, reader); + memgraph::slk::Load(&self->current_commit_timestamp, reader); +} + // Serialize SalientConfig void Save(const memgraph::storage::SalientConfig &self, memgraph::slk::Builder *builder) { diff --git a/src/storage/v2/replication/rpc.hpp b/src/storage/v2/replication/rpc.hpp index 67f98d0ae..fb19d82f2 100644 --- a/src/storage/v2/replication/rpc.hpp +++ b/src/storage/v2/replication/rpc.hpp @@ -210,6 +210,36 @@ struct TimestampRes { using TimestampRpc = rpc::RequestResponse<TimestampReq, TimestampRes>; +struct ForceResetStorageReq { + static const utils::TypeInfo kType; + static const utils::TypeInfo &GetTypeInfo() { return kType; } + + static void Load(ForceResetStorageReq *self, memgraph::slk::Reader *reader); + static void Save(const ForceResetStorageReq &self, memgraph::slk::Builder *builder); + ForceResetStorageReq() = default; + explicit ForceResetStorageReq(const utils::UUID &main_uuid, const utils::UUID &db_uuid) + : main_uuid{main_uuid}, db_uuid{db_uuid} {} + + utils::UUID main_uuid; + utils::UUID db_uuid; +}; + +struct ForceResetStorageRes { + static const utils::TypeInfo kType; + static const utils::TypeInfo &GetTypeInfo() { return kType; } + + static void Load(ForceResetStorageRes *self, memgraph::slk::Reader *reader); + static void Save(const ForceResetStorageRes &self, memgraph::slk::Builder *builder); + ForceResetStorageRes() = default; + ForceResetStorageRes(bool success, uint64_t current_commit_timestamp) + : success(success), current_commit_timestamp(current_commit_timestamp) {} + + bool success; + uint64_t current_commit_timestamp; +}; + +using ForceResetStorageRpc = rpc::RequestResponse<ForceResetStorageReq, ForceResetStorageRes>; + } // namespace memgraph::storage::replication // SLK serialization declarations @@ -267,4 +297,12 @@ void Save(const memgraph::storage::SalientConfig &self, memgraph::slk::Builder * void Load(memgraph::storage::SalientConfig *self, memgraph::slk::Reader *reader); +void Save(const memgraph::storage::replication::ForceResetStorageReq &self, memgraph::slk::Builder *builder); + +void Load(memgraph::storage::replication::ForceResetStorageReq *self, memgraph::slk::Reader *reader); + +void Save(const memgraph::storage::replication::ForceResetStorageRes &self, memgraph::slk::Builder *builder); + +void Load(memgraph::storage::replication::ForceResetStorageRes *self, memgraph::slk::Reader *reader); + } // namespace memgraph::slk diff --git a/src/utils/functional.hpp b/src/utils/functional.hpp index f5242944a..fe60edc5c 100644 --- a/src/utils/functional.hpp +++ b/src/utils/functional.hpp @@ -19,9 +19,9 @@ namespace memgraph::utils { template <template <typename, typename...> class Container, typename T, typename Allocator = std::allocator<T>, - typename F, typename R = std::invoke_result_t<F, T>> + typename F = std::identity, typename R = std::decay_t<std::invoke_result_t<F, T>>> requires ranges::range<Container<T, Allocator>> && - (!std::same_as<Container<T, Allocator>, std::string>)auto fmap(F &&f, const Container<T, Allocator> &v) + (!std::same_as<Container<T, Allocator>, std::string>)auto fmap(const Container<T, Allocator> &v, F &&f = {}) -> std::vector<R> { return v | ranges::views::transform(std::forward<F>(f)) | ranges::to<std::vector<R>>(); } diff --git a/src/utils/typeinfo.hpp b/src/utils/typeinfo.hpp index aadc8a07b..592473b8b 100644 --- a/src/utils/typeinfo.hpp +++ b/src/utils/typeinfo.hpp @@ -99,6 +99,8 @@ enum class TypeId : uint64_t { REP_DROP_AUTH_DATA_RES, REP_TRY_SET_MAIN_UUID_REQ, REP_TRY_SET_MAIN_UUID_RES, + REP_FORCE_RESET_STORAGE_REQ, + REP_FORCE_RESET_STORAGE_RES, // Coordinator COORD_FAILOVER_REQ, diff --git a/tests/e2e/high_availability/single_coordinator.py b/tests/e2e/high_availability/single_coordinator.py index f9bf789b8..7335d2847 100644 --- a/tests/e2e/high_availability/single_coordinator.py +++ b/tests/e2e/high_availability/single_coordinator.py @@ -596,9 +596,8 @@ def test_replication_works_on_failover_replica_2_epochs_more_commits_away(data_r mg_sleep_and_assert(5, get_vertex_count) -@pytest.mark.parametrize("data_recovery", ["false"]) +@pytest.mark.parametrize("data_recovery", ["true"]) def test_replication_forcefully_works_on_failover_replica_misses_epoch(data_recovery): - # TODO(antoniofilipovic) Test should pass when logic is added # Goal of this test is to check the replication works forcefully if replica misses epoch # 1. We start all replicas, main and coordinator manually # 2. We check that main has correct state @@ -606,9 +605,9 @@ def test_replication_forcefully_works_on_failover_replica_misses_epoch(data_reco # 4. Expect data to be copied on all replicas # 5. Kill instance_1 ( this one will miss complete epoch) # 6. Kill main (instance_3) - # 7. Instance_2 or instance_4 new main - # 8. New main commits - # 9. Instance_2 down (not main) + # 7. Instance_2 + # 8. Instance_2 commits + # 9. Instance_2 down # 10. instance_4 down # 11. Instance 1 up (missed epoch) # 12 Instance 1 new main @@ -617,7 +616,272 @@ def test_replication_forcefully_works_on_failover_replica_misses_epoch(data_reco temp_dir = tempfile.TemporaryDirectory().name - pass + MEMGRAPH_INNER_INSTANCES_DESCRIPTION = { + "instance_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7688", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10011", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_1.log", + "data_directory": f"{temp_dir}/instance_1", + "setup_queries": [], + }, + "instance_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7689", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10012", + "--replication-restore-state-on-startup", + "true", + f"--data-recovery-on-startup={data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_2.log", + "data_directory": f"{temp_dir}/instance_2", + "setup_queries": [], + }, + "instance_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7687", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10013", + "--replication-restore-state-on-startup", + "true", + "--data-recovery-on-startup", + f"{data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_3.log", + "data_directory": f"{temp_dir}/instance_3", + "setup_queries": [], + }, + "instance_4": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7691", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10014", + "--replication-restore-state-on-startup", + "true", + "--data-recovery-on-startup", + f"{data_recovery}", + "--storage-recover-on-startup=false", + ], + "log_file": "instance_4.log", + "data_directory": f"{temp_dir}/instance_4", + "setup_queries": [], + }, + "coordinator": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7690", + "--log-level=TRACE", + "--raft-server-id=1", + "--raft-server-port=10111", + ], + "log_file": "coordinator.log", + "setup_queries": [ + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001';", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002';", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003';", + "REGISTER INSTANCE instance_4 ON '127.0.0.1:10014' WITH '127.0.0.1:10004';", + "SET INSTANCE instance_3 TO MAIN", + ], + }, + } + + # 1 + + interactive_mg_runner.start_all(MEMGRAPH_INNER_INSTANCES_DESCRIPTION) + + # 2 + + main_cursor = connect(host="localhost", port=7687).cursor() + expected_data_on_main = [ + ( + "instance_1", + "127.0.0.1:10001", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ( + "instance_4", + "127.0.0.1:10004", + "sync", + {"behind": None, "status": "ready", "ts": 0}, + {"memgraph": {"behind": 0, "status": "ready", "ts": 0}}, + ), + ] + + main_cursor = connect(host="localhost", port=7687).cursor() + + def retrieve_data_show_replicas(): + return sorted(list(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))) + + mg_sleep_and_assert_collection(expected_data_on_main, retrieve_data_show_replicas) + + coord_cursor = connect(host="localhost", port=7690).cursor() + + def retrieve_data_show_instances(): + return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ("instance_4", "", "127.0.0.1:10014", "up", "replica"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 3 + + execute_and_fetch_all(main_cursor, "CREATE (:Epoch1Vertex {prop:1});") + execute_and_fetch_all(main_cursor, "CREATE (:Epoch1Vertex {prop:2});") + + # 4 + instance_1_cursor = connect(host="localhost", port=7688).cursor() + instance_2_cursor = connect(host="localhost", port=7689).cursor() + instance_4_cursor = connect(host="localhost", port=7691).cursor() + + assert execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + assert execute_and_fetch_all(instance_2_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + assert execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n);")[0][0] == 2 + + # 5 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + + # 6 + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_3") + + # 7 + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "up", "replica"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 8 + + with pytest.raises(Exception) as e: + execute_and_fetch_all(instance_2_cursor, "CREATE (:Epoch2Vertex {prop:1});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + def get_vertex_count(): + return execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(3, get_vertex_count) + + assert execute_and_fetch_all(instance_4_cursor, "MATCH (n) RETURN count(n);")[0][0] == 3 + + # 9 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_2") + + # 10 + + interactive_mg_runner.kill(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_4") + + # 11 + + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_1") + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "down", "unknown"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 12 + + interactive_mg_runner.start(MEMGRAPH_INNER_INSTANCES_DESCRIPTION, "instance_2") + + # 13 + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ("instance_4", "", "127.0.0.1:10014", "down", "unknown"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_instances) + + # 12 + instance_1_cursor = connect(host="localhost", port=7688).cursor() + instance_2_cursor = connect(host="localhost", port=7689).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(instance_1_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(2, get_vertex_count) + + def get_vertex_count(): + return execute_and_fetch_all(instance_2_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(2, get_vertex_count) + + # 13 + with pytest.raises(Exception) as e: + execute_and_fetch_all(instance_1_cursor, "CREATE (:Epoch3Vertex {prop:1});") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + # 14 + + def get_vertex_objects_func_creator(cursor): + def get_vertex_objects(): + return list( + execute_and_fetch_all( + cursor, "MATCH (n) " "WITH labels(n) as labels, properties(n) as props " "RETURN labels[0], props;" + ) + ) + + return get_vertex_objects + + vertex_objects = [("Epoch1Vertex", {"prop": 1}), ("Epoch1Vertex", {"prop": 2}), ("Epoch3Vertex", {"prop": 1})] + + mg_sleep_and_assert_collection(vertex_objects, get_vertex_objects_func_creator(instance_1_cursor)) + + mg_sleep_and_assert_collection(vertex_objects, get_vertex_objects_func_creator(instance_2_cursor)) + + # 15 @pytest.mark.parametrize("data_recovery", ["false", "true"]) diff --git a/tests/unit/coordination_utils.cpp b/tests/unit/coordination_utils.cpp index 2a595c19f..7c77b4e68 100644 --- a/tests/unit/coordination_utils.cpp +++ b/tests/unit/coordination_utils.cpp @@ -45,11 +45,9 @@ TEST_F(CoordinationUtils, MemgraphDbHistorySimple) { memgraph::utils::UUID db_uuid; std::string default_name = std::string(memgraph::dbms::kDefaultDB); - auto db_histories = memgraph::utils::fmap( - [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { - return std::make_pair(std::string(pair.first), pair.second); - }, - histories); + auto db_histories = memgraph::utils::fmap(histories, [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }); memgraph::replication_coordination_glue::DatabaseHistory history{ .db_uuid = db_uuid, .history = db_histories, .name = default_name}; @@ -91,11 +89,9 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryLastEpochDifferent) { memgraph::utils::UUID db_uuid; std::string default_name = std::string(memgraph::dbms::kDefaultDB); - auto db_histories = memgraph::utils::fmap( - [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { - return std::make_pair(std::string(pair.first), pair.second); - }, - histories); + auto db_histories = memgraph::utils::fmap(histories, [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }); db_histories.back().second = 51; memgraph::replication_coordination_glue::DatabaseHistory history1{ @@ -145,11 +141,9 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryOneInstanceAheadFewEpochs) { memgraph::utils::UUID db_uuid; std::string default_name = std::string(memgraph::dbms::kDefaultDB); - auto db_histories = memgraph::utils::fmap( - [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { - return std::make_pair(std::string(pair.first), pair.second); - }, - histories); + auto db_histories = memgraph::utils::fmap(histories, [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }); memgraph::replication_coordination_glue::DatabaseHistory history{ .db_uuid = db_uuid, .history = db_histories, .name = default_name}; @@ -162,11 +156,10 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryOneInstanceAheadFewEpochs) { histories.emplace_back(memgraph::utils::UUID{}, 60); histories.emplace_back(memgraph::utils::UUID{}, 65); - auto db_histories_longest = memgraph::utils::fmap( - [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + auto db_histories_longest = + memgraph::utils::fmap(histories, [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { return std::make_pair(std::string(pair.first), pair.second); - }, - histories); + }); memgraph::replication_coordination_glue::DatabaseHistory history_longest{ .db_uuid = db_uuid, .history = db_histories_longest, .name = default_name}; @@ -200,11 +193,9 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryInstancesHistoryDiverged) { memgraph::utils::UUID db_uuid; std::string default_name = std::string(memgraph::dbms::kDefaultDB); - auto db_histories = memgraph::utils::fmap( - [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { - return std::make_pair(std::string(pair.first), pair.second); - }, - histories); + auto db_histories = memgraph::utils::fmap(histories, [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + return std::make_pair(std::string(pair.first), pair.second); + }); memgraph::replication_coordination_glue::DatabaseHistory history{ .db_uuid = db_uuid, .history = db_histories, .name = default_name}; @@ -217,11 +208,10 @@ TEST_F(CoordinationUtils, MemgraphDbHistoryInstancesHistoryDiverged) { auto oldest_commit_timestamp{5}; auto newest_different_epoch = memgraph::utils::UUID{}; histories.emplace_back(newest_different_epoch, oldest_commit_timestamp); - auto db_histories_different = memgraph::utils::fmap( - [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { + auto db_histories_different = + memgraph::utils::fmap(histories, [](const std::pair<memgraph::utils::UUID, uint64_t> &pair) { return std::make_pair(std::string(pair.first), pair.second); - }, - histories); + }); memgraph::replication_coordination_glue::DatabaseHistory history_3{ .db_uuid = db_uuid, .history = db_histories_different, .name = default_name}; From 75aad72984c65a0e258df39659420cd059fa8302 Mon Sep 17 00:00:00 2001 From: Andi <andi8647@gmail.com> Date: Wed, 6 Mar 2024 09:16:46 +0100 Subject: [PATCH 11/18] Improve in-memory RAFT state (#1782) --- src/coordination/coordinator_client.cpp | 3 +- .../coordinator_cluster_state.cpp | 108 ++++------ src/coordination/coordinator_instance.cpp | 30 ++- .../include/coordination/coordinator_rpc.hpp | 2 +- .../include/coordination/raft_state.hpp | 4 +- .../nuraft/coordinator_cluster_state.hpp | 16 +- .../include/nuraft/raft_log_action.hpp | 21 -- src/coordination/raft_state.cpp | 22 +- src/io/network/endpoint.cpp | 200 +++++++----------- src/io/network/endpoint.hpp | 51 ++--- src/query/interpreter.cpp | 8 +- src/query/metadata.cpp | 4 +- src/query/metadata.hpp | 2 +- src/replication_coordination_glue/role.hpp | 6 + tests/e2e/replication/common.hpp | 9 +- tests/unit/CMakeLists.txt | 7 + tests/unit/coordinator_cluster_state.cpp | 163 ++++++++++++++ 17 files changed, 363 insertions(+), 293 deletions(-) create mode 100644 tests/unit/coordinator_cluster_state.cpp diff --git a/src/coordination/coordinator_client.cpp b/src/coordination/coordinator_client.cpp index bc7f42eaa..8530faff3 100644 --- a/src/coordination/coordinator_client.cpp +++ b/src/coordination/coordinator_client.cpp @@ -135,8 +135,7 @@ auto CoordinatorClient::SendSwapMainUUIDRpc(utils::UUID const &uuid) const -> bo auto CoordinatorClient::SendUnregisterReplicaRpc(std::string_view instance_name) const -> bool { try { - auto stream{rpc_client_.Stream<UnregisterReplicaRpc>( - std::string(instance_name))}; // TODO: (andi) Try to change to stream string_view and do just one copy later + auto stream{rpc_client_.Stream<UnregisterReplicaRpc>(instance_name)}; if (!stream.AwaitResponse().success) { spdlog::error("Failed to receive successful RPC response for unregistering replica!"); return false; diff --git a/src/coordination/coordinator_cluster_state.cpp b/src/coordination/coordinator_cluster_state.cpp index 60f0ca622..2213a052f 100644 --- a/src/coordination/coordinator_cluster_state.cpp +++ b/src/coordination/coordinator_cluster_state.cpp @@ -18,78 +18,87 @@ namespace memgraph::coordination { -using replication_coordination_glue::ReplicationRole; +void to_json(nlohmann::json &j, InstanceState const &instance_state) { + j = nlohmann::json{{"config", instance_state.config}, {"status", instance_state.status}}; +} -CoordinatorClusterState::CoordinatorClusterState(CoordinatorClusterState const &other) - : instance_roles_{other.instance_roles_} {} +void from_json(nlohmann::json const &j, InstanceState &instance_state) { + j.at("config").get_to(instance_state.config); + j.at("status").get_to(instance_state.status); +} + +CoordinatorClusterState::CoordinatorClusterState(std::map<std::string, InstanceState, std::less<>> instances) + : instances_{std::move(instances)} {} + +CoordinatorClusterState::CoordinatorClusterState(CoordinatorClusterState const &other) : instances_{other.instances_} {} CoordinatorClusterState &CoordinatorClusterState::operator=(CoordinatorClusterState const &other) { if (this == &other) { return *this; } - instance_roles_ = other.instance_roles_; + instances_ = other.instances_; return *this; } CoordinatorClusterState::CoordinatorClusterState(CoordinatorClusterState &&other) noexcept - : instance_roles_{std::move(other.instance_roles_)} {} + : instances_{std::move(other.instances_)} {} CoordinatorClusterState &CoordinatorClusterState::operator=(CoordinatorClusterState &&other) noexcept { if (this == &other) { return *this; } - instance_roles_ = std::move(other.instance_roles_); + instances_ = std::move(other.instances_); return *this; } auto CoordinatorClusterState::MainExists() const -> bool { auto lock = std::shared_lock{log_lock_}; - return std::ranges::any_of(instance_roles_, - [](auto const &entry) { return entry.second.role == ReplicationRole::MAIN; }); + return std::ranges::any_of(instances_, + [](auto const &entry) { return entry.second.status == ReplicationRole::MAIN; }); } auto CoordinatorClusterState::IsMain(std::string_view instance_name) const -> bool { auto lock = std::shared_lock{log_lock_}; - auto const it = instance_roles_.find(instance_name); - return it != instance_roles_.end() && it->second.role == ReplicationRole::MAIN; + auto const it = instances_.find(instance_name); + return it != instances_.end() && it->second.status == ReplicationRole::MAIN; } auto CoordinatorClusterState::IsReplica(std::string_view instance_name) const -> bool { auto lock = std::shared_lock{log_lock_}; - auto const it = instance_roles_.find(instance_name); - return it != instance_roles_.end() && it->second.role == ReplicationRole::REPLICA; + auto const it = instances_.find(instance_name); + return it != instances_.end() && it->second.status == ReplicationRole::REPLICA; } -auto CoordinatorClusterState::InsertInstance(std::string_view instance_name, ReplicationRole role) -> void { - auto lock = std::unique_lock{log_lock_}; - instance_roles_[instance_name.data()].role = role; +auto CoordinatorClusterState::InsertInstance(std::string instance_name, InstanceState instance_state) -> void { + auto lock = std::lock_guard{log_lock_}; + instances_.insert_or_assign(std::move(instance_name), std::move(instance_state)); } auto CoordinatorClusterState::DoAction(TRaftLog log_entry, RaftLogAction log_action) -> void { - auto lock = std::unique_lock{log_lock_}; + auto lock = std::lock_guard{log_lock_}; switch (log_action) { case RaftLogAction::REGISTER_REPLICATION_INSTANCE: { auto const &config = std::get<CoordinatorClientConfig>(log_entry); - instance_roles_[config.instance_name] = InstanceState{config, ReplicationRole::REPLICA}; + instances_[config.instance_name] = InstanceState{config, ReplicationRole::REPLICA}; break; } case RaftLogAction::UNREGISTER_REPLICATION_INSTANCE: { auto const instance_name = std::get<std::string>(log_entry); - instance_roles_.erase(instance_name); + instances_.erase(instance_name); break; } case RaftLogAction::SET_INSTANCE_AS_MAIN: { auto const instance_name = std::get<std::string>(log_entry); - auto it = instance_roles_.find(instance_name); - MG_ASSERT(it != instance_roles_.end(), "Instance does not exist as part of raft state!"); - it->second.role = ReplicationRole::MAIN; + auto it = instances_.find(instance_name); + MG_ASSERT(it != instances_.end(), "Instance does not exist as part of raft state!"); + it->second.status = ReplicationRole::MAIN; break; } case RaftLogAction::SET_INSTANCE_AS_REPLICA: { auto const instance_name = std::get<std::string>(log_entry); - auto it = instance_roles_.find(instance_name); - MG_ASSERT(it != instance_roles_.end(), "Instance does not exist as part of raft state!"); - it->second.role = ReplicationRole::REPLICA; + auto it = instances_.find(instance_name); + MG_ASSERT(it != instances_.end(), "Instance does not exist as part of raft state!"); + it->second.status = ReplicationRole::REPLICA; break; } case RaftLogAction::UPDATE_UUID: { @@ -99,64 +108,37 @@ auto CoordinatorClusterState::DoAction(TRaftLog log_entry, RaftLogAction log_act } } -// TODO: (andi) Improve based on Gareth's comments auto CoordinatorClusterState::Serialize(ptr<buffer> &data) -> void { auto lock = std::shared_lock{log_lock_}; - auto const role_to_string = [](auto const &role) -> std::string_view { - switch (role) { - case ReplicationRole::MAIN: - return "main"; - case ReplicationRole::REPLICA: - return "replica"; - } - }; - auto const entry_to_string = [&role_to_string](auto const &entry) { - return fmt::format("{}_{}", entry.first, role_to_string(entry.second.role)); - }; + // .at(0) is hack to solve the problem with json serialization of map + auto const log = nlohmann::json{instances_}.at(0).dump(); - auto instances_str_view = instance_roles_ | ranges::views::transform(entry_to_string); - uint32_t size = - std::accumulate(instances_str_view.begin(), instances_str_view.end(), 0, - [](uint32_t acc, auto const &entry) { return acc + sizeof(uint32_t) + entry.size(); }); - - data = buffer::alloc(size); + data = buffer::alloc(sizeof(uint32_t) + log.size()); buffer_serializer bs(data); - std::for_each(instances_str_view.begin(), instances_str_view.end(), [&bs](auto const &entry) { bs.put_str(entry); }); + bs.put_str(log); } auto CoordinatorClusterState::Deserialize(buffer &data) -> CoordinatorClusterState { - auto const str_to_role = [](auto const &str) -> ReplicationRole { - if (str == "main") { - return ReplicationRole::MAIN; - } - return ReplicationRole::REPLICA; - }; - - CoordinatorClusterState cluster_state; buffer_serializer bs(data); - while (bs.size() > 0) { - auto const entry = bs.get_str(); - auto const first_dash = entry.find('_'); - auto const instance_name = entry.substr(0, first_dash); - auto const role_str = entry.substr(first_dash + 1); - cluster_state.InsertInstance(instance_name, str_to_role(role_str)); - } - return cluster_state; + auto const j = nlohmann::json::parse(bs.get_str()); + auto instances = j.get<std::map<std::string, InstanceState, std::less<>>>(); + + return CoordinatorClusterState{std::move(instances)}; } auto CoordinatorClusterState::GetInstances() const -> std::vector<InstanceState> { auto lock = std::shared_lock{log_lock_}; - return instance_roles_ | ranges::views::values | ranges::to<std::vector<InstanceState>>; + return instances_ | ranges::views::values | ranges::to<std::vector<InstanceState>>; } auto CoordinatorClusterState::GetUUID() const -> utils::UUID { return uuid_; } auto CoordinatorClusterState::FindCurrentMainInstanceName() const -> std::optional<std::string> { auto lock = std::shared_lock{log_lock_}; - auto const it = std::ranges::find_if(instance_roles_, - [](auto const &entry) { return entry.second.role == ReplicationRole::MAIN; }); - if (it == instance_roles_.end()) { + auto const it = + std::ranges::find_if(instances_, [](auto const &entry) { return entry.second.status == ReplicationRole::MAIN; }); + if (it == instances_.end()) { return {}; } return it->first; diff --git a/src/coordination/coordinator_instance.cpp b/src/coordination/coordinator_instance.cpp index 9a00ca87c..920fea3cb 100644 --- a/src/coordination/coordinator_instance.cpp +++ b/src/coordination/coordinator_instance.cpp @@ -36,7 +36,7 @@ CoordinatorInstance::CoordinatorInstance() spdlog::info("Leader changed, starting all replication instances!"); auto const instances = raft_state_.GetInstances(); auto replicas = instances | ranges::views::filter([](auto const &instance) { - return instance.role == ReplicationRole::REPLICA; + return instance.status == ReplicationRole::REPLICA; }); std::ranges::for_each(replicas, [this](auto &replica) { @@ -47,10 +47,7 @@ CoordinatorInstance::CoordinatorInstance() }); auto main = instances | ranges::views::filter( - [](auto const &instance) { return instance.role == ReplicationRole::MAIN; }); - - // TODO: (andi) Add support for this - // MG_ASSERT(std::ranges::distance(main) == 1, "There should be exactly one main instance"); + [](auto const &instance) { return instance.status == ReplicationRole::MAIN; }); std::ranges::for_each(main, [this](auto &main_instance) { spdlog::info("Starting main instance {}", main_instance.config.instance_name); @@ -60,7 +57,7 @@ CoordinatorInstance::CoordinatorInstance() }); std::ranges::for_each(repl_instances_, [this](auto &instance) { - instance.SetNewMainUUID(raft_state_.GetUUID()); // TODO: (andi) Rename + instance.SetNewMainUUID(raft_state_.GetUUID()); instance.StartFrequentCheck(); }); }, @@ -69,13 +66,13 @@ CoordinatorInstance::CoordinatorInstance() repl_instances_.clear(); })) { client_succ_cb_ = [](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { - auto lock = std::unique_lock{self->coord_instance_lock_}; + auto lock = std::lock_guard{self->coord_instance_lock_}; auto &repl_instance = self->FindReplicationInstance(repl_instance_name); std::invoke(repl_instance.GetSuccessCallback(), self, repl_instance_name); }; client_fail_cb_ = [](CoordinatorInstance *self, std::string_view repl_instance_name) -> void { - auto lock = std::unique_lock{self->coord_instance_lock_}; + auto lock = std::lock_guard{self->coord_instance_lock_}; auto &repl_instance = self->FindReplicationInstance(repl_instance_name); std::invoke(repl_instance.GetFailCallback(), self, repl_instance_name); }; @@ -98,7 +95,6 @@ auto CoordinatorInstance::ShowInstances() const -> std::vector<InstanceStatus> { .raft_socket_address = instance->get_endpoint(), .cluster_role = "coordinator", .health = "unknown"}; // TODO: (andi) Get this info from RAFT and test it or when we will move - // CoordinatorState to every instance, we can be smarter about this using our RPC. }; auto instances_status = utils::fmap(raft_state_.GetAllCoordinators(), coord_instance_to_status); @@ -126,14 +122,14 @@ auto CoordinatorInstance::ShowInstances() const -> std::vector<InstanceStatus> { std::ranges::transform(repl_instances_, std::back_inserter(instances_status), process_repl_instance_as_leader); } } else { - auto const stringify_repl_role = [](ReplicationRole role) -> std::string { - return role == ReplicationRole::MAIN ? "main" : "replica"; + auto const stringify_inst_status = [](ReplicationRole status) -> std::string { + return status == ReplicationRole::MAIN ? "main" : "replica"; }; // TODO: (andi) Add capability that followers can also return socket addresses - auto process_repl_instance_as_follower = [&stringify_repl_role](auto const &instance) -> InstanceStatus { + auto process_repl_instance_as_follower = [&stringify_inst_status](auto const &instance) -> InstanceStatus { return {.instance_name = instance.config.instance_name, - .cluster_role = stringify_repl_role(instance.role), + .cluster_role = stringify_inst_status(instance.status), .health = "unknown"}; }; @@ -355,11 +351,11 @@ auto CoordinatorInstance::UnregisterReplicationInstance(std::string_view instanc return UnregisterInstanceCoordinatorStatus::NO_INSTANCE_WITH_NAME; } - // TODO: (andi) Change so that RaftLogState is the central place for asking who is main... + auto const is_main = [this](ReplicationInstance const &instance) { + return IsMain(instance.InstanceName()) && instance.GetMainUUID() == raft_state_.GetUUID() && instance.IsAlive(); + }; - auto const is_main = [this](ReplicationInstance const &instance) { return IsMain(instance.InstanceName()); }; - - if (is_main(*inst_to_remove) && inst_to_remove->IsAlive()) { + if (is_main(*inst_to_remove)) { return UnregisterInstanceCoordinatorStatus::IS_MAIN; } diff --git a/src/coordination/include/coordination/coordinator_rpc.hpp b/src/coordination/include/coordination/coordinator_rpc.hpp index 2bf88fe46..d799b2955 100644 --- a/src/coordination/include/coordination/coordinator_rpc.hpp +++ b/src/coordination/include/coordination/coordinator_rpc.hpp @@ -90,7 +90,7 @@ struct UnregisterReplicaReq { static void Load(UnregisterReplicaReq *self, memgraph::slk::Reader *reader); static void Save(UnregisterReplicaReq const &self, memgraph::slk::Builder *builder); - explicit UnregisterReplicaReq(std::string instance_name) : instance_name(std::move(instance_name)) {} + explicit UnregisterReplicaReq(std::string_view inst_name) : instance_name(inst_name) {} UnregisterReplicaReq() = default; diff --git a/src/coordination/include/coordination/raft_state.hpp b/src/coordination/include/coordination/raft_state.hpp index d702697f1..34da3e2a6 100644 --- a/src/coordination/include/coordination/raft_state.hpp +++ b/src/coordination/include/coordination/raft_state.hpp @@ -14,6 +14,7 @@ #ifdef MG_ENTERPRISE #include <flags/replication.hpp> +#include "io/network/endpoint.hpp" #include "nuraft/coordinator_state_machine.hpp" #include "nuraft/coordinator_state_manager.hpp" @@ -79,9 +80,8 @@ class RaftState { private: // TODO: (andi) I think variables below can be abstracted/clean them. + io::network::Endpoint raft_endpoint_; uint32_t raft_server_id_; - uint32_t raft_port_; - std::string raft_address_; ptr<CoordinatorStateMachine> state_machine_; ptr<CoordinatorStateManager> state_manager_; diff --git a/src/coordination/include/nuraft/coordinator_cluster_state.hpp b/src/coordination/include/nuraft/coordinator_cluster_state.hpp index f38d00073..11d539a14 100644 --- a/src/coordination/include/nuraft/coordinator_cluster_state.hpp +++ b/src/coordination/include/nuraft/coordinator_cluster_state.hpp @@ -21,6 +21,7 @@ #include <libnuraft/nuraft.hxx> #include <range/v3/view.hpp> +#include "json/json.hpp" #include <map> #include <numeric> @@ -33,9 +34,16 @@ using replication_coordination_glue::ReplicationRole; struct InstanceState { CoordinatorClientConfig config; - ReplicationRole role; + ReplicationRole status; + + friend auto operator==(InstanceState const &lhs, InstanceState const &rhs) -> bool { + return lhs.config == rhs.config && lhs.status == rhs.status; + } }; +void to_json(nlohmann::json &j, InstanceState const &instance_state); +void from_json(nlohmann::json const &j, InstanceState &instance_state); + using TRaftLog = std::variant<CoordinatorClientConfig, std::string, utils::UUID>; using nuraft::buffer; @@ -45,6 +53,8 @@ using nuraft::ptr; class CoordinatorClusterState { public: CoordinatorClusterState() = default; + explicit CoordinatorClusterState(std::map<std::string, InstanceState, std::less<>> instances); + CoordinatorClusterState(CoordinatorClusterState const &); CoordinatorClusterState &operator=(CoordinatorClusterState const &); @@ -60,7 +70,7 @@ class CoordinatorClusterState { auto IsReplica(std::string_view instance_name) const -> bool; - auto InsertInstance(std::string_view instance_name, ReplicationRole role) -> void; + auto InsertInstance(std::string instance_name, InstanceState instance_state) -> void; auto DoAction(TRaftLog log_entry, RaftLogAction log_action) -> void; @@ -73,7 +83,7 @@ class CoordinatorClusterState { auto GetUUID() const -> utils::UUID; private: - std::map<std::string, InstanceState, std::less<>> instance_roles_; + std::map<std::string, InstanceState, std::less<>> instances_{}; utils::UUID uuid_{}; mutable utils::ResourceLock log_lock_{}; }; diff --git a/src/coordination/include/nuraft/raft_log_action.hpp b/src/coordination/include/nuraft/raft_log_action.hpp index 953049038..3f1b26dfa 100644 --- a/src/coordination/include/nuraft/raft_log_action.hpp +++ b/src/coordination/include/nuraft/raft_log_action.hpp @@ -38,26 +38,5 @@ NLOHMANN_JSON_SERIALIZE_ENUM(RaftLogAction, { {RaftLogAction::UPDATE_UUID, "update_uuid"}, }) -inline auto ParseRaftLogAction(std::string_view action) -> RaftLogAction { - if (action == "register") { - return RaftLogAction::REGISTER_REPLICATION_INSTANCE; - } - if (action == "unregister") { - return RaftLogAction::UNREGISTER_REPLICATION_INSTANCE; - } - if (action == "promote") { - return RaftLogAction::SET_INSTANCE_AS_MAIN; - } - if (action == "demote") { - return RaftLogAction::SET_INSTANCE_AS_REPLICA; - } - - if (action == "update_uuid") { - return RaftLogAction::UPDATE_UUID; - } - - throw InvalidRaftLogActionException("Invalid Raft log action: {}.", action); -} - } // namespace memgraph::coordination #endif diff --git a/src/coordination/raft_state.cpp b/src/coordination/raft_state.cpp index 365388b06..d4d65cc36 100644 --- a/src/coordination/raft_state.cpp +++ b/src/coordination/raft_state.cpp @@ -32,12 +32,10 @@ using raft_result = cmd_result<ptr<buffer>>; RaftState::RaftState(BecomeLeaderCb become_leader_cb, BecomeFollowerCb become_follower_cb, uint32_t raft_server_id, uint32_t raft_port, std::string raft_address) - : raft_server_id_(raft_server_id), - raft_port_(raft_port), - raft_address_(std::move(raft_address)), + : raft_endpoint_(raft_address, raft_port), + raft_server_id_(raft_server_id), state_machine_(cs_new<CoordinatorStateMachine>()), - state_manager_( - cs_new<CoordinatorStateManager>(raft_server_id_, raft_address_ + ":" + std::to_string(raft_port_))), + state_manager_(cs_new<CoordinatorStateManager>(raft_server_id_, raft_endpoint_.SocketAddress())), logger_(nullptr), become_leader_cb_(std::move(become_leader_cb)), become_follower_cb_(std::move(become_follower_cb)) {} @@ -71,11 +69,11 @@ auto RaftState::InitRaftServer() -> void { raft_launcher launcher; - raft_server_ = launcher.init(state_machine_, state_manager_, logger_, static_cast<int>(raft_port_), asio_opts, params, - init_opts); + raft_server_ = + launcher.init(state_machine_, state_manager_, logger_, raft_endpoint_.port, asio_opts, params, init_opts); if (!raft_server_) { - throw RaftServerStartException("Failed to launch raft server on {}:{}", raft_address_, raft_port_); + throw RaftServerStartException("Failed to launch raft server on {}", raft_endpoint_.SocketAddress()); } auto maybe_stop = utils::ResettableCounter<20>(); @@ -86,7 +84,7 @@ auto RaftState::InitRaftServer() -> void { std::this_thread::sleep_for(std::chrono::milliseconds(250)); } while (!maybe_stop()); - throw RaftServerStartException("Failed to initialize raft server on {}:{}", raft_address_, raft_port_); + throw RaftServerStartException("Failed to initialize raft server on {}", raft_endpoint_.SocketAddress()); } auto RaftState::MakeRaftState(BecomeLeaderCb &&become_leader_cb, BecomeFollowerCb &&become_follower_cb) -> RaftState { @@ -102,9 +100,11 @@ auto RaftState::MakeRaftState(BecomeLeaderCb &&become_leader_cb, BecomeFollowerC RaftState::~RaftState() { launcher_.shutdown(); } -auto RaftState::InstanceName() const -> std::string { return "coordinator_" + std::to_string(raft_server_id_); } +auto RaftState::InstanceName() const -> std::string { + return fmt::format("coordinator_{}", std::to_string(raft_server_id_)); +} -auto RaftState::RaftSocketAddress() const -> std::string { return raft_address_ + ":" + std::to_string(raft_port_); } +auto RaftState::RaftSocketAddress() const -> std::string { return raft_endpoint_.SocketAddress(); } auto RaftState::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_port, std::string_view raft_address) -> void { diff --git a/src/io/network/endpoint.cpp b/src/io/network/endpoint.cpp index bb6dcfd10..6ed4a6753 100644 --- a/src/io/network/endpoint.cpp +++ b/src/io/network/endpoint.cpp @@ -22,113 +22,15 @@ #include "utils/message.hpp" #include "utils/string.hpp" +namespace { +constexpr std::string_view delimiter = ":"; +} // namespace + namespace memgraph::io::network { -Endpoint::IpFamily Endpoint::GetIpFamily(std::string_view address) { - in_addr addr4; - in6_addr addr6; - int ipv4_result = inet_pton(AF_INET, address.data(), &addr4); - int ipv6_result = inet_pton(AF_INET6, address.data(), &addr6); - if (ipv4_result == 1) { - return IpFamily::IP4; - } - if (ipv6_result == 1) { - return IpFamily::IP6; - } - return IpFamily::NONE; -} - -std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrIpAddress( - std::string_view address, const std::optional<uint16_t> default_port) { - /// expected address format: - /// - "ip_address:port_number" - /// - "ip_address" - /// We parse the address first. If it's an IP address, a default port must - // be given, or we return nullopt. If it's a socket address, we try to parse - // it into an ip address and a port number; even if a default port is given, - // it won't be used, as we expect that it is given in the address string. - const std::string delimiter = ":"; - std::string ip_address; - - std::vector<std::string> parts = utils::Split(address, delimiter); - if (parts.size() == 1) { - if (default_port) { - if (GetIpFamily(address) == IpFamily::NONE) { - return std::nullopt; - } - return std::pair{std::string(address), *default_port}; // TODO: (andi) Optimize throughout the code - } - } else if (parts.size() == 2) { - ip_address = std::move(parts[0]); - if (GetIpFamily(ip_address) == IpFamily::NONE) { - return std::nullopt; - } - int64_t int_port{0}; - try { - int_port = utils::ParseInt(parts[1]); - } catch (utils::BasicException &e) { - spdlog::error(utils::MessageWithLink("Invalid port number {}.", parts[1], "https://memgr.ph/ports")); - return std::nullopt; - } - if (int_port < 0) { - spdlog::error(utils::MessageWithLink("Invalid port number {}. The port number must be a positive integer.", - int_port, "https://memgr.ph/ports")); - return std::nullopt; - } - if (int_port > std::numeric_limits<uint16_t>::max()) { - spdlog::error(utils::MessageWithLink("Invalid port number. The port number exceedes the maximum possible size.", - "https://memgr.ph/ports")); - return std::nullopt; - } - - return std::pair{ip_address, static_cast<uint16_t>(int_port)}; - } - - return std::nullopt; -} - -std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseHostname( - std::string_view address, const std::optional<uint16_t> default_port = {}) { - const std::string delimiter = ":"; - std::string ip_address; - std::vector<std::string> parts = utils::Split(address, delimiter); - if (parts.size() == 1) { - if (default_port) { - if (!IsResolvableAddress(address, *default_port)) { - return std::nullopt; - } - return std::pair{std::string(address), *default_port}; // TODO: (andi) Optimize throughout the code - } - } else if (parts.size() == 2) { - int64_t int_port{0}; - auto hostname = std::move(parts[0]); - try { - int_port = utils::ParseInt(parts[1]); - } catch (utils::BasicException &e) { - spdlog::error(utils::MessageWithLink("Invalid port number {}.", parts[1], "https://memgr.ph/ports")); - return std::nullopt; - } - if (int_port < 0) { - spdlog::error(utils::MessageWithLink("Invalid port number {}. The port number must be a positive integer.", - int_port, "https://memgr.ph/ports")); - return std::nullopt; - } - if (int_port > std::numeric_limits<uint16_t>::max()) { - spdlog::error(utils::MessageWithLink("Invalid port number. The port number exceedes the maximum possible size.", - "https://memgr.ph/ports")); - return std::nullopt; - } - if (IsResolvableAddress(hostname, static_cast<uint16_t>(int_port))) { - return std::pair{hostname, static_cast<u_int16_t>(int_port)}; - } - } - return std::nullopt; -} - -std::string Endpoint::SocketAddress() const { - auto ip_address = address.empty() ? "EMPTY" : address; - return ip_address + ":" + std::to_string(port); -} +// NOLINTNEXTLINE +Endpoint::Endpoint(needs_resolving_t, std::string hostname, uint16_t port) + : address(std::move(hostname)), port(port), family{GetIpFamily(address)} {} Endpoint::Endpoint(std::string ip_address, uint16_t port) : address(std::move(ip_address)), port(port) { IpFamily ip_family = GetIpFamily(address); @@ -138,9 +40,23 @@ Endpoint::Endpoint(std::string ip_address, uint16_t port) : address(std::move(ip family = ip_family; } -// NOLINTNEXTLINE -Endpoint::Endpoint(needs_resolving_t, std::string hostname, uint16_t port) - : address(std::move(hostname)), port(port), family{GetIpFamily(address)} {} +std::string Endpoint::SocketAddress() const { return fmt::format("{}:{}", address, port); } + +Endpoint::IpFamily Endpoint::GetIpFamily(std::string_view address) { + // Ensure null-terminated + auto const tmp = std::string(address); + in_addr addr4; + in6_addr addr6; + int ipv4_result = inet_pton(AF_INET, tmp.c_str(), &addr4); + int ipv6_result = inet_pton(AF_INET6, tmp.c_str(), &addr6); + if (ipv4_result == 1) { + return IpFamily::IP4; + } + if (ipv6_result == 1) { + return IpFamily::IP6; + } + return IpFamily::NONE; +} std::ostream &operator<<(std::ostream &os, const Endpoint &endpoint) { // no need to cover the IpFamily::NONE case, as you can't even construct an @@ -153,6 +69,7 @@ std::ostream &operator<<(std::ostream &os, const Endpoint &endpoint) { return os << endpoint.address << ":" << endpoint.port; } +// NOTE: Intentional copy to ensure null-terminated string bool Endpoint::IsResolvableAddress(std::string_view address, uint16_t port) { addrinfo hints{ .ai_flags = AI_PASSIVE, @@ -160,28 +77,65 @@ bool Endpoint::IsResolvableAddress(std::string_view address, uint16_t port) { .ai_socktype = SOCK_STREAM // TCP socket }; addrinfo *info = nullptr; - auto status = getaddrinfo(address.data(), std::to_string(port).c_str(), &hints, &info); + auto status = getaddrinfo(std::string(address).c_str(), std::to_string(port).c_str(), &hints, &info); if (info) freeaddrinfo(info); return status == 0; } -std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrAddress( - std::string_view address, const std::optional<uint16_t> default_port) { - const std::string delimiter = ":"; - std::vector<std::string> parts = utils::Split(address, delimiter); - if (parts.size() == 1) { - if (GetIpFamily(address) == IpFamily::NONE) { - return ParseHostname(address, default_port); - } - return ParseSocketOrIpAddress(address, default_port); +std::optional<ParsedAddress> Endpoint::ParseSocketOrAddress(std::string_view address, + std::optional<uint16_t> default_port) { + auto const parts = utils::SplitView(address, delimiter); + + if (parts.size() > 2) { + return std::nullopt; } - if (parts.size() == 2) { - if (GetIpFamily(parts[0]) == IpFamily::NONE) { - return ParseHostname(address, default_port); + + auto const port = [default_port, &parts]() -> std::optional<uint16_t> { + if (parts.size() == 2) { + return static_cast<uint16_t>(utils::ParseInt(parts[1])); } - return ParseSocketOrIpAddress(address, default_port); + return default_port; + }(); + + if (!ValidatePort(port)) { + return std::nullopt; } - return std::nullopt; + + auto const addr = [address, &parts]() { + if (parts.size() == 2) { + return parts[0]; + } + return address; + }(); + + if (GetIpFamily(addr) == IpFamily::NONE) { + if (IsResolvableAddress(addr, *port)) { // NOLINT + return std::pair{addr, *port}; // NOLINT + } + return std::nullopt; + } + + return std::pair{addr, *port}; // NOLINT +} + +auto Endpoint::ValidatePort(std::optional<uint16_t> port) -> bool { + if (!port) { + return false; + } + + if (port < 0) { + spdlog::error(utils::MessageWithLink("Invalid port number {}. The port number must be a positive integer.", *port, + "https://memgr.ph/ports")); + return false; + } + + if (port > std::numeric_limits<uint16_t>::max()) { + spdlog::error(utils::MessageWithLink("Invalid port number. The port number exceedes the maximum possible size.", + "https://memgr.ph/ports")); + return false; + } + + return true; } } // namespace memgraph::io::network diff --git a/src/io/network/endpoint.hpp b/src/io/network/endpoint.hpp index b0201240b..f46d28ace 100644 --- a/src/io/network/endpoint.hpp +++ b/src/io/network/endpoint.hpp @@ -19,11 +19,8 @@ namespace memgraph::io::network { -/** - * This class represents a network endpoint that is used in Socket. - * It is used when connecting to an address and to get the current - * connection address. - */ +using ParsedAddress = std::pair<std::string_view, uint16_t>; + struct Endpoint { static const struct needs_resolving_t { } needs_resolving; @@ -31,59 +28,35 @@ struct Endpoint { Endpoint() = default; Endpoint(std::string ip_address, uint16_t port); Endpoint(needs_resolving_t, std::string hostname, uint16_t port); + Endpoint(Endpoint const &) = default; Endpoint(Endpoint &&) noexcept = default; + Endpoint &operator=(Endpoint const &) = default; Endpoint &operator=(Endpoint &&) noexcept = default; + ~Endpoint() = default; enum class IpFamily : std::uint8_t { NONE, IP4, IP6 }; - std::string SocketAddress() const; + static std::optional<ParsedAddress> ParseSocketOrAddress(std::string_view address, + std::optional<uint16_t> default_port = {}); - bool operator==(const Endpoint &other) const = default; - friend std::ostream &operator<<(std::ostream &os, const Endpoint &endpoint); + std::string SocketAddress() const; std::string address; uint16_t port{0}; IpFamily family{IpFamily::NONE}; - static std::optional<std::pair<std::string, uint16_t>> ParseSocketOrAddress( - std::string_view address, std::optional<uint16_t> default_port = {}); - - /** - * Tries to parse the given string as either a socket address or ip address. - * Expected address format: - * - "ip_address:port_number" - * - "ip_address" - * We parse the address first. If it's an IP address, a default port must - * be given, or we return nullopt. If it's a socket address, we try to parse - * it into an ip address and a port number; even if a default port is given, - * it won't be used, as we expect that it is given in the address string. - */ - static std::optional<std::pair<std::string, uint16_t>> ParseSocketOrIpAddress( - std::string_view address, std::optional<uint16_t> default_port = {}); - - /** - * Tries to parse given string as either socket address or hostname. - * Expected address format: - * - "hostname:port_number" - * - "hostname" - * After we parse hostname and port we try to resolve the hostname into an ip_address. - */ - static std::optional<std::pair<std::string, uint16_t>> ParseHostname(std::string_view address, - std::optional<uint16_t> default_port); + bool operator==(const Endpoint &other) const = default; + friend std::ostream &operator<<(std::ostream &os, const Endpoint &endpoint); + private: static IpFamily GetIpFamily(std::string_view address); static bool IsResolvableAddress(std::string_view address, uint16_t port); - /** - * Tries to resolve hostname to its corresponding IP address. - * Given a DNS hostname, this function performs resolution and returns - * the IP address associated with the hostname. - */ - static std::string ResolveHostnameIntoIpAddress(const std::string &address, uint16_t port); + static auto ValidatePort(std::optional<uint16_t> port) -> bool; }; } // namespace memgraph::io::network diff --git a/src/query/interpreter.cpp b/src/query/interpreter.cpp index e6d39ab9a..e51620bf6 100644 --- a/src/query/interpreter.cpp +++ b/src/query/interpreter.cpp @@ -355,7 +355,7 @@ class ReplQueryHandler { const auto replication_config = replication::ReplicationClientConfig{.name = name, .mode = repl_mode, - .ip_address = ip, + .ip_address = std::string(ip), .port = port, .replica_check_frequency = replica_check_frequency, .ssl = std::nullopt}; @@ -454,12 +454,12 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { const auto repl_config = coordination::CoordinatorClientConfig::ReplicationClientInfo{ .instance_name = std::string(instance_name), .replication_mode = convertFromCoordinatorToReplicationMode(sync_mode), - .replication_ip_address = replication_ip, + .replication_ip_address = std::string(replication_ip), .replication_port = replication_port}; auto coordinator_client_config = coordination::CoordinatorClientConfig{.instance_name = std::string(instance_name), - .ip_address = coordinator_server_ip, + .ip_address = std::string(coordinator_server_ip), .port = coordinator_server_port, .instance_health_check_frequency_sec = instance_check_frequency, .instance_down_timeout_sec = instance_down_timeout, @@ -1212,7 +1212,7 @@ Callback HandleCoordinatorQuery(CoordinatorQuery *coordinator_query, const Param }; notifications->emplace_back( - SeverityLevel::INFO, NotificationCode::REGISTER_COORDINATOR_SERVER, + SeverityLevel::INFO, NotificationCode::REGISTER_REPLICATION_INSTANCE, fmt::format("Coordinator has registered coordinator server on {} for instance {}.", coordinator_socket_address_tv.ValueString(), coordinator_query->instance_name_)); return callback; diff --git a/src/query/metadata.cpp b/src/query/metadata.cpp index e339aad57..af3b8d15f 100644 --- a/src/query/metadata.cpp +++ b/src/query/metadata.cpp @@ -67,8 +67,8 @@ constexpr std::string_view GetCodeString(const NotificationCode code) { case NotificationCode::REGISTER_REPLICA: return "RegisterReplica"sv; #ifdef MG_ENTERPRISE - case NotificationCode::REGISTER_COORDINATOR_SERVER: - return "RegisterCoordinatorServer"sv; + case NotificationCode::REGISTER_REPLICATION_INSTANCE: + return "RegisterReplicationInstance"sv; case NotificationCode::ADD_COORDINATOR_INSTANCE: return "AddCoordinatorInstance"sv; case NotificationCode::UNREGISTER_INSTANCE: diff --git a/src/query/metadata.hpp b/src/query/metadata.hpp index dd8c2db07..fba672f4b 100644 --- a/src/query/metadata.hpp +++ b/src/query/metadata.hpp @@ -43,7 +43,7 @@ enum class NotificationCode : uint8_t { REPLICA_PORT_WARNING, REGISTER_REPLICA, #ifdef MG_ENTERPRISE - REGISTER_COORDINATOR_SERVER, // TODO: (andi) What is this? + REGISTER_REPLICATION_INSTANCE, ADD_COORDINATOR_INSTANCE, UNREGISTER_INSTANCE, #endif diff --git a/src/replication_coordination_glue/role.hpp b/src/replication_coordination_glue/role.hpp index d472cb454..3fbf522ba 100644 --- a/src/replication_coordination_glue/role.hpp +++ b/src/replication_coordination_glue/role.hpp @@ -12,8 +12,14 @@ #pragma once #include <cstdint> + +#include "json/json.hpp" + namespace memgraph::replication_coordination_glue { // TODO: figure out a way of ensuring that usage of this type is never uninitialed/defaulted incorrectly to MAIN enum class ReplicationRole : uint8_t { MAIN, REPLICA }; + +NLOHMANN_JSON_SERIALIZE_ENUM(ReplicationRole, {{ReplicationRole::MAIN, "main"}, {ReplicationRole::REPLICA, "replica"}}) + } // namespace memgraph::replication_coordination_glue diff --git a/tests/e2e/replication/common.hpp b/tests/e2e/replication/common.hpp index f5113ac37..1938eb0f3 100644 --- a/tests/e2e/replication/common.hpp +++ b/tests/e2e/replication/common.hpp @@ -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 @@ -34,12 +34,13 @@ DEFINE_double(reads_duration_limit, 10.0, "How long should the client perform re namespace mg::e2e::replication { auto ParseDatabaseEndpoints(const std::string &database_endpoints_str) { - const auto db_endpoints_strs = memgraph::utils::Split(database_endpoints_str, ","); + const auto db_endpoints_strs = memgraph::utils::SplitView(database_endpoints_str, ","); std::vector<memgraph::io::network::Endpoint> database_endpoints; for (const auto &db_endpoint_str : db_endpoints_strs) { - const auto maybe_host_port = memgraph::io::network::Endpoint::ParseSocketOrIpAddress(db_endpoint_str, 7687); + const auto maybe_host_port = memgraph::io::network::Endpoint::ParseSocketOrAddress(db_endpoint_str, 7687); MG_ASSERT(maybe_host_port); - database_endpoints.emplace_back(maybe_host_port->first, maybe_host_port->second); + auto const [ip, port] = *maybe_host_port; + database_endpoints.emplace_back(std::string(ip), port); } return database_endpoints; } diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt index f1afcdf15..44b24b6f6 100644 --- a/tests/unit/CMakeLists.txt +++ b/tests/unit/CMakeLists.txt @@ -445,3 +445,10 @@ add_unit_test(raft_log_serialization.cpp) target_link_libraries(${test_prefix}raft_log_serialization gflags mg-coordination mg-repl_coord_glue) target_include_directories(${test_prefix}raft_log_serialization PRIVATE ${CMAKE_SOURCE_DIR}/include) endif() + +# Test Raft log serialization +if(MG_ENTERPRISE) +add_unit_test(coordinator_cluster_state.cpp) +target_link_libraries(${test_prefix}coordinator_cluster_state gflags mg-coordination mg-repl_coord_glue) +target_include_directories(${test_prefix}coordinator_cluster_state PRIVATE ${CMAKE_SOURCE_DIR}/include) +endif() diff --git a/tests/unit/coordinator_cluster_state.cpp b/tests/unit/coordinator_cluster_state.cpp new file mode 100644 index 000000000..8df2797f2 --- /dev/null +++ b/tests/unit/coordinator_cluster_state.cpp @@ -0,0 +1,163 @@ +// 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 +// 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 "nuraft/coordinator_cluster_state.hpp" +#include "nuraft/coordinator_state_machine.hpp" +#include "replication_coordination_glue/role.hpp" + +#include "utils/file.hpp" + +#include <gflags/gflags.h> +#include <gtest/gtest.h> +#include "json/json.hpp" + +#include "libnuraft/nuraft.hxx" + +using memgraph::coordination::CoordinatorClientConfig; +using memgraph::coordination::CoordinatorClusterState; +using memgraph::coordination::CoordinatorStateMachine; +using memgraph::coordination::InstanceState; +using memgraph::coordination::RaftLogAction; +using memgraph::replication_coordination_glue::ReplicationMode; +using memgraph::replication_coordination_glue::ReplicationRole; +using nuraft::buffer; +using nuraft::buffer_serializer; +using nuraft::ptr; + +class CoordinatorClusterStateTest : public ::testing::Test { + protected: + void SetUp() override {} + + void TearDown() override {} + + std::filesystem::path test_folder_{std::filesystem::temp_directory_path() / + "MG_tests_unit_coordinator_cluster_state"}; +}; + +TEST_F(CoordinatorClusterStateTest, InstanceStateSerialization) { + InstanceState instance_state{ + CoordinatorClientConfig{"instance3", + "127.0.0.1", + 10112, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10001}, + .ssl = std::nullopt}, + ReplicationRole::MAIN}; + + nlohmann::json j = instance_state; + InstanceState deserialized_instance_state = j.get<InstanceState>(); + + EXPECT_EQ(instance_state.config, deserialized_instance_state.config); + EXPECT_EQ(instance_state.status, deserialized_instance_state.status); +} + +TEST_F(CoordinatorClusterStateTest, DoActionRegisterInstances) { + auto coordinator_cluster_state = memgraph::coordination::CoordinatorClusterState{}; + + { + CoordinatorClientConfig config{"instance1", + "127.0.0.1", + 10111, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10001}, + .ssl = std::nullopt}; + + auto buffer = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + + coordinator_cluster_state.DoAction(payload, action); + } + { + CoordinatorClientConfig config{"instance2", + "127.0.0.1", + 10112, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10002}, + .ssl = std::nullopt}; + + auto buffer = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + + coordinator_cluster_state.DoAction(payload, action); + } + { + CoordinatorClientConfig config{"instance3", + "127.0.0.1", + 10113, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10003}, + .ssl = std::nullopt}; + + auto buffer = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + + coordinator_cluster_state.DoAction(payload, action); + } + { + CoordinatorClientConfig config{"instance4", + "127.0.0.1", + 10114, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10004}, + .ssl = std::nullopt}; + + auto buffer = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + + coordinator_cluster_state.DoAction(payload, action); + } + { + CoordinatorClientConfig config{"instance5", + "127.0.0.1", + 10115, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10005}, + .ssl = std::nullopt}; + + auto buffer = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + + coordinator_cluster_state.DoAction(payload, action); + } + { + CoordinatorClientConfig config{"instance6", + "127.0.0.1", + 10116, + std::chrono::seconds{1}, + std::chrono::seconds{5}, + std::chrono::seconds{10}, + {"instance_name", ReplicationMode::ASYNC, "replication_ip_address", 10006}, + .ssl = std::nullopt}; + + auto buffer = CoordinatorStateMachine::SerializeRegisterInstance(config); + auto [payload, action] = CoordinatorStateMachine::DecodeLog(*buffer); + + coordinator_cluster_state.DoAction(payload, action); + } + + ptr<buffer> data; + coordinator_cluster_state.Serialize(data); + + auto deserialized_coordinator_cluster_state = CoordinatorClusterState::Deserialize(*data); + ASSERT_EQ(coordinator_cluster_state.GetInstances(), deserialized_coordinator_cluster_state.GetInstances()); +} From 6f849a14df98ef2ae54613819204004af59553a4 Mon Sep 17 00:00:00 2001 From: Katarina Supe <61758502+katarinasupe@users.noreply.github.com> Date: Thu, 7 Mar 2024 10:04:36 +0100 Subject: [PATCH 12/18] Update cypherl transform script (#1701) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Update cypherl transform script * Add new script and fix typo * Add convert to separate files script --------- Co-authored-by: Marko Budiselić <marko.budiselic@memgraph.com> --- import/n2mg_cypherl.sh | 8 +++- import/n2mg_separate_files_cypherl.sh | 61 ++++++++++++++++++++++++ import/n2mg_separate_files_cypherls.sh | 64 ++++++++++++++++++++++++++ 3 files changed, 131 insertions(+), 2 deletions(-) create mode 100755 import/n2mg_separate_files_cypherl.sh create mode 100755 import/n2mg_separate_files_cypherls.sh diff --git a/import/n2mg_cypherl.sh b/import/n2mg_cypherl.sh index b11f5d3e3..2605fc6c7 100755 --- a/import/n2mg_cypherl.sh +++ b/import/n2mg_cypherl.sh @@ -20,14 +20,18 @@ if [ ! -f "$INPUT" ]; then fi echo -e "${COLOR_ORANGE}NOTE:${COLOR_NULL} BEGIN and COMMIT are required because variables share the same name (e.g. row)" -echo -e "${COLOR_ORANGE}NOTE:${COLOR_NULL} CONSTRAINTS are just skipped -> ${COLOR_RED}please create consraints manually if needed${COLOR_NULL}" +echo -e "${COLOR_ORANGE}NOTE:${COLOR_NULL} CONSTRAINTS are just skipped -> ${COLOR_RED}please create constraints manually if needed${COLOR_NULL}" + +echo 'CREATE INDEX ON :`UNIQUE IMPORT LABEL`(`UNIQUE IMPORT ID`);' > "$OUTPUT" sed -e 's/^:begin/BEGIN/g; s/^BEGIN$/BEGIN;/g;' \ -e 's/^:commit/COMMIT/g; s/^COMMIT$/COMMIT;/g;' \ -e '/^CALL/d; /^SCHEMA AWAIT/d;' \ -e 's/CREATE RANGE INDEX FOR (n:/CREATE INDEX ON :/g;' \ -e 's/) ON (n./(/g;' \ - -e '/^CREATE CONSTRAINT/d; /^DROP CONSTRAINT/d;' "$INPUT" > "$OUTPUT" + -e '/^CREATE CONSTRAINT/d; /^DROP CONSTRAINT/d;' "$INPUT" >> "$OUTPUT" + +echo 'DROP INDEX ON :`UNIQUE IMPORT LABEL`(`UNIQUE IMPORT ID`);' >> "$OUTPUT" echo "" echo -e "${COLOR_GREEN}DONE!${COLOR_NULL} Please find Memgraph compatible cypherl|.cypher file under $OUTPUT" diff --git a/import/n2mg_separate_files_cypherl.sh b/import/n2mg_separate_files_cypherl.sh new file mode 100755 index 000000000..98049f193 --- /dev/null +++ b/import/n2mg_separate_files_cypherl.sh @@ -0,0 +1,61 @@ +#!/bin/bash -e +COLOR_ORANGE="\e[38;5;208m" +COLOR_GREEN="\e[38;5;35m" +COLOR_RED="\e[0;31m" +COLOR_NULL="\e[0m" + +print_help() { + echo -e "${COLOR_ORANGE}HOW TO RUN:${COLOR_NULL} $0 input_file_schema_path input_file_nodes_path input_file_relationships_path input_file_cleanup_path output_file_path" + exit 1 +} + +if [ "$#" -ne 5 ]; then + print_help +fi +INPUT_SCHEMA="$1" +INPUT_NODES="$2" +INPUT_RELATIONSHIPS="$3" +INPUT_CLEANUP="$4" +OUTPUT="$5" + +if [ ! -f "$INPUT_SCHEMA" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +if [ ! -f "$INPUT_NODES" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +if [ ! -f "$INPUT_RELATIONSHIPS" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +if [ ! -f "$INPUT_CLEANUP" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +echo -e "${COLOR_ORANGE}NOTE:${COLOR_NULL} BEGIN and COMMIT are required because variables share the same name (e.g. row)" +echo -e "${COLOR_ORANGE}NOTE:${COLOR_NULL} CONSTRAINTS are just skipped -> ${COLOR_RED}please create constraints manually if needed${COLOR_NULL}" + + +echo 'CREATE INDEX ON :`UNIQUE IMPORT LABEL`(`UNIQUE IMPORT ID`);' > "$OUTPUT" + +sed -e 's/CREATE RANGE INDEX FOR (n:/CREATE INDEX ON :/g;' \ + -e 's/) ON (n./(/g;' \ + -e '/^CREATE CONSTRAINT/d' $INPUT_SCHEMA >> "$OUTPUT" + +cat "$INPUT_NODES" >> "$OUTPUT" +cat "$INPUT_RELATIONSHIPS" >> "$OUTPUT" + +sed -e '/^DROP CONSTRAINT/d' "$INPUT_CLEANUP" >> "$OUTPUT" + +echo 'DROP INDEX ON :`UNIQUE IMPORT LABEL`(`UNIQUE IMPORT ID`);' >> "$OUTPUT" + +echo "" +echo -e "${COLOR_GREEN}DONE!${COLOR_NULL} Please find Memgraph compatible cypherl|.cypher file under $OUTPUT" +echo "" +echo "Please import data by executing => \`cat $OUTPUT | mgconsole\`" diff --git a/import/n2mg_separate_files_cypherls.sh b/import/n2mg_separate_files_cypherls.sh new file mode 100755 index 000000000..5b9057e24 --- /dev/null +++ b/import/n2mg_separate_files_cypherls.sh @@ -0,0 +1,64 @@ +#!/bin/bash -e +COLOR_ORANGE="\e[38;5;208m" +COLOR_GREEN="\e[38;5;35m" +COLOR_RED="\e[0;31m" +COLOR_NULL="\e[0m" + +print_help() { + echo -e "${COLOR_ORANGE}HOW TO RUN:${COLOR_NULL} $0 input_file_schema_path input_file_nodes_path input_file_relationships_path input_file_cleanup_path output_file_schema_path output_file_nodes_path output_file_relationships_path output_file_cleanup_path" + exit 1 +} + +if [ "$#" -ne 8 ]; then + print_help +fi +INPUT_SCHEMA="$1" +INPUT_NODES="$2" +INPUT_RELATIONSHIPS="$3" +INPUT_CLEANUP="$4" +OUTPUT_SCHEMA="$5" +OUTPUT_NODES="$6" +OUTPUT_RELATIONSHIPS="$7" +OUTPUT_CLEANUP="$8" + +if [ ! -f "$INPUT_SCHEMA" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +if [ ! -f "$INPUT_NODES" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +if [ ! -f "$INPUT_RELATIONSHIPS" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +if [ ! -f "$INPUT_CLEANUP" ]; then + echo -e "${COLOR_RED}ERROR:${COLOR_NULL} input_file_path is not a file!" + print_help +fi + +echo -e "${COLOR_ORANGE}NOTE:${COLOR_NULL} BEGIN and COMMIT are required because variables share the same name (e.g. row)" +echo -e "${COLOR_ORANGE}NOTE:${COLOR_NULL} CONSTRAINTS are just skipped -> ${COLOR_RED}please create constraints manually if needed${COLOR_NULL}" + + +echo 'CREATE INDEX ON :`UNIQUE IMPORT LABEL`(`UNIQUE IMPORT ID`);' > "$OUTPUT_SCHEMA" + +sed -e 's/CREATE RANGE INDEX FOR (n:/CREATE INDEX ON :/g;' \ + -e 's/) ON (n./(/g;' \ + -e '/^CREATE CONSTRAINT/d' $INPUT_SCHEMA >> "$OUTPUT_SCHEMA" + +cat "$INPUT_NODES" > "$OUTPUT_NODES" +cat "$INPUT_RELATIONSHIPS" > "$OUTPUT_RELATIONSHIPS" + +sed -e '/^DROP CONSTRAINT/d' "$INPUT_CLEANUP" >> "$OUTPUT_CLEANUP" + +echo 'DROP INDEX ON :`UNIQUE IMPORT LABEL`(`UNIQUE IMPORT ID`);' >> "$OUTPUT_CLEANUP" + +echo "" +echo -e "${COLOR_GREEN}DONE!${COLOR_NULL} Please find Memgraph compatible cypherl|.cypher files under $OUTPUT_SCHEMA, $OUTPUT_NODES, $OUTPUT_RELATIONSHIPS and $OUTPUT_CLEANUP" +echo "" +echo "Please import data by executing => \`cat $OUTPUT_SCHEMA | mgconsole\`, \`cat $OUTPUT_NODES | mgconsole\`, \`cat $OUTPUT_RELATIONSHIPS | mgconsole\` and \`cat $OUTPUT_CLEANUP | mgconsole\`" From 02325f8673a2d5c3bb160ccee413aa6f78b7c5c6 Mon Sep 17 00:00:00 2001 From: Antonio Filipovic <61245998+antoniofilipovic@users.noreply.github.com> Date: Thu, 7 Mar 2024 12:10:33 +0100 Subject: [PATCH 13/18] Fix bug prone add server to cluster behavior (#1792) --- src/coordination/raft_state.cpp | 33 ++++- .../coord_cluster_registration.py | 1 + .../high_availability/distributed_coords.py | 122 ++++++++++++++++++ 3 files changed, 152 insertions(+), 4 deletions(-) diff --git a/src/coordination/raft_state.cpp b/src/coordination/raft_state.cpp index d4d65cc36..dd441db74 100644 --- a/src/coordination/raft_state.cpp +++ b/src/coordination/raft_state.cpp @@ -11,10 +11,11 @@ #ifdef MG_ENTERPRISE -#include "coordination/raft_state.hpp" +#include <chrono> #include "coordination/coordinator_config.hpp" #include "coordination/coordinator_exceptions.hpp" +#include "coordination/raft_state.hpp" #include "utils/counter.hpp" namespace memgraph::coordination { @@ -110,10 +111,34 @@ auto RaftState::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_po -> void { auto const endpoint = fmt::format("{}:{}", raft_address, raft_port); srv_config const srv_config_to_add(static_cast<int>(raft_server_id), endpoint); - if (!raft_server_->add_srv(srv_config_to_add)->get_accepted()) { - throw RaftAddServerException("Failed to add server {} to the cluster", endpoint); + + auto cmd_result = raft_server_->add_srv(srv_config_to_add); + + if (cmd_result->get_result_code() == nuraft::cmd_result_code::OK) { + spdlog::info("Request to add server {} to the cluster accepted", endpoint); + } else { + throw RaftAddServerException("Failed to accept request to add server {} to the cluster", endpoint); + } + + // Waiting for server to join + constexpr int max_tries{10}; + auto maybe_stop = utils::ResettableCounter<max_tries>(); + constexpr int waiting_period{200}; + bool added{false}; + while (!maybe_stop()) { + std::this_thread::sleep_for(std::chrono::milliseconds(waiting_period)); + const auto server_config = raft_server_->get_srv_config(static_cast<nuraft::int32>(raft_server_id)); + if (server_config) { + spdlog::trace("Server with id {} added to cluster", raft_server_id); + added = true; + break; + } + } + + if (!added) { + throw RaftAddServerException("Failed to add server {} to the cluster in {}ms", endpoint, + max_tries * waiting_period); } - spdlog::info("Request to add server {} to the cluster accepted", endpoint); } auto RaftState::GetAllCoordinators() const -> std::vector<ptr<srv_config>> { diff --git a/tests/e2e/high_availability/coord_cluster_registration.py b/tests/e2e/high_availability/coord_cluster_registration.py index 774c6dca1..a285adcea 100644 --- a/tests/e2e/high_availability/coord_cluster_registration.py +++ b/tests/e2e/high_availability/coord_cluster_registration.py @@ -430,6 +430,7 @@ def test_unregister_main(): coordinator2_cursor = connect(host="localhost", port=7691).cursor() coordinator3_cursor = connect(host="localhost", port=7692).cursor() assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 1 ON '127.0.0.1:10111'") + assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 2 ON '127.0.0.1:10112'") execute_and_fetch_all( coordinator3_cursor, "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'" diff --git a/tests/e2e/high_availability/distributed_coords.py b/tests/e2e/high_availability/distributed_coords.py index 9fa654d68..33901f1d4 100644 --- a/tests/e2e/high_availability/distributed_coords.py +++ b/tests/e2e/high_availability/distributed_coords.py @@ -292,5 +292,127 @@ def test_distributed_automatic_failover_after_coord_dies(): mg_sleep_and_assert_collection(expected_data_on_new_main_old_alive, retrieve_data_show_replicas) +def test_registering_4_coords(): + # Goal of this test is to assure registering of multiple coordinators in row works + INSTANCES_DESCRIPTION = { + "instance_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7687", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10011", + ], + "log_file": "instance_1.log", + "data_directory": f"{TEMP_DIR}/instance_1", + "setup_queries": [], + }, + "instance_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7688", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10012", + ], + "log_file": "instance_2.log", + "data_directory": f"{TEMP_DIR}/instance_2", + "setup_queries": [], + }, + "instance_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7689", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10013", + ], + "log_file": "instance_3.log", + "data_directory": f"{TEMP_DIR}/instance_3", + "setup_queries": [], + }, + "coordinator_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7690", + "--log-level=TRACE", + "--raft-server-id=1", + "--raft-server-port=10111", + ], + "log_file": "coordinator1.log", + "setup_queries": [], + }, + "coordinator_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7691", + "--log-level=TRACE", + "--raft-server-id=2", + "--raft-server-port=10112", + ], + "log_file": "coordinator2.log", + "setup_queries": [], + }, + "coordinator_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7692", + "--log-level=TRACE", + "--raft-server-id=3", + "--raft-server-port=10113", + ], + "log_file": "coordinator3.log", + "setup_queries": [], + }, + "coordinator_4": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7693", + "--log-level=TRACE", + "--raft-server-id=4", + "--raft-server-port=10114", + ], + "log_file": "coordinator4.log", + "setup_queries": [ + "ADD COORDINATOR 1 ON '127.0.0.1:10111';", + "ADD COORDINATOR 2 ON '127.0.0.1:10112';", + "ADD COORDINATOR 3 ON '127.0.0.1:10113';", + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002'", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003'", + "SET INSTANCE instance_3 TO MAIN", + ], + }, + } + + interactive_mg_runner.start_all(INSTANCES_DESCRIPTION) + + coord_cursor = connect(host="localhost", port=7693).cursor() + + def retrieve_data_show_repl_cluster(): + return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) + + expected_data_on_coord = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("coordinator_4", "127.0.0.1:10114", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) + + if __name__ == "__main__": sys.exit(pytest.main([__file__, "-rA"])) From a099417c563a2cedda933ef5f07eb8d0b7f9f0e8 Mon Sep 17 00:00:00 2001 From: Aidar Samerkhanov <aidar.samerkhanov@memgraph.io> Date: Thu, 7 Mar 2024 18:41:02 +0400 Subject: [PATCH 14/18] List Pattern Comprehension planner (#1686) --- src/query/frontend/ast/ast.hpp | 9 +- .../frontend/semantic/symbol_generator.cpp | 26 +++++ .../frontend/semantic/symbol_generator.hpp | 4 +- src/query/plan/hint_provider.hpp | 10 +- src/query/plan/operator.cpp | 21 ++++ src/query/plan/operator.hpp | 38 ++++++- src/query/plan/operator_type_info.cpp | 5 +- src/query/plan/preprocess.cpp | 45 +++++++-- src/query/plan/preprocess.hpp | 44 ++++++--- src/query/plan/pretty_print.cpp | 9 +- src/query/plan/pretty_print.hpp | 3 +- src/query/plan/rewrite/index_lookup.hpp | 12 +++ src/query/plan/rewrite/join.hpp | 14 ++- src/query/plan/rule_based_planner.cpp | 62 +++++++++--- src/query/plan/rule_based_planner.hpp | 39 +++++--- src/utils/typeinfo.hpp | 1 + .../features/list_operations.feature | 42 ++++++++ .../memgraph_V1/graphs/graph_keanu.cypher | 7 +- tests/unit/cypher_main_visitor.cpp | 98 +++++++++++++++++++ tests/unit/query_common.hpp | 4 +- tests/unit/query_semantic.cpp | 24 +++++ 21 files changed, 460 insertions(+), 57 deletions(-) diff --git a/src/query/frontend/ast/ast.hpp b/src/query/frontend/ast/ast.hpp index ad3cdb7c5..ed354f6ca 100644 --- a/src/query/frontend/ast/ast.hpp +++ b/src/query/frontend/ast/ast.hpp @@ -21,6 +21,7 @@ #include "query/interpret/awesome_memgraph_functions.hpp" #include "query/typed_value.hpp" #include "storage/v2/property_value.hpp" +#include "utils/exceptions.hpp" #include "utils/typeinfo.hpp" namespace memgraph::query { @@ -3586,7 +3587,7 @@ class PatternComprehension : public memgraph::query::Expression { bool Accept(HierarchicalTreeVisitor &visitor) override { if (visitor.PreVisit(*this)) { if (variable_) { - variable_->Accept(visitor); + throw utils::NotYetImplemented("Variable in pattern comprehension."); } pattern_->Accept(visitor); if (filter_) { @@ -3615,7 +3616,8 @@ class PatternComprehension : public memgraph::query::Expression { int32_t symbol_pos_{-1}; PatternComprehension *Clone(AstStorage *storage) const override { - PatternComprehension *object = storage->Create<PatternComprehension>(); + auto *object = storage->Create<PatternComprehension>(); + object->variable_ = variable_ ? variable_->Clone(storage) : nullptr; object->pattern_ = pattern_ ? pattern_->Clone(storage) : nullptr; object->filter_ = filter_ ? filter_->Clone(storage) : nullptr; object->resultExpr_ = resultExpr_ ? resultExpr_->Clone(storage) : nullptr; @@ -3625,7 +3627,8 @@ class PatternComprehension : public memgraph::query::Expression { } protected: - PatternComprehension(Identifier *variable, Pattern *pattern) : variable_(variable), pattern_(pattern) {} + PatternComprehension(Identifier *variable, Pattern *pattern, Where *filter, Expression *resultExpr) + : variable_(variable), pattern_(pattern), filter_(filter), resultExpr_(resultExpr) {} private: friend class AstStorage; diff --git a/src/query/frontend/semantic/symbol_generator.cpp b/src/query/frontend/semantic/symbol_generator.cpp index e8ef3cba5..2cfbee584 100644 --- a/src/query/frontend/semantic/symbol_generator.cpp +++ b/src/query/frontend/semantic/symbol_generator.cpp @@ -721,6 +721,32 @@ bool SymbolGenerator::PostVisit(EdgeAtom &) { return true; } +bool SymbolGenerator::PreVisit(PatternComprehension &pc) { + auto &scope = scopes_.back(); + + if (scope.in_set_property) { + throw utils::NotYetImplemented("Pattern Comprehension cannot be used within SET clause.!"); + } + + if (scope.in_with) { + throw utils::NotYetImplemented("Pattern Comprehension cannot be used within WITH!"); + } + + if (scope.in_reduce) { + throw utils::NotYetImplemented("Pattern Comprehension cannot be used within REDUCE!"); + } + + if (scope.num_if_operators) { + throw utils::NotYetImplemented("IF operator cannot be used with Pattern Comprehension!"); + } + + const auto &symbol = CreateAnonymousSymbol(); + pc.MapTo(symbol); + return true; +} + +bool SymbolGenerator::PostVisit(PatternComprehension & /*pc*/) { return true; } + void SymbolGenerator::VisitWithIdentifiers(Expression *expr, const std::vector<Identifier *> &identifiers) { auto &scope = scopes_.back(); std::vector<std::pair<std::optional<Symbol>, Identifier *>> prev_symbols; diff --git a/src/query/frontend/semantic/symbol_generator.hpp b/src/query/frontend/semantic/symbol_generator.hpp index f9e6468f6..e5b46fbfe 100644 --- a/src/query/frontend/semantic/symbol_generator.hpp +++ b/src/query/frontend/semantic/symbol_generator.hpp @@ -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 @@ -97,6 +97,8 @@ class SymbolGenerator : public HierarchicalTreeVisitor { bool PostVisit(NodeAtom &) override; bool PreVisit(EdgeAtom &) override; bool PostVisit(EdgeAtom &) override; + bool PreVisit(PatternComprehension &) override; + bool PostVisit(PatternComprehension &) override; private: // Scope stores the state of where we are when visiting the AST and a map of diff --git a/src/query/plan/hint_provider.hpp b/src/query/plan/hint_provider.hpp index 74dde2f46..b70de9aaf 100644 --- a/src/query/plan/hint_provider.hpp +++ b/src/query/plan/hint_provider.hpp @@ -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 @@ -206,6 +206,14 @@ class PlanHintsProvider final : public HierarchicalLogicalOperatorVisitor { bool PostVisit(IndexedJoin & /*unused*/) override { return true; } + bool PreVisit(RollUpApply &op) override { + op.input()->Accept(*this); + op.list_collection_branch_->Accept(*this); + return false; + } + + bool PostVisit(RollUpApply & /*unused*/) override { return true; } + private: const SymbolTable &symbol_table_; std::vector<std::string> hints_; diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp index 75b531261..ba421b653 100644 --- a/src/query/plan/operator.cpp +++ b/src/query/plan/operator.cpp @@ -5624,4 +5624,25 @@ UniqueCursorPtr HashJoin::MakeCursor(utils::MemoryResource *mem) const { return MakeUniqueCursorPtr<HashJoinCursor>(mem, *this, mem); } +RollUpApply::RollUpApply(const std::shared_ptr<LogicalOperator> &input, + std::shared_ptr<LogicalOperator> &&second_branch) + : input_(input), list_collection_branch_(second_branch) {} + +std::vector<Symbol> RollUpApply::OutputSymbols(const SymbolTable & /*symbol_table*/) const { + std::vector<Symbol> symbols; + return symbols; +} + +std::vector<Symbol> RollUpApply::ModifiedSymbols(const SymbolTable &table) const { return OutputSymbols(table); } + +bool RollUpApply::Accept(HierarchicalLogicalOperatorVisitor &visitor) { + if (visitor.PreVisit(*this)) { + if (!input_ || !list_collection_branch_) { + throw utils::NotYetImplemented("One of the branches in pattern comprehension is null! Please contact support."); + } + input_->Accept(visitor) && list_collection_branch_->Accept(visitor); + } + return visitor.PostVisit(*this); +} + } // namespace memgraph::query::plan diff --git a/src/query/plan/operator.hpp b/src/query/plan/operator.hpp index 516ef2e38..cdaca2875 100644 --- a/src/query/plan/operator.hpp +++ b/src/query/plan/operator.hpp @@ -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 @@ -130,6 +130,7 @@ class EvaluatePatternFilter; class Apply; class IndexedJoin; class HashJoin; +class RollUpApply; using LogicalOperatorCompositeVisitor = utils::CompositeVisitor<Once, CreateNode, CreateExpand, ScanAll, ScanAllByLabel, ScanAllByLabelPropertyRange, @@ -137,7 +138,7 @@ using LogicalOperatorCompositeVisitor = ConstructNamedPath, Filter, Produce, Delete, SetProperty, SetProperties, SetLabels, RemoveProperty, RemoveLabels, EdgeUniquenessFilter, Accumulate, Aggregate, Skip, Limit, OrderBy, Merge, Optional, Unwind, Distinct, Union, Cartesian, CallProcedure, LoadCsv, - Foreach, EmptyResult, EvaluatePatternFilter, Apply, IndexedJoin, HashJoin>; + Foreach, EmptyResult, EvaluatePatternFilter, Apply, IndexedJoin, HashJoin, RollUpApply>; using LogicalOperatorLeafVisitor = utils::LeafVisitor<Once>; @@ -2634,5 +2635,38 @@ class HashJoin : public memgraph::query::plan::LogicalOperator { } }; +/// RollUpApply operator is used to execute an expression which takes as input a pattern, +/// and returns a list with content from the matched pattern +/// It's used for a pattern expression or pattern comprehension in a query. +class RollUpApply : public memgraph::query::plan::LogicalOperator { + public: + static const utils::TypeInfo kType; + const utils::TypeInfo &GetTypeInfo() const override { return kType; } + + RollUpApply() = default; + RollUpApply(const std::shared_ptr<LogicalOperator> &input, std::shared_ptr<LogicalOperator> &&second_branch); + + bool HasSingleInput() const override { return false; } + std::shared_ptr<LogicalOperator> input() const override { return input_; } + void set_input(std::shared_ptr<LogicalOperator> input) override { input_ = input; } + + bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override; + UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override { + throw utils::NotYetImplemented("Execution of Pattern comprehension is currently unsupported."); + } + std::vector<Symbol> OutputSymbols(const SymbolTable &) const override; + std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override; + + std::unique_ptr<LogicalOperator> Clone(AstStorage *storage) const override { + auto object = std::make_unique<RollUpApply>(); + object->input_ = input_ ? input_->Clone(storage) : nullptr; + object->list_collection_branch_ = list_collection_branch_ ? list_collection_branch_->Clone(storage) : nullptr; + return object; + } + + std::shared_ptr<memgraph::query::plan::LogicalOperator> input_; + std::shared_ptr<memgraph::query::plan::LogicalOperator> list_collection_branch_; +}; + } // namespace plan } // namespace memgraph::query diff --git a/src/query/plan/operator_type_info.cpp b/src/query/plan/operator_type_info.cpp index 3b3ffe14e..168137552 100644 --- a/src/query/plan/operator_type_info.cpp +++ b/src/query/plan/operator_type_info.cpp @@ -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 @@ -154,4 +154,7 @@ constexpr utils::TypeInfo query::plan::IndexedJoin::kType{utils::TypeId::INDEXED constexpr utils::TypeInfo query::plan::HashJoin::kType{utils::TypeId::HASH_JOIN, "HashJoin", &query::plan::LogicalOperator::kType}; + +constexpr utils::TypeInfo query::plan::RollUpApply::kType{utils::TypeId::ROLLUP_APPLY, "RollUpApply", + &query::plan::LogicalOperator::kType}; } // namespace memgraph diff --git a/src/query/plan/preprocess.cpp b/src/query/plan/preprocess.cpp index c3bfdf462..1554ce8ce 100644 --- a/src/query/plan/preprocess.cpp +++ b/src/query/plan/preprocess.cpp @@ -632,20 +632,20 @@ void AddMatching(const Match &match, SymbolTable &symbol_table, AstStorage &stor // If there are any pattern filters, we add those as well for (auto &filter : matching.filters) { - PatternFilterVisitor visitor(symbol_table, storage); + PatternVisitor visitor(symbol_table, storage); filter.expression->Accept(visitor); - filter.matchings = visitor.getMatchings(); + filter.matchings = visitor.getFilterMatchings(); } } -PatternFilterVisitor::PatternFilterVisitor(SymbolTable &symbol_table, AstStorage &storage) +PatternVisitor::PatternVisitor(SymbolTable &symbol_table, AstStorage &storage) : symbol_table_(symbol_table), storage_(storage) {} -PatternFilterVisitor::PatternFilterVisitor(const PatternFilterVisitor &) = default; -PatternFilterVisitor::PatternFilterVisitor(PatternFilterVisitor &&) noexcept = default; -PatternFilterVisitor::~PatternFilterVisitor() = default; +PatternVisitor::PatternVisitor(const PatternVisitor &) = default; +PatternVisitor::PatternVisitor(PatternVisitor &&) noexcept = default; +PatternVisitor::~PatternVisitor() = default; -void PatternFilterVisitor::Visit(Exists &op) { +void PatternVisitor::Visit(Exists &op) { std::vector<Pattern *> patterns; patterns.push_back(op.pattern_); @@ -655,10 +655,10 @@ void PatternFilterVisitor::Visit(Exists &op) { filter_matching.type = PatternFilterType::EXISTS; filter_matching.symbol = std::make_optional<Symbol>(symbol_table_.at(op)); - matchings_.push_back(std::move(filter_matching)); + filter_matchings_.push_back(std::move(filter_matching)); } -std::vector<FilterMatching> PatternFilterVisitor::getMatchings() { return matchings_; } +std::vector<FilterMatching> PatternVisitor::getFilterMatchings() { return filter_matchings_; } static void ParseForeach(query::Foreach &foreach, SingleQueryPart &query_part, AstStorage &storage, SymbolTable &symbol_table) { @@ -672,6 +672,30 @@ static void ParseForeach(query::Foreach &foreach, SingleQueryPart &query_part, A } } +static void ParseReturn(query::Return &ret, AstStorage &storage, SymbolTable &symbol_table, + std::unordered_map<std::string, PatternComprehensionMatching> &matchings) { + PatternVisitor visitor(symbol_table, storage); + + for (auto *expr : ret.body_.named_expressions) { + expr->Accept(visitor); + auto pattern_comprehension_matchings = visitor.getPatternComprehensionMatchings(); + for (auto &matching : pattern_comprehension_matchings) { + matchings.emplace(expr->name_, matching); + } + } +} + +void PatternVisitor::Visit(NamedExpression &op) { op.expression_->Accept(*this); } + +void PatternVisitor::Visit(PatternComprehension &op) { + PatternComprehensionMatching matching; + AddMatching({op.pattern_}, op.filter_, symbol_table_, storage_, matching); + matching.result_expr = storage_.Create<NamedExpression>(symbol_table_.at(op).name(), op.resultExpr_); + matching.result_expr->MapTo(symbol_table_.at(op)); + + pattern_comprehension_matchings_.push_back(std::move(matching)); +} + // Converts a Query to multiple QueryParts. In the process new Ast nodes may be // created, e.g. filter expressions. std::vector<SingleQueryPart> CollectSingleQueryParts(SymbolTable &symbol_table, AstStorage &storage, @@ -703,7 +727,8 @@ std::vector<SingleQueryPart> CollectSingleQueryParts(SymbolTable &symbol_table, // This query part is done, continue with a new one. query_parts.emplace_back(SingleQueryPart{}); query_part = &query_parts.back(); - } else if (utils::IsSubtype(*clause, Return::kType)) { + } else if (auto *ret = utils::Downcast<Return>(clause)) { + ParseReturn(*ret, storage, symbol_table, query_part->pattern_comprehension_matchings); return query_parts; } } diff --git a/src/query/plan/preprocess.hpp b/src/query/plan/preprocess.hpp index 01b10ebaf..b1ad253c4 100644 --- a/src/query/plan/preprocess.hpp +++ b/src/query/plan/preprocess.hpp @@ -153,19 +153,20 @@ struct Expansion { ExpansionGroupId expansion_group_id = ExpansionGroupId(); }; +struct PatternComprehensionMatching; struct FilterMatching; enum class PatternFilterType { EXISTS }; -/// Collects matchings from filters that include patterns -class PatternFilterVisitor : public ExpressionVisitor<void> { +/// Collects matchings that include patterns +class PatternVisitor : public ExpressionVisitor<void> { public: - explicit PatternFilterVisitor(SymbolTable &symbol_table, AstStorage &storage); - PatternFilterVisitor(const PatternFilterVisitor &); - PatternFilterVisitor &operator=(const PatternFilterVisitor &) = delete; - PatternFilterVisitor(PatternFilterVisitor &&) noexcept; - PatternFilterVisitor &operator=(PatternFilterVisitor &&) noexcept = delete; - ~PatternFilterVisitor() override; + explicit PatternVisitor(SymbolTable &symbol_table, AstStorage &storage); + PatternVisitor(const PatternVisitor &); + PatternVisitor &operator=(const PatternVisitor &) = delete; + PatternVisitor(PatternVisitor &&) noexcept; + PatternVisitor &operator=(PatternVisitor &&) noexcept = delete; + ~PatternVisitor() override; using ExpressionVisitor<void>::Visit; @@ -233,18 +234,24 @@ class PatternFilterVisitor : public ExpressionVisitor<void> { void Visit(PropertyLookup &op) override{}; void Visit(AllPropertiesLookup &op) override{}; void Visit(ParameterLookup &op) override{}; - void Visit(NamedExpression &op) override{}; void Visit(RegexMatch &op) override{}; - void Visit(PatternComprehension &op) override{}; + void Visit(NamedExpression &op) override; + void Visit(PatternComprehension &op) override; - std::vector<FilterMatching> getMatchings(); + std::vector<FilterMatching> getFilterMatchings(); + std::vector<PatternComprehensionMatching> getPatternComprehensionMatchings() { + return pattern_comprehension_matchings_; + } SymbolTable &symbol_table_; AstStorage &storage_; private: /// Collection of matchings in the filter expression being analyzed. - std::vector<FilterMatching> matchings_; + std::vector<FilterMatching> filter_matchings_; + + /// Collection of matchings in the pattern comprehension being analyzed. + std::vector<PatternComprehensionMatching> pattern_comprehension_matchings_; }; /// Stores the symbols and expression used to filter a property. @@ -495,6 +502,11 @@ inline auto Filters::IdFilters(const Symbol &symbol) const -> std::vector<Filter return filters; } +struct PatternComprehensionMatching : Matching { + /// Pattern comprehension result named expression + NamedExpression *result_expr = nullptr; +}; + /// @brief Represents a read (+ write) part of a query. Parts are split on /// `WITH` clauses. /// @@ -537,6 +549,14 @@ struct SingleQueryPart { /// in the `remaining_clauses` but rather in the `Foreach` itself and are guranteed /// to be processed in the same order by the semantics of the `RuleBasedPlanner`. std::vector<Matching> merge_matching{}; + + /// @brief @c NamedExpression name to @c PatternComprehensionMatching for each pattern comprehension. + /// + /// Storing the normalized pattern of a @c PatternComprehension does not preclude storing the + /// @c PatternComprehension clause itself inside `remaining_clauses`. The reason is that we + /// need to have access to other parts of the clause, such as pattern, filter clauses. + std::unordered_map<std::string, PatternComprehensionMatching> pattern_comprehension_matchings{}; + /// @brief All the remaining clauses (without @c Match). std::vector<Clause *> remaining_clauses{}; /// The subqueries vector are all the subqueries in this query part ordered in a list by diff --git a/src/query/plan/pretty_print.cpp b/src/query/plan/pretty_print.cpp index a2df9422c..7938f9c73 100644 --- a/src/query/plan/pretty_print.cpp +++ b/src/query/plan/pretty_print.cpp @@ -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 @@ -143,6 +143,13 @@ bool PlanPrinter::PreVisit(query::plan::Union &op) { return false; } +bool PlanPrinter::PreVisit(query::plan::RollUpApply &op) { + WithPrintLn([&op](auto &out) { out << "* " << op.ToString(); }); + Branch(*op.list_collection_branch_); + op.input_->Accept(*this); + return false; +} + bool PlanPrinter::PreVisit(query::plan::CallProcedure &op) { WithPrintLn([&op](auto &out) { out << "* " << op.ToString(); }); return true; diff --git a/src/query/plan/pretty_print.hpp b/src/query/plan/pretty_print.hpp index 645fe17a5..af8429b85 100644 --- a/src/query/plan/pretty_print.hpp +++ b/src/query/plan/pretty_print.hpp @@ -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 @@ -91,6 +91,7 @@ class PlanPrinter : public virtual HierarchicalLogicalOperatorVisitor { bool PreVisit(OrderBy &) override; bool PreVisit(Distinct &) override; bool PreVisit(Union &) override; + bool PreVisit(RollUpApply &) override; bool PreVisit(Unwind &) override; bool PreVisit(CallProcedure &) override; diff --git a/src/query/plan/rewrite/index_lookup.hpp b/src/query/plan/rewrite/index_lookup.hpp index 09c6e2014..90c222b42 100644 --- a/src/query/plan/rewrite/index_lookup.hpp +++ b/src/query/plan/rewrite/index_lookup.hpp @@ -595,6 +595,18 @@ class IndexLookupRewriter final : public HierarchicalLogicalOperatorVisitor { return true; } + bool PreVisit(RollUpApply &op) override { + prev_ops_.push_back(&op); + op.input()->Accept(*this); + RewriteBranch(&op.list_collection_branch_); + return false; + } + + bool PostVisit(RollUpApply &) override { + prev_ops_.pop_back(); + return true; + } + std::shared_ptr<LogicalOperator> new_root_; private: diff --git a/src/query/plan/rewrite/join.hpp b/src/query/plan/rewrite/join.hpp index e346ded45..9ef6c6aec 100644 --- a/src/query/plan/rewrite/join.hpp +++ b/src/query/plan/rewrite/join.hpp @@ -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 @@ -455,6 +455,18 @@ class JoinRewriter final : public HierarchicalLogicalOperatorVisitor { return true; } + bool PreVisit(RollUpApply &op) override { + prev_ops_.push_back(&op); + op.input()->Accept(*this); + RewriteBranch(&op.list_collection_branch_); + return false; + } + + bool PostVisit(RollUpApply &) override { + prev_ops_.pop_back(); + return true; + } + std::shared_ptr<LogicalOperator> new_root_; private: diff --git a/src/query/plan/rule_based_planner.cpp b/src/query/plan/rule_based_planner.cpp index bf5e66158..54b5c3834 100644 --- a/src/query/plan/rule_based_planner.cpp +++ b/src/query/plan/rule_based_planner.cpp @@ -14,9 +14,12 @@ #include <algorithm> #include <functional> #include <limits> +#include <memory> #include <stack> #include <unordered_set> +#include "query/frontend/ast/ast.hpp" +#include "query/plan/operator.hpp" #include "query/plan/preprocess.hpp" #include "utils/algorithm.hpp" #include "utils/exceptions.hpp" @@ -40,7 +43,8 @@ namespace { class ReturnBodyContext : public HierarchicalTreeVisitor { public: ReturnBodyContext(const ReturnBody &body, SymbolTable &symbol_table, const std::unordered_set<Symbol> &bound_symbols, - AstStorage &storage, Where *where = nullptr) + AstStorage &storage, std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops, + Where *where = nullptr) : body_(body), symbol_table_(symbol_table), bound_symbols_(bound_symbols), storage_(storage), where_(where) { // Collect symbols from named expressions. output_symbols_.reserve(body_.named_expressions.size()); @@ -53,6 +57,14 @@ class ReturnBodyContext : public HierarchicalTreeVisitor { output_symbols_.emplace_back(symbol_table_.at(*named_expr)); named_expr->Accept(*this); named_expressions_.emplace_back(named_expr); + if (pattern_comprehension_) { + if (auto it = pc_ops.find(named_expr->name_); it != pc_ops.end()) { + pattern_comprehension_op_ = std::move(it->second); + pc_ops.erase(it); + } else { + throw utils::NotYetImplemented("Operation on top of pattern comprehension"); + } + } } // Collect symbols used in group by expressions. if (!aggregations_.empty()) { @@ -386,8 +398,20 @@ class ReturnBodyContext : public HierarchicalTreeVisitor { return true; } - bool PostVisit(PatternComprehension & /*unused*/) override { - throw utils::NotYetImplemented("Planner can not handle pattern comprehension."); + bool PreVisit(PatternComprehension & /*unused*/) override { + pattern_compression_aggregations_start_index_ = has_aggregation_.size(); + return true; + } + + bool PostVisit(PatternComprehension &pattern_comprehension) override { + bool has_aggr = false; + for (auto i = has_aggregation_.size(); i > pattern_compression_aggregations_start_index_; --i) { + has_aggr |= has_aggregation_.back(); + has_aggregation_.pop_back(); + } + has_aggregation_.emplace_back(has_aggr); + pattern_comprehension_ = &pattern_comprehension; + return true; } // Creates NamedExpression with an Identifier for each user declared symbol. @@ -444,6 +468,10 @@ class ReturnBodyContext : public HierarchicalTreeVisitor { // named_expressions. const auto &output_symbols() const { return output_symbols_; } + const auto *pattern_comprehension() const { return pattern_comprehension_; } + + std::shared_ptr<LogicalOperator> pattern_comprehension_op() const { return pattern_comprehension_op_; } + private: const ReturnBody &body_; SymbolTable &symbol_table_; @@ -465,10 +493,13 @@ class ReturnBodyContext : public HierarchicalTreeVisitor { // group by it. std::list<bool> has_aggregation_; std::vector<NamedExpression *> named_expressions_; + PatternComprehension *pattern_comprehension_ = nullptr; + std::shared_ptr<LogicalOperator> pattern_comprehension_op_; + size_t pattern_compression_aggregations_start_index_ = 0; }; std::unique_ptr<LogicalOperator> GenReturnBody(std::unique_ptr<LogicalOperator> input_op, bool advance_command, - const ReturnBodyContext &body, bool accumulate = false) { + const ReturnBodyContext &body, bool accumulate) { std::vector<Symbol> used_symbols(body.used_symbols().begin(), body.used_symbols().end()); auto last_op = std::move(input_op); if (accumulate) { @@ -482,6 +513,11 @@ std::unique_ptr<LogicalOperator> GenReturnBody(std::unique_ptr<LogicalOperator> std::vector<Symbol> remember(body.group_by_used_symbols().begin(), body.group_by_used_symbols().end()); last_op = std::make_unique<Aggregate>(std::move(last_op), body.aggregations(), body.group_by(), remember); } + + if (body.pattern_comprehension()) { + last_op = std::make_unique<RollUpApply>(std::move(last_op), body.pattern_comprehension_op()); + } + last_op = std::make_unique<Produce>(std::move(last_op), body.named_expressions()); // Distinct in ReturnBody only makes Produce values unique, so plan after it. if (body.distinct()) { @@ -506,6 +542,7 @@ std::unique_ptr<LogicalOperator> GenReturnBody(std::unique_ptr<LogicalOperator> last_op = std::make_unique<Filter>(std::move(last_op), std::vector<std::shared_ptr<LogicalOperator>>{}, body.where()->expression_); } + return last_op; } @@ -543,8 +580,9 @@ Expression *ExtractFilters(const std::unordered_set<Symbol> &bound_symbols, Filt return filter_expr; } -std::unordered_set<Symbol> GetSubqueryBoundSymbols(const std::vector<SingleQueryPart> &single_query_parts, - SymbolTable &symbol_table, AstStorage &storage) { +std::unordered_set<Symbol> GetSubqueryBoundSymbols( + const std::vector<SingleQueryPart> &single_query_parts, SymbolTable &symbol_table, AstStorage &storage, + std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) { const auto &query = single_query_parts[0]; if (!query.matching.expansions.empty() || query.remaining_clauses.empty()) { @@ -552,7 +590,7 @@ std::unordered_set<Symbol> GetSubqueryBoundSymbols(const std::vector<SingleQuery } if (std::unordered_set<Symbol> bound_symbols; auto *with = utils::Downcast<query::With>(query.remaining_clauses[0])) { - auto input_op = impl::GenWith(*with, nullptr, symbol_table, false, bound_symbols, storage); + auto input_op = impl::GenWith(*with, nullptr, symbol_table, false, bound_symbols, storage, pc_ops); return bound_symbols; } @@ -583,7 +621,8 @@ std::unique_ptr<LogicalOperator> GenNamedPaths(std::unique_ptr<LogicalOperator> std::unique_ptr<LogicalOperator> GenReturn(Return &ret, std::unique_ptr<LogicalOperator> input_op, SymbolTable &symbol_table, bool is_write, - const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage) { + const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage, + std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) { // Similar to WITH clause, but we want to accumulate when the query writes to // the database. This way we handle the case when we want to return // expressions with the latest updated results. For example, `MATCH (n) -- () @@ -592,13 +631,14 @@ std::unique_ptr<LogicalOperator> GenReturn(Return &ret, std::unique_ptr<LogicalO // final result of 'k' increments. bool accumulate = is_write; bool advance_command = false; - ReturnBodyContext body(ret.body_, symbol_table, bound_symbols, storage); + ReturnBodyContext body(ret.body_, symbol_table, bound_symbols, storage, pc_ops); return GenReturnBody(std::move(input_op), advance_command, body, accumulate); } std::unique_ptr<LogicalOperator> GenWith(With &with, std::unique_ptr<LogicalOperator> input_op, SymbolTable &symbol_table, bool is_write, - std::unordered_set<Symbol> &bound_symbols, AstStorage &storage) { + std::unordered_set<Symbol> &bound_symbols, AstStorage &storage, + std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) { // WITH clause is Accumulate/Aggregate (advance_command) + Produce and // optional Filter. In case of update and aggregation, we want to accumulate // first, so that when aggregating, we get the latest results. Similar to @@ -606,7 +646,7 @@ std::unique_ptr<LogicalOperator> GenWith(With &with, std::unique_ptr<LogicalOper bool accumulate = is_write; // No need to advance the command if we only performed reads. bool advance_command = is_write; - ReturnBodyContext body(with.body_, symbol_table, bound_symbols, storage, with.where_); + ReturnBodyContext body(with.body_, symbol_table, bound_symbols, storage, pc_ops, with.where_); auto last_op = GenReturnBody(std::move(input_op), advance_command, body, accumulate); // Reset bound symbols, so that only those in WITH are exposed. bound_symbols.clear(); diff --git a/src/query/plan/rule_based_planner.hpp b/src/query/plan/rule_based_planner.hpp index 7fba3b623..27f46e764 100644 --- a/src/query/plan/rule_based_planner.hpp +++ b/src/query/plan/rule_based_planner.hpp @@ -21,6 +21,7 @@ #include "query/frontend/ast/ast_visitor.hpp" #include "query/plan/operator.hpp" #include "query/plan/preprocess.hpp" +#include "utils/exceptions.hpp" #include "utils/logging.hpp" #include "utils/typeinfo.hpp" @@ -87,8 +88,9 @@ bool HasBoundFilterSymbols(const std::unordered_set<Symbol> &bound_symbols, cons // Returns the set of symbols for the subquery that are actually referenced from the outer scope and // used in the subquery. -std::unordered_set<Symbol> GetSubqueryBoundSymbols(const std::vector<SingleQueryPart> &single_query_parts, - SymbolTable &symbol_table, AstStorage &storage); +std::unordered_set<Symbol> GetSubqueryBoundSymbols( + const std::vector<SingleQueryPart> &single_query_parts, SymbolTable &symbol_table, AstStorage &storage, + std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops); Symbol GetSymbol(NodeAtom *atom, const SymbolTable &symbol_table); Symbol GetSymbol(EdgeAtom *atom, const SymbolTable &symbol_table); @@ -142,11 +144,13 @@ std::unique_ptr<LogicalOperator> GenNamedPaths(std::unique_ptr<LogicalOperator> std::unique_ptr<LogicalOperator> GenReturn(Return &ret, std::unique_ptr<LogicalOperator> input_op, SymbolTable &symbol_table, bool is_write, - const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage); + const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage, + std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops); std::unique_ptr<LogicalOperator> GenWith(With &with, std::unique_ptr<LogicalOperator> input_op, SymbolTable &symbol_table, bool is_write, - std::unordered_set<Symbol> &bound_symbols, AstStorage &storage); + std::unordered_set<Symbol> &bound_symbols, AstStorage &storage, + std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops); std::unique_ptr<LogicalOperator> GenUnion(const CypherUnion &cypher_union, std::shared_ptr<LogicalOperator> left_op, std::shared_ptr<LogicalOperator> right_op, SymbolTable &symbol_table); @@ -190,11 +194,24 @@ class RuleBasedPlanner { uint64_t merge_id = 0; uint64_t subquery_id = 0; + std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pattern_comprehension_ops; + + if (single_query_part.pattern_comprehension_matchings.size() > 1) { + throw utils::NotYetImplemented("Multiple pattern comprehensions."); + } + for (const auto &matching : single_query_part.pattern_comprehension_matchings) { + std::unique_ptr<LogicalOperator> new_input; + MatchContext match_ctx{matching.second, *context.symbol_table, context.bound_symbols}; + new_input = PlanMatching(match_ctx, std::move(new_input)); + new_input = std::make_unique<Produce>(std::move(new_input), std::vector{matching.second.result_expr}); + pattern_comprehension_ops.emplace(matching.first, std::move(new_input)); + } + for (const auto &clause : single_query_part.remaining_clauses) { MG_ASSERT(!utils::IsSubtype(*clause, Match::kType), "Unexpected Match in remaining clauses"); if (auto *ret = utils::Downcast<Return>(clause)) { input_op = impl::GenReturn(*ret, std::move(input_op), *context.symbol_table, context.is_write_query, - context.bound_symbols, *context.ast_storage); + context.bound_symbols, *context.ast_storage, pattern_comprehension_ops); } else if (auto *merge = utils::Downcast<query::Merge>(clause)) { input_op = GenMerge(*merge, std::move(input_op), single_query_part.merge_matching[merge_id++]); // Treat MERGE clause as write, because we do not know if it will @@ -202,7 +219,7 @@ class RuleBasedPlanner { context.is_write_query = true; } else if (auto *with = utils::Downcast<query::With>(clause)) { input_op = impl::GenWith(*with, std::move(input_op), *context.symbol_table, context.is_write_query, - context.bound_symbols, *context.ast_storage); + context.bound_symbols, *context.ast_storage, pattern_comprehension_ops); // WITH clause advances the command, so reset the flag. context.is_write_query = false; } else if (auto op = HandleWriteClause(clause, input_op, *context.symbol_table, context.bound_symbols)) { @@ -241,7 +258,7 @@ class RuleBasedPlanner { single_query_part, merge_id); } else if (auto *call_sub = utils::Downcast<query::CallSubquery>(clause)) { input_op = HandleSubquery(std::move(input_op), single_query_part.subqueries[subquery_id++], - *context.symbol_table, *context_->ast_storage); + *context.symbol_table, *context_->ast_storage, pattern_comprehension_ops); } else { throw utils::NotYetImplemented("clause '{}' conversion to operator(s)", clause->GetTypeInfo().name); } @@ -860,15 +877,15 @@ class RuleBasedPlanner { symbol); } - std::unique_ptr<LogicalOperator> HandleSubquery(std::unique_ptr<LogicalOperator> last_op, - std::shared_ptr<QueryParts> subquery, SymbolTable &symbol_table, - AstStorage &storage) { + std::unique_ptr<LogicalOperator> HandleSubquery( + std::unique_ptr<LogicalOperator> last_op, std::shared_ptr<QueryParts> subquery, SymbolTable &symbol_table, + AstStorage &storage, std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) { std::unordered_set<Symbol> outer_scope_bound_symbols; outer_scope_bound_symbols.insert(std::make_move_iterator(context_->bound_symbols.begin()), std::make_move_iterator(context_->bound_symbols.end())); context_->bound_symbols = - impl::GetSubqueryBoundSymbols(subquery->query_parts[0].single_query_parts, symbol_table, storage); + impl::GetSubqueryBoundSymbols(subquery->query_parts[0].single_query_parts, symbol_table, storage, pc_ops); auto subquery_op = Plan(*subquery); diff --git a/src/utils/typeinfo.hpp b/src/utils/typeinfo.hpp index 592473b8b..3ed6128fc 100644 --- a/src/utils/typeinfo.hpp +++ b/src/utils/typeinfo.hpp @@ -68,6 +68,7 @@ enum class TypeId : uint64_t { APPLY, INDEXED_JOIN, HASH_JOIN, + ROLLUP_APPLY, // Replication // NOTE: these NEED to be stable in the 2000+ range (see rpc version) diff --git a/tests/gql_behave/tests/memgraph_V1/features/list_operations.feature b/tests/gql_behave/tests/memgraph_V1/features/list_operations.feature index 8c5538d6b..a6a4b15d2 100644 --- a/tests/gql_behave/tests/memgraph_V1/features/list_operations.feature +++ b/tests/gql_behave/tests/memgraph_V1/features/list_operations.feature @@ -291,3 +291,45 @@ Feature: List operators # Then the result should be: # | years | # | [2021,2003,2003,1999] | + + Scenario: Multiple entries with list pattern comprehension + Given graph "graph_keanu" + When executing query: + """ + MATCH (n:Person) + RETURN n.name, [(n)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | b.released] AS years + """ + Then an error should be raised + + Scenario: Multiple list pattern comprehensions in Return + Given graph "graph_keanu" + When executing query: + """ + MATCH (n:Person) + RETURN n.name, + [(n)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | b.released] AS years, + [(n)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | b.title] AS titles + """ + Then an error should be raised + + Scenario: Function inside pattern comprehension's expression + Given graph "graph_keanu" + When executing query: + """ + MATCH (keanu:Person {name: 'Keanu Reeves'}) + RETURN [p = (keanu)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | size(nodes(p))] AS nodes + """ + Then an error should be raised + + Scenario: Multiple list pattern comprehensions in With + Given graph "graph_keanu" + When executing query: + """ + MATCH (n) WHERE size(n.name) > 5 + WITH + n AS actor, + [(n)-->(m) WHERE m.released > 2000 | m.title] AS titles, + [(n)-->(m) WHERE m.released > 2000 | m.released] AS years + RETURN actor.name, years, titles; + """ + Then an error should be raised diff --git a/tests/gql_behave/tests/memgraph_V1/graphs/graph_keanu.cypher b/tests/gql_behave/tests/memgraph_V1/graphs/graph_keanu.cypher index a7a72aced..98f48c3c1 100644 --- a/tests/gql_behave/tests/memgraph_V1/graphs/graph_keanu.cypher +++ b/tests/gql_behave/tests/memgraph_V1/graphs/graph_keanu.cypher @@ -1,5 +1,7 @@ CREATE (keanu:Person {name: 'Keanu Reeves'}), + (trinity:Person {name: 'Carrie-Anne Moss'}), + (morpheus:Person {name: 'Laurence Fishburne'}), (johnnyMnemonic:Movie {title: 'Johnny Mnemonic', released: 1995}), (theMatrixRevolutions:Movie {title: 'The Matrix Revolutions', released: 2003}), (theMatrixReloaded:Movie {title: 'The Matrix Reloaded', released: 2003}), @@ -13,4 +15,7 @@ CREATE (keanu)-[:ACTED_IN]->(theReplacements), (keanu)-[:ACTED_IN]->(theMatrix), (keanu)-[:ACTED_IN]->(theDevilsAdvocate), - (keanu)-[:ACTED_IN]->(theMatrixResurrections); + (keanu)-[:ACTED_IN]->(theMatrixResurrections), + (trinity)-[:ACTED_IN]->(theMatrix), + (trinity)-[:ACTED_IN]->(theMatrixReloaded), + (morpheus)-[:ACTED_IN]->(theMatrix); diff --git a/tests/unit/cypher_main_visitor.cpp b/tests/unit/cypher_main_visitor.cpp index 63cca3aa4..bcc6767f4 100644 --- a/tests/unit/cypher_main_visitor.cpp +++ b/tests/unit/cypher_main_visitor.cpp @@ -4624,3 +4624,101 @@ TEST_P(CypherMainVisitorTest, CallSubquery) { ASSERT_TRUE(nested_match); } } + +TEST_P(CypherMainVisitorTest, PatternComprehension) { + auto &ast_generator = *GetParam(); + { + const auto *query = + dynamic_cast<CypherQuery *>(ast_generator.ParseQuery("MATCH (n) RETURN [(n)-->(b) | b.val] AS res;")); + const auto *ret = dynamic_cast<Return *>(query->single_query_->clauses_[1]); + + const auto *pc = dynamic_cast<PatternComprehension *>(ret->body_.named_expressions[0]->expression_); + ASSERT_TRUE(pc); + + // Check for variable_ + EXPECT_EQ(pc->variable_, nullptr); + + // Check for pattern_ + const auto pattern = pc->pattern_; + ASSERT_TRUE(pattern->atoms_.size() == 3); + + const auto *node1 = dynamic_cast<NodeAtom *>(pattern->atoms_[0]); + const auto *edge = dynamic_cast<EdgeAtom *>(pattern->atoms_[1]); + const auto *node2 = dynamic_cast<NodeAtom *>(pattern->atoms_[2]); + + ASSERT_TRUE(node1); + ASSERT_TRUE(edge); + ASSERT_TRUE(node2); + + // Check for filter_ + EXPECT_EQ(pc->filter_, nullptr); + + // Check for resultExpr_ + const auto *result_expr = pc->resultExpr_; + ASSERT_TRUE(result_expr); + } + { + const auto *query = dynamic_cast<CypherQuery *>( + ast_generator.ParseQuery("MATCH (n) RETURN [(n)-->(b) WHERE b.id=1 | b.val] AS res;")); + const auto *ret = dynamic_cast<Return *>(query->single_query_->clauses_[1]); + + const auto *pc = dynamic_cast<PatternComprehension *>(ret->body_.named_expressions[0]->expression_); + ASSERT_TRUE(pc); + + // Check for variable_ + EXPECT_EQ(pc->variable_, nullptr); + + // Check for pattern_ + const auto pattern = pc->pattern_; + ASSERT_TRUE(pattern->atoms_.size() == 3); + + const auto *node1 = dynamic_cast<NodeAtom *>(pattern->atoms_[0]); + const auto *edge = dynamic_cast<EdgeAtom *>(pattern->atoms_[1]); + const auto *node2 = dynamic_cast<NodeAtom *>(pattern->atoms_[2]); + + ASSERT_TRUE(node1); + ASSERT_TRUE(edge); + ASSERT_TRUE(node2); + + // Check for filter_ + const auto *filter = pc->filter_; + ASSERT_TRUE(filter); + ASSERT_TRUE(filter->expression_); + + // Check for resultExpr_ + const auto *result_expr = pc->resultExpr_; + ASSERT_TRUE(result_expr); + } + { + const auto *query = dynamic_cast<CypherQuery *>( + ast_generator.ParseQuery("MATCH (n) RETURN [p = (n)-->(b) WHERE b.id=1 | b.val] AS res;")); + const auto *ret = dynamic_cast<Return *>(query->single_query_->clauses_[1]); + + const auto *pc = dynamic_cast<PatternComprehension *>(ret->body_.named_expressions[0]->expression_); + ASSERT_TRUE(pc); + + // Check for variable_ + ASSERT_TRUE(pc->variable_); + + // Check for pattern_ + const auto pattern = pc->pattern_; + ASSERT_TRUE(pattern->atoms_.size() == 3); + + const auto *node1 = dynamic_cast<NodeAtom *>(pattern->atoms_[0]); + const auto *edge = dynamic_cast<EdgeAtom *>(pattern->atoms_[1]); + const auto *node2 = dynamic_cast<NodeAtom *>(pattern->atoms_[2]); + + ASSERT_TRUE(node1); + ASSERT_TRUE(edge); + ASSERT_TRUE(node2); + + // Check for filter_ + const auto *filter = pc->filter_; + ASSERT_TRUE(filter); + ASSERT_TRUE(filter->expression_); + + // Check for resultExpr_ + const auto *result_expr = pc->resultExpr_; + ASSERT_TRUE(result_expr); + } +} diff --git a/tests/unit/query_common.hpp b/tests/unit/query_common.hpp index a14ef2d30..c18e06abf 100644 --- a/tests/unit/query_common.hpp +++ b/tests/unit/query_common.hpp @@ -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 @@ -635,3 +635,5 @@ auto GetForeach(AstStorage &storage, NamedExpression *named_expr, const std::vec #define DROP_USER(usernames) storage.Create<memgraph::query::DropUser>((usernames)) #define CALL_PROCEDURE(...) memgraph::query::test_common::GetCallProcedure(storage, __VA_ARGS__) #define CALL_SUBQUERY(...) memgraph::query::test_common::GetCallSubquery(this->storage, __VA_ARGS__) +#define PATTERN_COMPREHENSION(variable, pattern, filter, resultExpr) \ + this->storage.template Create<memgraph::query::PatternComprehension>(variable, pattern, filter, resultExpr) diff --git a/tests/unit/query_semantic.cpp b/tests/unit/query_semantic.cpp index c4bb966eb..50a52c828 100644 --- a/tests/unit/query_semantic.cpp +++ b/tests/unit/query_semantic.cpp @@ -1442,3 +1442,27 @@ TYPED_TEST(TestSymbolGenerator, PropertyCachingMixedLookups2) { ASSERT_TRUE(prop3_eval_mode == PropertyLookup::EvaluationMode::GET_ALL_PROPERTIES); ASSERT_TRUE(prop4_eval_mode == PropertyLookup::EvaluationMode::GET_ALL_PROPERTIES); } + +TYPED_TEST(TestSymbolGenerator, PatternComprehension) { + auto prop = this->dba.NameToProperty("prop"); + + // MATCH (n) RETURN [(n)-[edge]->(m) | m.prop] AS alias + auto query = QUERY(SINGLE_QUERY( + MATCH(PATTERN(NODE("n"))), + RETURN(NEXPR("alias", PATTERN_COMPREHENSION(nullptr, + PATTERN(NODE("n"), EDGE("edge", EdgeAtom::Direction::BOTH, {}, false), + NODE("m", std::nullopt, false)), + nullptr, PROPERTY_LOOKUP(this->dba, "m", prop)))))); + + auto symbol_table = MakeSymbolTable(query); + ASSERT_EQ(symbol_table.max_position(), 7); + + memgraph::query::plan::UsedSymbolsCollector collector(symbol_table); + auto *ret = dynamic_cast<Return *>(query->single_query_->clauses_[1]); + auto *pc = dynamic_cast<PatternComprehension *>(ret->body_.named_expressions[0]->expression_); + + pc->Accept(collector); + + // n, edge, m, Path + ASSERT_EQ(collector.symbols_.size(), 4); +} From 5ca98f9543b6def15369886f6fa8db28b22c70fa Mon Sep 17 00:00:00 2001 From: Andi <andi8647@gmail.com> Date: Thu, 7 Mar 2024 18:40:32 +0100 Subject: [PATCH 15/18] Fix snapshot creation in RSM and forbid multiple leaders (#1788) --- .../coordinator_cluster_state.cpp | 3 +- src/coordination/coordinator_instance.cpp | 9 +- src/coordination/coordinator_log_store.cpp | 42 +-- .../coordinator_state_machine.cpp | 41 ++- .../nuraft/coordinator_state_machine.hpp | 4 - src/coordination/raft_state.cpp | 16 +- src/query/interpreter.cpp | 2 +- .../coord_cluster_registration.py | 1 - .../high_availability/distributed_coords.py | 339 +++++++++++++++++- 9 files changed, 402 insertions(+), 55 deletions(-) diff --git a/src/coordination/coordinator_cluster_state.cpp b/src/coordination/coordinator_cluster_state.cpp index 2213a052f..cf6e1a574 100644 --- a/src/coordination/coordinator_cluster_state.cpp +++ b/src/coordination/coordinator_cluster_state.cpp @@ -111,8 +111,7 @@ auto CoordinatorClusterState::DoAction(TRaftLog log_entry, RaftLogAction log_act auto CoordinatorClusterState::Serialize(ptr<buffer> &data) -> void { auto lock = std::shared_lock{log_lock_}; - // .at(0) is hack to solve the problem with json serialization of map - auto const log = nlohmann::json{instances_}.at(0).dump(); + auto const log = nlohmann::json(instances_).dump(); data = buffer::alloc(sizeof(uint32_t) + log.size()); buffer_serializer bs(data); diff --git a/src/coordination/coordinator_instance.cpp b/src/coordination/coordinator_instance.cpp index 920fea3cb..791ffbc59 100644 --- a/src/coordination/coordinator_instance.cpp +++ b/src/coordination/coordinator_instance.cpp @@ -40,7 +40,7 @@ CoordinatorInstance::CoordinatorInstance() }); std::ranges::for_each(replicas, [this](auto &replica) { - spdlog::info("Starting replication instance {}", replica.config.instance_name); + spdlog::info("Started pinging replication instance {}", replica.config.instance_name); repl_instances_.emplace_back(this, replica.config, client_succ_cb_, client_fail_cb_, &CoordinatorInstance::ReplicaSuccessCallback, &CoordinatorInstance::ReplicaFailCallback); @@ -50,7 +50,7 @@ CoordinatorInstance::CoordinatorInstance() [](auto const &instance) { return instance.status == ReplicationRole::MAIN; }); std::ranges::for_each(main, [this](auto &main_instance) { - spdlog::info("Starting main instance {}", main_instance.config.instance_name); + spdlog::info("Started pinging main instance {}", main_instance.config.instance_name); repl_instances_.emplace_back(this, main_instance.config, client_succ_cb_, client_fail_cb_, &CoordinatorInstance::MainSuccessCallback, &CoordinatorInstance::MainFailCallback); @@ -314,17 +314,20 @@ auto CoordinatorInstance::RegisterReplicationInstance(CoordinatorClientConfig co return RegisterInstanceCoordinatorStatus::NOT_LEADER; } + auto const undo_action_ = [this]() { repl_instances_.pop_back(); }; + auto *new_instance = &repl_instances_.emplace_back(this, config, client_succ_cb_, client_fail_cb_, &CoordinatorInstance::ReplicaSuccessCallback, &CoordinatorInstance::ReplicaFailCallback); if (!new_instance->SendDemoteToReplicaRpc()) { spdlog::error("Failed to send demote to replica rpc for instance {}", config.instance_name); - repl_instances_.pop_back(); + undo_action_(); return RegisterInstanceCoordinatorStatus::RPC_FAILED; } if (!raft_state_.AppendRegisterReplicationInstanceLog(config)) { + undo_action_(); return RegisterInstanceCoordinatorStatus::RAFT_LOG_ERROR; } diff --git a/src/coordination/coordinator_log_store.cpp b/src/coordination/coordinator_log_store.cpp index 37126b747..d5e134492 100644 --- a/src/coordination/coordinator_log_store.cpp +++ b/src/coordination/coordinator_log_store.cpp @@ -62,34 +62,33 @@ ptr<log_entry> CoordinatorLogStore::last_entry() const { uint64_t CoordinatorLogStore::append(ptr<log_entry> &entry) { ptr<log_entry> clone = MakeClone(entry); - uint64_t next_slot{0}; - { - auto lock = std::lock_guard{logs_lock_}; - next_slot = start_idx_ + logs_.size() - 1; - logs_[next_slot] = clone; - } + + auto lock = std::lock_guard{logs_lock_}; + uint64_t next_slot = start_idx_ + logs_.size() - 1; + logs_[next_slot] = clone; + return next_slot; } +// TODO: (andi) I think this is used for resolving conflicts inside NuRaft, check... +// different compared to in_memory_log_store.cxx void CoordinatorLogStore::write_at(uint64_t index, ptr<log_entry> &entry) { ptr<log_entry> clone = MakeClone(entry); // Discard all logs equal to or greater than `index. - { - auto lock = std::lock_guard{logs_lock_}; - auto itr = logs_.lower_bound(index); - while (itr != logs_.end()) { - itr = logs_.erase(itr); - } - logs_[index] = clone; + auto lock = std::lock_guard{logs_lock_}; + auto itr = logs_.lower_bound(index); + while (itr != logs_.end()) { + itr = logs_.erase(itr); } + logs_[index] = clone; } ptr<std::vector<ptr<log_entry>>> CoordinatorLogStore::log_entries(uint64_t start, uint64_t end) { auto ret = cs_new<std::vector<ptr<log_entry>>>(); ret->resize(end - start); - for (uint64_t i = start, curr_index = 0; i < end; ++i, ++curr_index) { + for (uint64_t i = start, curr_index = 0; i < end; i++, curr_index++) { ptr<log_entry> src = nullptr; { auto lock = std::lock_guard{logs_lock_}; @@ -105,21 +104,14 @@ ptr<std::vector<ptr<log_entry>>> CoordinatorLogStore::log_entries(uint64_t start } ptr<log_entry> CoordinatorLogStore::entry_at(uint64_t index) { - ptr<log_entry> src = nullptr; - { - auto lock = std::lock_guard{logs_lock_}; - src = FindOrDefault_(index); - } + auto lock = std::lock_guard{logs_lock_}; + ptr<log_entry> src = FindOrDefault_(index); return MakeClone(src); } uint64_t CoordinatorLogStore::term_at(uint64_t index) { - uint64_t term = 0; - { - auto lock = std::lock_guard{logs_lock_}; - term = FindOrDefault_(index)->get_term(); - } - return term; + auto lock = std::lock_guard{logs_lock_}; + return FindOrDefault_(index)->get_term(); } ptr<buffer> CoordinatorLogStore::pack(uint64_t index, int32 cnt) { diff --git a/src/coordination/coordinator_state_machine.cpp b/src/coordination/coordinator_state_machine.cpp index 564303f22..631c3c4d2 100644 --- a/src/coordination/coordinator_state_machine.cpp +++ b/src/coordination/coordinator_state_machine.cpp @@ -14,6 +14,10 @@ #include "nuraft/coordinator_state_machine.hpp" #include "utils/logging.hpp" +namespace { +constexpr int MAX_SNAPSHOTS = 3; +} // namespace + namespace memgraph::coordination { auto CoordinatorStateMachine::FindCurrentMainInstanceName() const -> std::optional<std::string> { @@ -82,6 +86,7 @@ auto CoordinatorStateMachine::DecodeLog(buffer &data) -> std::pair<TRaftLog, Raf auto CoordinatorStateMachine::pre_commit(ulong const /*log_idx*/, buffer & /*data*/) -> ptr<buffer> { return nullptr; } auto CoordinatorStateMachine::commit(ulong const log_idx, buffer &data) -> ptr<buffer> { + spdlog::debug("Commit: log_idx={}, data.size()={}", log_idx, data.size()); auto const [parsed_data, log_action] = DecodeLog(data); cluster_state_.DoAction(parsed_data, log_action); last_committed_idx_ = log_idx; @@ -95,15 +100,17 @@ auto CoordinatorStateMachine::commit(ulong const log_idx, buffer &data) -> ptr<b auto CoordinatorStateMachine::commit_config(ulong const log_idx, ptr<cluster_config> & /*new_conf*/) -> void { last_committed_idx_ = log_idx; + spdlog::debug("Commit config: log_idx={}", log_idx); } auto CoordinatorStateMachine::rollback(ulong const log_idx, buffer &data) -> void { // NOTE: Nothing since we don't do anything in pre_commit + spdlog::debug("Rollback: log_idx={}, data.size()={}", log_idx, data.size()); } auto CoordinatorStateMachine::read_logical_snp_obj(snapshot &snapshot, void *& /*user_snp_ctx*/, ulong obj_id, ptr<buffer> &data_out, bool &is_last_obj) -> int { - spdlog::info("read logical snapshot object, obj_id: {}", obj_id); + spdlog::debug("read logical snapshot object, obj_id: {}", obj_id); ptr<SnapshotCtx> ctx = nullptr; { @@ -116,20 +123,33 @@ auto CoordinatorStateMachine::read_logical_snp_obj(snapshot &snapshot, void *& / } ctx = entry->second; } - ctx->cluster_state_.Serialize(data_out); - is_last_obj = true; + + if (obj_id == 0) { + // Object ID == 0: first object, put dummy data. + data_out = buffer::alloc(sizeof(int32)); + buffer_serializer bs(data_out); + bs.put_i32(0); + is_last_obj = false; + } else { + // Object ID > 0: second object, put actual value. + ctx->cluster_state_.Serialize(data_out); + } + return 0; } auto CoordinatorStateMachine::save_logical_snp_obj(snapshot &snapshot, ulong &obj_id, buffer &data, bool is_first_obj, bool is_last_obj) -> void { - spdlog::info("save logical snapshot object, obj_id: {}, is_first_obj: {}, is_last_obj: {}", obj_id, is_first_obj, - is_last_obj); + spdlog::debug("save logical snapshot object, obj_id: {}, is_first_obj: {}, is_last_obj: {}", obj_id, is_first_obj, + is_last_obj); - buffer_serializer bs(data); - auto cluster_state = CoordinatorClusterState::Deserialize(data); + if (obj_id == 0) { + ptr<buffer> snp_buf = snapshot.serialize(); + auto ss = snapshot::deserialize(*snp_buf); + create_snapshot_internal(ss); + } else { + auto cluster_state = CoordinatorClusterState::Deserialize(data); - { auto ll = std::lock_guard{snapshots_lock_}; auto entry = snapshots_.find(snapshot.get_last_log_idx()); DMG_ASSERT(entry != snapshots_.end()); @@ -139,6 +159,7 @@ auto CoordinatorStateMachine::save_logical_snp_obj(snapshot &snapshot, ulong &ob auto CoordinatorStateMachine::apply_snapshot(snapshot &s) -> bool { auto ll = std::lock_guard{snapshots_lock_}; + spdlog::debug("apply snapshot, last_log_idx: {}", s.get_last_log_idx()); auto entry = snapshots_.find(s.get_last_log_idx()); if (entry == snapshots_.end()) return false; @@ -151,6 +172,7 @@ auto CoordinatorStateMachine::free_user_snp_ctx(void *&user_snp_ctx) -> void {} auto CoordinatorStateMachine::last_snapshot() -> ptr<snapshot> { auto ll = std::lock_guard{snapshots_lock_}; + spdlog::debug("last_snapshot"); auto entry = snapshots_.rbegin(); if (entry == snapshots_.rend()) return nullptr; @@ -161,6 +183,7 @@ auto CoordinatorStateMachine::last_snapshot() -> ptr<snapshot> { auto CoordinatorStateMachine::last_commit_index() -> ulong { return last_committed_idx_; } auto CoordinatorStateMachine::create_snapshot(snapshot &s, async_result<bool>::handler_type &when_done) -> void { + spdlog::debug("create_snapshot, last_log_idx: {}", s.get_last_log_idx()); ptr<buffer> snp_buf = s.serialize(); ptr<snapshot> ss = snapshot::deserialize(*snp_buf); create_snapshot_internal(ss); @@ -172,11 +195,11 @@ auto CoordinatorStateMachine::create_snapshot(snapshot &s, async_result<bool>::h auto CoordinatorStateMachine::create_snapshot_internal(ptr<snapshot> snapshot) -> void { auto ll = std::lock_guard{snapshots_lock_}; + spdlog::debug("create_snapshot_internal, last_log_idx: {}", snapshot->get_last_log_idx()); auto ctx = cs_new<SnapshotCtx>(snapshot, cluster_state_); snapshots_[snapshot->get_last_log_idx()] = ctx; - constexpr int MAX_SNAPSHOTS = 3; while (snapshots_.size() > MAX_SNAPSHOTS) { snapshots_.erase(snapshots_.begin()); } diff --git a/src/coordination/include/nuraft/coordinator_state_machine.hpp b/src/coordination/include/nuraft/coordinator_state_machine.hpp index 516b8efc5..836ac17a6 100644 --- a/src/coordination/include/nuraft/coordinator_state_machine.hpp +++ b/src/coordination/include/nuraft/coordinator_state_machine.hpp @@ -95,12 +95,8 @@ class CoordinatorStateMachine : public state_machine { auto create_snapshot_internal(ptr<snapshot> snapshot) -> void; CoordinatorClusterState cluster_state_; - - // mutable utils::RWLock lock{utils::RWLock::Priority::READ}; - std::atomic<uint64_t> last_committed_idx_{0}; - // TODO: (andi) Maybe not needed, remove it std::map<uint64_t, ptr<SnapshotCtx>> snapshots_; std::mutex snapshots_lock_; diff --git a/src/coordination/raft_state.cpp b/src/coordination/raft_state.cpp index dd441db74..38acfd85e 100644 --- a/src/coordination/raft_state.cpp +++ b/src/coordination/raft_state.cpp @@ -10,7 +10,6 @@ // licenses/APL.txt. #ifdef MG_ENTERPRISE - #include <chrono> #include "coordination/coordinator_config.hpp" @@ -43,19 +42,25 @@ RaftState::RaftState(BecomeLeaderCb become_leader_cb, BecomeFollowerCb become_fo auto RaftState::InitRaftServer() -> void { asio_service::options asio_opts; - asio_opts.thread_pool_size_ = 1; // TODO: (andi) Improve this + asio_opts.thread_pool_size_ = 1; raft_params params; params.heart_beat_interval_ = 100; params.election_timeout_lower_bound_ = 200; params.election_timeout_upper_bound_ = 400; - // 5 logs are preserved before the last snapshot params.reserved_log_items_ = 5; - // Create snapshot for every 5 log appends params.snapshot_distance_ = 5; params.client_req_timeout_ = 3000; params.return_method_ = raft_params::blocking; + // If the leader doesn't receive any response from quorum nodes + // in 200ms, it will step down. + // This allows us to achieve strong consistency even if network partition + // happens between the current leader and followers. + // The value must be <= election_timeout_lower_bound_ so that cluster can never + // have multiple leaders. + params.leadership_expiry_ = 200; + raft_server::init_options init_opts; init_opts.raft_callback_ = [this](cb_func::Type event_type, cb_func::Param *param) -> nuraft::CbReturnCode { if (event_type == cb_func::BecomeLeader) { @@ -117,7 +122,8 @@ auto RaftState::AddCoordinatorInstance(uint32_t raft_server_id, uint32_t raft_po if (cmd_result->get_result_code() == nuraft::cmd_result_code::OK) { spdlog::info("Request to add server {} to the cluster accepted", endpoint); } else { - throw RaftAddServerException("Failed to accept request to add server {} to the cluster", endpoint); + throw RaftAddServerException("Failed to accept request to add server {} to the cluster with error code {}", + endpoint, cmd_result->get_result_code()); } // Waiting for server to join diff --git a/src/query/interpreter.cpp b/src/query/interpreter.cpp index e51620bf6..ecec4fccb 100644 --- a/src/query/interpreter.cpp +++ b/src/query/interpreter.cpp @@ -497,7 +497,7 @@ class CoordQueryHandler final : public query::CoordinatorQueryHandler { auto const maybe_ip_and_port = io::network::Endpoint::ParseSocketOrAddress(raft_socket_address); if (maybe_ip_and_port) { auto const [ip, port] = *maybe_ip_and_port; - spdlog::info("Adding instance {} with raft socket address {}:{}.", raft_server_id, port, ip); + spdlog::info("Adding instance {} with raft socket address {}:{}.", raft_server_id, ip, port); coordinator_handler_.AddCoordinatorInstance(raft_server_id, port, ip); } else { spdlog::error("Invalid raft socket address {}.", raft_socket_address); diff --git a/tests/e2e/high_availability/coord_cluster_registration.py b/tests/e2e/high_availability/coord_cluster_registration.py index a285adcea..774c6dca1 100644 --- a/tests/e2e/high_availability/coord_cluster_registration.py +++ b/tests/e2e/high_availability/coord_cluster_registration.py @@ -430,7 +430,6 @@ def test_unregister_main(): coordinator2_cursor = connect(host="localhost", port=7691).cursor() coordinator3_cursor = connect(host="localhost", port=7692).cursor() assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 1 ON '127.0.0.1:10111'") - assert add_coordinator(coordinator3_cursor, "ADD COORDINATOR 2 ON '127.0.0.1:10112'") execute_and_fetch_all( coordinator3_cursor, "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'" diff --git a/tests/e2e/high_availability/distributed_coords.py b/tests/e2e/high_availability/distributed_coords.py index 33901f1d4..3b0964111 100644 --- a/tests/e2e/high_availability/distributed_coords.py +++ b/tests/e2e/high_availability/distributed_coords.py @@ -121,6 +121,202 @@ MEMGRAPH_INSTANCES_DESCRIPTION = { } +def get_instances_description_no_setup(): + return { + "instance_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7687", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10011", + ], + "log_file": "instance_1.log", + "data_directory": f"{TEMP_DIR}/instance_1", + "setup_queries": [], + }, + "instance_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7688", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10012", + ], + "log_file": "instance_2.log", + "data_directory": f"{TEMP_DIR}/instance_2", + "setup_queries": [], + }, + "instance_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7689", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10013", + ], + "log_file": "instance_3.log", + "data_directory": f"{TEMP_DIR}/instance_3", + "setup_queries": [], + }, + "coordinator_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7690", + "--log-level=TRACE", + "--raft-server-id=1", + "--raft-server-port=10111", + ], + "log_file": "coordinator1.log", + "setup_queries": [], + }, + "coordinator_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7691", + "--log-level=TRACE", + "--raft-server-id=2", + "--raft-server-port=10112", + ], + "log_file": "coordinator2.log", + "setup_queries": [], + }, + "coordinator_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7692", + "--log-level=TRACE", + "--raft-server-id=3", + "--raft-server-port=10113", + ], + "log_file": "coordinator3.log", + "setup_queries": [], + }, + } + + +def test_old_main_comes_back_on_new_leader_as_replica(): + # 1. Start all instances. + # 2. Kill the main instance + # 3. Kill the leader + # 4. Start the old main instance + # 5. Run SHOW INSTANCES on the new leader and check that the old main instance is registered as a replica + # 6. Start again previous leader + + safe_execute(shutil.rmtree, TEMP_DIR) + inner_instances_description = get_instances_description_no_setup() + + interactive_mg_runner.start_all(inner_instances_description) + + setup_queries = [ + "ADD COORDINATOR 1 ON '127.0.0.1:10111'", + "ADD COORDINATOR 2 ON '127.0.0.1:10112'", + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002'", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003'", + "SET INSTANCE instance_3 TO MAIN", + ] + coord_cursor_3 = connect(host="localhost", port=7692).cursor() + for query in setup_queries: + execute_and_fetch_all(coord_cursor_3, query) + + interactive_mg_runner.kill(inner_instances_description, "coordinator_3") + interactive_mg_runner.kill(inner_instances_description, "instance_3") + + coord_cursor_1 = connect(host="localhost", port=7690).cursor() + + def show_instances_coord1(): + return sorted(list(execute_and_fetch_all(coord_cursor_1, "SHOW INSTANCES;"))) + + coord_cursor_2 = connect(host="localhost", port=7691).cursor() + + def show_instances_coord2(): + return sorted(list(execute_and_fetch_all(coord_cursor_2, "SHOW INSTANCES;"))) + + leader_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ] + mg_sleep_and_assert_any_function(leader_data, [show_instances_coord1, show_instances_coord2]) + + follower_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "main"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), # TODO: (andi) Will become unknown. + ] + mg_sleep_and_assert_any_function(leader_data, [show_instances_coord1, show_instances_coord2]) + mg_sleep_and_assert_any_function(follower_data, [show_instances_coord1, show_instances_coord2]) + + interactive_mg_runner.start(inner_instances_description, "instance_3") + + leader_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "replica"), + ] + mg_sleep_and_assert_any_function(leader_data, [show_instances_coord1, show_instances_coord2]) + + new_main_cursor = connect(host="localhost", port=7687).cursor() + + def show_replicas(): + return sorted(list(execute_and_fetch_all(new_main_cursor, "SHOW REPLICAS;"))) + + replicas = [ + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + ), + ( + "instance_3", + "127.0.0.1:10003", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + ), + ] + mg_sleep_and_assert_collection(replicas, show_replicas) + + execute_and_fetch_all(new_main_cursor, "CREATE (n:Node {name: 'node'})") + + replica_2_cursor = connect(host="localhost", port=7688).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(replica_2_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(1, get_vertex_count) + + replica_3_cursor = connect(host="localhost", port=7689).cursor() + + def get_vertex_count(): + return execute_and_fetch_all(replica_3_cursor, "MATCH (n) RETURN count(n)")[0][0] + + mg_sleep_and_assert(1, get_vertex_count) + + interactive_mg_runner.start(inner_instances_description, "coordinator_3") + + def test_distributed_automatic_failover(): safe_execute(shutil.rmtree, TEMP_DIR) interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION) @@ -163,6 +359,7 @@ def test_distributed_automatic_failover(): ("instance_2", "", "127.0.0.1:10012", "up", "replica"), ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), ] + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) new_main_cursor = connect(host="localhost", port=7687).cursor() @@ -209,13 +406,26 @@ def test_distributed_automatic_failover(): mg_sleep_and_assert_collection(expected_data_on_new_main_old_alive, retrieve_data_show_replicas) -def test_distributed_automatic_failover_after_coord_dies(): +def test_distributed_automatic_failover_with_leadership_change(): safe_execute(shutil.rmtree, TEMP_DIR) - interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION) + inner_instances_description = get_instances_description_no_setup() - interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_3") + interactive_mg_runner.start_all(inner_instances_description) - interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") + setup_queries = [ + "ADD COORDINATOR 1 ON '127.0.0.1:10111'", + "ADD COORDINATOR 2 ON '127.0.0.1:10112'", + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002'", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003'", + "SET INSTANCE instance_3 TO MAIN", + ] + coord_cursor_3 = connect(host="localhost", port=7692).cursor() + for query in setup_queries: + execute_and_fetch_all(coord_cursor_3, query) + + interactive_mg_runner.kill(inner_instances_description, "coordinator_3") + interactive_mg_runner.kill(inner_instances_description, "instance_3") coord_cursor_1 = connect(host="localhost", port=7690).cursor() @@ -271,7 +481,7 @@ def test_distributed_automatic_failover_after_coord_dies(): ] mg_sleep_and_assert_collection(expected_data_on_new_main, retrieve_data_show_replicas) - interactive_mg_runner.start(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3") + interactive_mg_runner.start(inner_instances_description, "instance_3") expected_data_on_new_main_old_alive = [ ( "instance_2", @@ -291,6 +501,125 @@ def test_distributed_automatic_failover_after_coord_dies(): mg_sleep_and_assert_collection(expected_data_on_new_main_old_alive, retrieve_data_show_replicas) + interactive_mg_runner.start(inner_instances_description, "coordinator_3") + + +def test_no_leader_after_leader_and_follower_die(): + # 1. Register all but one replication instnce on the first leader. + # 2. Kill the leader and a follower. + # 3. Check that the remaining follower is not promoted to leader by trying to register remaining replication instance. + + safe_execute(shutil.rmtree, TEMP_DIR) + + interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION) + + interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_3") + interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "coordinator_2") + + coord_cursor_1 = connect(host="localhost", port=7690).cursor() + + with pytest.raises(Exception) as e: + execute_and_fetch_all(coord_cursor_1, "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.10001'") + assert str(e) == "Couldn't register replica instance since coordinator is not a leader!" + + +def test_old_main_comes_back_on_new_leader_as_main(): + # 1. Start all instances. + # 2. Kill all instances + # 3. Kill the leader + # 4. Start the old main instance + # 5. Run SHOW INSTANCES on the new leader and check that the old main instance is main once again + + inner_memgraph_instances = get_instances_description_no_setup() + interactive_mg_runner.start_all(inner_memgraph_instances) + + coord_cursor_3 = connect(host="localhost", port=7692).cursor() + + setup_queries = [ + "ADD COORDINATOR 1 ON '127.0.0.1:10111'", + "ADD COORDINATOR 2 ON '127.0.0.1:10112'", + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002'", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003'", + "SET INSTANCE instance_3 TO MAIN", + ] + + for query in setup_queries: + execute_and_fetch_all(coord_cursor_3, query) + + interactive_mg_runner.kill(inner_memgraph_instances, "instance_1") + interactive_mg_runner.kill(inner_memgraph_instances, "instance_2") + interactive_mg_runner.kill(inner_memgraph_instances, "instance_3") + interactive_mg_runner.kill(inner_memgraph_instances, "coordinator_3") + + coord_cursor_1 = connect(host="localhost", port=7690).cursor() + + def show_instances_coord1(): + return sorted(list(execute_and_fetch_all(coord_cursor_1, "SHOW INSTANCES;"))) + + coord_cursor_2 = connect(host="localhost", port=7691).cursor() + + def show_instances_coord2(): + return sorted(list(execute_and_fetch_all(coord_cursor_2, "SHOW INSTANCES;"))) + + interactive_mg_runner.start(inner_memgraph_instances, "instance_3") + + leader_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + + follower_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), + ] + mg_sleep_and_assert_any_function(leader_data, [show_instances_coord1, show_instances_coord2]) + mg_sleep_and_assert_any_function(follower_data, [show_instances_coord1, show_instances_coord2]) + + interactive_mg_runner.start(inner_memgraph_instances, "instance_1") + interactive_mg_runner.start(inner_memgraph_instances, "instance_2") + + new_main_cursor = connect(host="localhost", port=7689).cursor() + + def show_replicas(): + return sorted(list(execute_and_fetch_all(new_main_cursor, "SHOW REPLICAS;"))) + + replicas = [ + ( + "instance_1", + "127.0.0.1:10001", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + ), + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 0, "behind": 0, "status": "ready"}}, + ), + ] + mg_sleep_and_assert_collection(replicas, show_replicas) + + execute_and_fetch_all(new_main_cursor, "CREATE (n:Node {name: 'node'})") + + replica_1_cursor = connect(host="localhost", port=7687).cursor() + assert len(execute_and_fetch_all(replica_1_cursor, "MATCH (n) RETURN n;")) == 1 + + replica_2_cursor = connect(host="localhost", port=7688).cursor() + assert len(execute_and_fetch_all(replica_2_cursor, "MATCH (n) RETURN n;")) == 1 + + interactive_mg_runner.start(inner_memgraph_instances, "coordinator_3") + def test_registering_4_coords(): # Goal of this test is to assure registering of multiple coordinators in row works From 619b01f3f81072ef6621bbae6e1192ca76b91fea Mon Sep 17 00:00:00 2001 From: gvolfing <107616712+gvolfing@users.noreply.github.com> Date: Fri, 8 Mar 2024 08:44:48 +0100 Subject: [PATCH 16/18] Implement edge type indices (#1542) Implement edge type indices (#1542 ) --- src/communication/result_stream_faker.hpp | 2 +- src/dbms/database_handler.hpp | 2 +- src/dbms/inmemory/replication_handlers.cpp | 14 + src/glue/communication.hpp | 2 +- src/query/db_accessor.hpp | 70 +++ src/query/dump.cpp | 35 +- src/query/dump.hpp | 1 + src/query/frontend/ast/ast.cpp | 3 + src/query/frontend/ast/ast.hpp | 28 + src/query/frontend/ast/ast_visitor.hpp | 13 +- .../frontend/ast/cypher_main_visitor.cpp | 21 + .../frontend/ast/cypher_main_visitor.hpp | 15 + .../opencypher/grammar/MemgraphCypher.g4 | 7 + .../frontend/semantic/required_privileges.cpp | 2 + src/query/frontend/semantic/symbol.hpp | 2 + src/query/interpreter.cpp | 82 ++- src/query/plan/hint_provider.hpp | 3 + src/query/plan/operator.cpp | 82 +++ src/query/plan/operator.hpp | 45 +- src/query/plan/operator_type_info.cpp | 2 + src/query/plan/planner.hpp | 8 +- src/query/plan/pretty_print.cpp | 20 + src/query/plan/pretty_print.hpp | 2 + .../plan/rewrite/edge_type_index_lookup.hpp | 534 ++++++++++++++++++ src/query/plan/vertex_count_cache.hpp | 2 + src/query/procedure/module.hpp | 2 +- src/query/procedure/py_module.hpp | 2 +- src/storage/v2/CMakeLists.txt | 3 + src/storage/v2/disk/edge_type_index.cpp | 49 ++ src/storage/v2/disk/edge_type_index.hpp | 35 ++ src/storage/v2/disk/storage.cpp | 33 ++ src/storage/v2/disk/storage.hpp | 10 + src/storage/v2/durability/durability.cpp | 14 +- src/storage/v2/durability/marker.hpp | 9 +- src/storage/v2/durability/metadata.hpp | 3 +- src/storage/v2/durability/serialization.cpp | 8 +- src/storage/v2/durability/snapshot.cpp | 410 +++++++++++++- src/storage/v2/durability/snapshot.hpp | 3 +- .../durability/storage_global_operation.hpp | 4 +- src/storage/v2/durability/version.hpp | 4 +- src/storage/v2/durability/wal.cpp | 72 +++ src/storage/v2/durability/wal.hpp | 17 +- src/storage/v2/edges_iterable.cpp | 149 +++++ src/storage/v2/edges_iterable.hpp | 73 +++ src/storage/v2/indices/edge_type_index.hpp | 46 ++ src/storage/v2/indices/indices.cpp | 11 + src/storage/v2/indices/indices.hpp | 5 + src/storage/v2/inmemory/edge_type_index.cpp | 318 +++++++++++ src/storage/v2/inmemory/edge_type_index.hpp | 113 ++++ src/storage/v2/inmemory/storage.cpp | 60 +- src/storage/v2/inmemory/storage.hpp | 34 +- src/storage/v2/metadata_delta.hpp | 17 +- .../v2/replication/replication_client.cpp | 6 + .../v2/replication/replication_client.hpp | 3 + .../replication/replication_storage_state.cpp | 10 + .../replication/replication_storage_state.hpp | 2 + src/storage/v2/storage.hpp | 12 + src/storage/v2/vertices_iterable.cpp | 3 +- src/storage/v2/vertices_iterable.hpp | 2 +- src/utils/atomic_memory_block.hpp | 2 +- src/utils/event_counter.cpp | 1 + src/utils/settings.cpp | 2 +- src/utils/typeinfo.hpp | 2 + .../tests/v17/test_all/create_dataset.cypher | 22 + .../v17/test_all/expected_snapshot.cypher | 19 + .../tests/v17/test_all/expected_wal.cypher | 19 + .../tests/v17/test_all/snapshot.bin | Bin 0 -> 2067 bytes .../durability/tests/v17/test_all/wal.bin | Bin 0 -> 3582 bytes .../test_constraints/create_dataset.cypher | 6 + .../test_constraints/expected_snapshot.cypher | 6 + .../v17/test_constraints/expected_wal.cypher | 6 + .../tests/v17/test_constraints/snapshot.bin | Bin 0 -> 625 bytes .../tests/v17/test_constraints/wal.bin | Bin 0 -> 460 bytes .../v17/test_edges/create_dataset.cypher | 60 ++ .../v17/test_edges/expected_snapshot.cypher | 58 ++ .../tests/v17/test_edges/expected_wal.cypher | 58 ++ .../tests/v17/test_edges/snapshot.bin | Bin 0 -> 4297 bytes .../durability/tests/v17/test_edges/wal.bin | Bin 0 -> 6616 bytes .../v17/test_indices/create_dataset.cypher | 6 + .../v17/test_indices/expected_snapshot.cypher | 5 + .../v17/test_indices/expected_wal.cypher | 5 + .../tests/v17/test_indices/snapshot.bin | Bin 0 -> 731 bytes .../durability/tests/v17/test_indices/wal.bin | Bin 0 -> 847 bytes .../v17/test_vertices/create_dataset.cypher | 18 + .../test_vertices/expected_snapshot.cypher | 16 + .../v17/test_vertices/expected_wal.cypher | 16 + .../tests/v17/test_vertices/snapshot.bin | Bin 0 -> 1739 bytes .../tests/v17/test_vertices/wal.bin | Bin 0 -> 4355 bytes tests/manual/interactive_planning.cpp | 2 + tests/unit/dbms_database.cpp | 2 +- tests/unit/query_plan.cpp | 59 +- tests/unit/query_plan_checker.hpp | 19 +- tests/unit/storage_v2_decoder_encoder.cpp | 5 +- tests/unit/storage_v2_durability_inmemory.cpp | 75 ++- tests/unit/storage_v2_indices.cpp | 385 ++++++++++++- tests/unit/storage_v2_wal_file.cpp | 39 ++ 96 files changed, 3390 insertions(+), 62 deletions(-) create mode 100644 src/query/plan/rewrite/edge_type_index_lookup.hpp create mode 100644 src/storage/v2/disk/edge_type_index.cpp create mode 100644 src/storage/v2/disk/edge_type_index.hpp create mode 100644 src/storage/v2/edges_iterable.cpp create mode 100644 src/storage/v2/edges_iterable.hpp create mode 100644 src/storage/v2/indices/edge_type_index.hpp create mode 100644 src/storage/v2/inmemory/edge_type_index.cpp create mode 100644 src/storage/v2/inmemory/edge_type_index.hpp create mode 100644 tests/integration/durability/tests/v17/test_all/create_dataset.cypher create mode 100644 tests/integration/durability/tests/v17/test_all/expected_snapshot.cypher create mode 100644 tests/integration/durability/tests/v17/test_all/expected_wal.cypher create mode 100644 tests/integration/durability/tests/v17/test_all/snapshot.bin create mode 100644 tests/integration/durability/tests/v17/test_all/wal.bin create mode 100644 tests/integration/durability/tests/v17/test_constraints/create_dataset.cypher create mode 100644 tests/integration/durability/tests/v17/test_constraints/expected_snapshot.cypher create mode 100644 tests/integration/durability/tests/v17/test_constraints/expected_wal.cypher create mode 100644 tests/integration/durability/tests/v17/test_constraints/snapshot.bin create mode 100644 tests/integration/durability/tests/v17/test_constraints/wal.bin create mode 100644 tests/integration/durability/tests/v17/test_edges/create_dataset.cypher create mode 100644 tests/integration/durability/tests/v17/test_edges/expected_snapshot.cypher create mode 100644 tests/integration/durability/tests/v17/test_edges/expected_wal.cypher create mode 100644 tests/integration/durability/tests/v17/test_edges/snapshot.bin create mode 100644 tests/integration/durability/tests/v17/test_edges/wal.bin create mode 100644 tests/integration/durability/tests/v17/test_indices/create_dataset.cypher create mode 100644 tests/integration/durability/tests/v17/test_indices/expected_snapshot.cypher create mode 100644 tests/integration/durability/tests/v17/test_indices/expected_wal.cypher create mode 100644 tests/integration/durability/tests/v17/test_indices/snapshot.bin create mode 100644 tests/integration/durability/tests/v17/test_indices/wal.bin create mode 100644 tests/integration/durability/tests/v17/test_vertices/create_dataset.cypher create mode 100644 tests/integration/durability/tests/v17/test_vertices/expected_snapshot.cypher create mode 100644 tests/integration/durability/tests/v17/test_vertices/expected_wal.cypher create mode 100644 tests/integration/durability/tests/v17/test_vertices/snapshot.bin create mode 100644 tests/integration/durability/tests/v17/test_vertices/wal.bin diff --git a/src/communication/result_stream_faker.hpp b/src/communication/result_stream_faker.hpp index 779d039cc..c0a40cecf 100644 --- a/src/communication/result_stream_faker.hpp +++ b/src/communication/result_stream_faker.hpp @@ -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 diff --git a/src/dbms/database_handler.hpp b/src/dbms/database_handler.hpp index de5f813ba..cae54088e 100644 --- a/src/dbms/database_handler.hpp +++ b/src/dbms/database_handler.hpp @@ -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 diff --git a/src/dbms/inmemory/replication_handlers.cpp b/src/dbms/inmemory/replication_handlers.cpp index 6a78977bb..3e4a31884 100644 --- a/src/dbms/inmemory/replication_handlers.cpp +++ b/src/dbms/inmemory/replication_handlers.cpp @@ -840,6 +840,20 @@ uint64_t InMemoryReplicationHandlers::ReadAndApplyDelta(storage::InMemoryStorage transaction->DeleteLabelPropertyIndexStats(storage->NameToLabel(info.label)); break; } + case WalDeltaData::Type::EDGE_INDEX_CREATE: { + spdlog::trace(" Create edge index on :{}", delta.operation_edge_type.edge_type); + auto *transaction = get_transaction(timestamp, kUniqueAccess); + if (transaction->CreateIndex(storage->NameToEdgeType(delta.operation_label.label)).HasError()) + throw utils::BasicException("Invalid transaction! Please raise an issue, {}:{}", __FILE__, __LINE__); + break; + } + case WalDeltaData::Type::EDGE_INDEX_DROP: { + spdlog::trace(" Drop edge index on :{}", delta.operation_edge_type.edge_type); + auto *transaction = get_transaction(timestamp, kUniqueAccess); + if (transaction->DropIndex(storage->NameToEdgeType(delta.operation_label.label)).HasError()) + throw utils::BasicException("Invalid transaction! Please raise an issue, {}:{}", __FILE__, __LINE__); + break; + } case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: { spdlog::trace(" Create existence constraint on :{} ({})", delta.operation_label_property.label, delta.operation_label_property.property); diff --git a/src/glue/communication.hpp b/src/glue/communication.hpp index 737f32db2..a448b05fc 100644 --- a/src/glue/communication.hpp +++ b/src/glue/communication.hpp @@ -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 diff --git a/src/query/db_accessor.hpp b/src/query/db_accessor.hpp index e10102ee5..915ea9936 100644 --- a/src/query/db_accessor.hpp +++ b/src/query/db_accessor.hpp @@ -371,6 +371,62 @@ class VerticesIterable final { } }; +class EdgesIterable final { + std::variant<storage::EdgesIterable, std::unordered_set<EdgeAccessor, std::hash<EdgeAccessor>, std::equal_to<void>, + utils::Allocator<EdgeAccessor>> *> + iterable_; + + public: + class Iterator final { + std::variant<storage::EdgesIterable::Iterator, + std::unordered_set<EdgeAccessor, std::hash<EdgeAccessor>, std::equal_to<void>, + utils::Allocator<EdgeAccessor>>::iterator> + it_; + + public: + explicit Iterator(storage::EdgesIterable::Iterator it) : it_(std::move(it)) {} + explicit Iterator(std::unordered_set<EdgeAccessor, std::hash<EdgeAccessor>, std::equal_to<void>, + utils::Allocator<EdgeAccessor>>::iterator it) + : it_(it) {} + + EdgeAccessor operator*() const { + return std::visit([](auto &it_) { return EdgeAccessor(*it_); }, it_); + } + + Iterator &operator++() { + std::visit([](auto &it_) { ++it_; }, it_); + return *this; + } + + bool operator==(const Iterator &other) const { return it_ == other.it_; } + + bool operator!=(const Iterator &other) const { return !(other == *this); } + }; + + explicit EdgesIterable(storage::EdgesIterable iterable) : iterable_(std::move(iterable)) {} + explicit EdgesIterable(std::unordered_set<EdgeAccessor, std::hash<EdgeAccessor>, std::equal_to<void>, + utils::Allocator<EdgeAccessor>> *edges) + : iterable_(edges) {} + + Iterator begin() { + return std::visit( + memgraph::utils::Overloaded{ + [](storage::EdgesIterable &iterable_) { return Iterator(iterable_.begin()); }, + [](std::unordered_set<EdgeAccessor, std::hash<EdgeAccessor>, std::equal_to<void>, + utils::Allocator<EdgeAccessor>> *iterable_) { return Iterator(iterable_->begin()); }}, + iterable_); + } + + Iterator end() { + return std::visit( + memgraph::utils::Overloaded{ + [](storage::EdgesIterable &iterable_) { return Iterator(iterable_.end()); }, + [](std::unordered_set<EdgeAccessor, std::hash<EdgeAccessor>, std::equal_to<void>, + utils::Allocator<EdgeAccessor>> *iterable_) { return Iterator(iterable_->end()); }}, + iterable_); + } +}; + class DbAccessor final { storage::Storage::Accessor *accessor_; @@ -416,6 +472,10 @@ class DbAccessor final { return VerticesIterable(accessor_->Vertices(label, property, lower, upper, view)); } + EdgesIterable Edges(storage::View view, storage::EdgeTypeId edge_type) { + return EdgesIterable(accessor_->Edges(edge_type, view)); + } + VertexAccessor InsertVertex() { return VertexAccessor(accessor_->CreateVertex()); } storage::Result<EdgeAccessor> InsertEdge(VertexAccessor *from, VertexAccessor *to, @@ -572,6 +632,8 @@ class DbAccessor final { return accessor_->LabelPropertyIndexExists(label, prop); } + bool EdgeTypeIndexExists(storage::EdgeTypeId edge_type) const { return accessor_->EdgeTypeIndexExists(edge_type); } + std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const { return accessor_->GetIndexStats(label); } @@ -638,6 +700,10 @@ class DbAccessor final { return accessor_->CreateIndex(label, property); } + utils::BasicResult<storage::StorageIndexDefinitionError, void> CreateIndex(storage::EdgeTypeId edge_type) { + return accessor_->CreateIndex(edge_type); + } + utils::BasicResult<storage::StorageIndexDefinitionError, void> DropIndex(storage::LabelId label) { return accessor_->DropIndex(label); } @@ -647,6 +713,10 @@ class DbAccessor final { return accessor_->DropIndex(label, property); } + utils::BasicResult<storage::StorageIndexDefinitionError, void> DropIndex(storage::EdgeTypeId edge_type) { + return accessor_->DropIndex(edge_type); + } + utils::BasicResult<storage::StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint( storage::LabelId label, storage::PropertyId property) { return accessor_->CreateExistenceConstraint(label, property); diff --git a/src/query/dump.cpp b/src/query/dump.cpp index 2925023fb..f1dd08c8d 100644 --- a/src/query/dump.cpp +++ b/src/query/dump.cpp @@ -242,6 +242,10 @@ void DumpLabelIndex(std::ostream *os, query::DbAccessor *dba, const storage::Lab *os << "CREATE INDEX ON :" << EscapeName(dba->LabelToName(label)) << ";"; } +void DumpEdgeTypeIndex(std::ostream *os, query::DbAccessor *dba, const storage::EdgeTypeId edge_type) { + *os << "CREATE EDGE INDEX ON :" << EscapeName(dba->EdgeTypeToName(edge_type)) << ";"; +} + void DumpLabelPropertyIndex(std::ostream *os, query::DbAccessor *dba, storage::LabelId label, storage::PropertyId property) { *os << "CREATE INDEX ON :" << EscapeName(dba->LabelToName(label)) << "(" << EscapeName(dba->PropertyToName(property)) @@ -297,7 +301,9 @@ PullPlanDump::PullPlanDump(DbAccessor *dba, dbms::DatabaseAccess db_acc) // Internal index cleanup CreateInternalIndexCleanupPullChunk(), // Dump all triggers - CreateTriggersPullChunk()} {} + CreateTriggersPullChunk(), + // Dump all edge-type indices + CreateEdgeTypeIndicesPullChunk()} {} bool PullPlanDump::Pull(AnyStream *stream, std::optional<int> n) { // Iterate all functions that stream some results. @@ -352,6 +358,33 @@ PullPlanDump::PullChunk PullPlanDump::CreateLabelIndicesPullChunk() { }; } +PullPlanDump::PullChunk PullPlanDump::CreateEdgeTypeIndicesPullChunk() { + // Dump all label indices + return [this, global_index = 0U](AnyStream *stream, std::optional<int> n) mutable -> std::optional<size_t> { + // Delay the construction of indices vectors + if (!indices_info_) { + indices_info_.emplace(dba_->ListAllIndices()); + } + const auto &edge_type = indices_info_->edge_type; + + size_t local_counter = 0; + while (global_index < edge_type.size() && (!n || local_counter < *n)) { + std::ostringstream os; + DumpEdgeTypeIndex(&os, dba_, edge_type[global_index]); + stream->Result({TypedValue(os.str())}); + + ++global_index; + ++local_counter; + } + + if (global_index == edge_type.size()) { + return local_counter; + } + + return std::nullopt; + }; +} + PullPlanDump::PullChunk PullPlanDump::CreateLabelPropertyIndicesPullChunk() { return [this, global_index = 0U](AnyStream *stream, std::optional<int> n) mutable -> std::optional<size_t> { // Delay the construction of indices vectors diff --git a/src/query/dump.hpp b/src/query/dump.hpp index a9d68d45c..05bd42967 100644 --- a/src/query/dump.hpp +++ b/src/query/dump.hpp @@ -63,5 +63,6 @@ struct PullPlanDump { PullChunk CreateDropInternalIndexPullChunk(); PullChunk CreateInternalIndexCleanupPullChunk(); PullChunk CreateTriggersPullChunk(); + PullChunk CreateEdgeTypeIndicesPullChunk(); }; } // namespace memgraph::query diff --git a/src/query/frontend/ast/ast.cpp b/src/query/frontend/ast/ast.cpp index 57d5398ab..7da5c09a0 100644 --- a/src/query/frontend/ast/ast.cpp +++ b/src/query/frontend/ast/ast.cpp @@ -186,6 +186,9 @@ constexpr utils::TypeInfo query::ProfileQuery::kType{utils::TypeId::AST_PROFILE_ constexpr utils::TypeInfo query::IndexQuery::kType{utils::TypeId::AST_INDEX_QUERY, "IndexQuery", &query::Query::kType}; +constexpr utils::TypeInfo query::EdgeIndexQuery::kType{utils::TypeId::AST_EDGE_INDEX_QUERY, "EdgeIndexQuery", + &query::Query::kType}; + constexpr utils::TypeInfo query::Create::kType{utils::TypeId::AST_CREATE, "Create", &query::Clause::kType}; constexpr utils::TypeInfo query::CallProcedure::kType{utils::TypeId::AST_CALL_PROCEDURE, "CallProcedure", diff --git a/src/query/frontend/ast/ast.hpp b/src/query/frontend/ast/ast.hpp index ed354f6ca..b8d8c9e1a 100644 --- a/src/query/frontend/ast/ast.hpp +++ b/src/query/frontend/ast/ast.hpp @@ -2224,6 +2224,34 @@ class IndexQuery : public memgraph::query::Query { friend class AstStorage; }; +class EdgeIndexQuery : public memgraph::query::Query { + public: + static const utils::TypeInfo kType; + const utils::TypeInfo &GetTypeInfo() const override { return kType; } + + enum class Action { CREATE, DROP }; + + EdgeIndexQuery() = default; + + DEFVISITABLE(QueryVisitor<void>); + + memgraph::query::EdgeIndexQuery::Action action_; + memgraph::query::EdgeTypeIx edge_type_; + + EdgeIndexQuery *Clone(AstStorage *storage) const override { + EdgeIndexQuery *object = storage->Create<EdgeIndexQuery>(); + object->action_ = action_; + object->edge_type_ = storage->GetEdgeTypeIx(edge_type_.name); + return object; + } + + protected: + EdgeIndexQuery(Action action, EdgeTypeIx edge_type) : action_(action), edge_type_(edge_type) {} + + private: + friend class AstStorage; +}; + class Create : public memgraph::query::Clause { public: static const utils::TypeInfo kType; diff --git a/src/query/frontend/ast/ast_visitor.hpp b/src/query/frontend/ast/ast_visitor.hpp index 5d463d3ee..bf11878da 100644 --- a/src/query/frontend/ast/ast_visitor.hpp +++ b/src/query/frontend/ast/ast_visitor.hpp @@ -82,6 +82,7 @@ class AuthQuery; class ExplainQuery; class ProfileQuery; class IndexQuery; +class EdgeIndexQuery; class DatabaseInfoQuery; class SystemInfoQuery; class ConstraintQuery; @@ -143,11 +144,11 @@ class ExpressionVisitor template <class TResult> class QueryVisitor - : public utils::Visitor<TResult, CypherQuery, ExplainQuery, ProfileQuery, IndexQuery, AuthQuery, DatabaseInfoQuery, - SystemInfoQuery, ConstraintQuery, DumpQuery, ReplicationQuery, LockPathQuery, - FreeMemoryQuery, TriggerQuery, IsolationLevelQuery, CreateSnapshotQuery, StreamQuery, - SettingQuery, VersionQuery, ShowConfigQuery, TransactionQueueQuery, StorageModeQuery, - AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery, EdgeImportModeQuery, - CoordinatorQuery> {}; + : public utils::Visitor<TResult, CypherQuery, ExplainQuery, ProfileQuery, IndexQuery, EdgeIndexQuery, AuthQuery, + DatabaseInfoQuery, SystemInfoQuery, ConstraintQuery, DumpQuery, ReplicationQuery, + LockPathQuery, FreeMemoryQuery, TriggerQuery, IsolationLevelQuery, CreateSnapshotQuery, + StreamQuery, SettingQuery, VersionQuery, ShowConfigQuery, TransactionQueueQuery, + StorageModeQuery, AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery, + EdgeImportModeQuery, CoordinatorQuery> {}; } // namespace memgraph::query diff --git a/src/query/frontend/ast/cypher_main_visitor.cpp b/src/query/frontend/ast/cypher_main_visitor.cpp index d3747bc3f..467c73125 100644 --- a/src/query/frontend/ast/cypher_main_visitor.cpp +++ b/src/query/frontend/ast/cypher_main_visitor.cpp @@ -265,6 +265,27 @@ antlrcpp::Any CypherMainVisitor::visitDropIndex(MemgraphCypher::DropIndexContext return index_query; } +antlrcpp::Any CypherMainVisitor::visitEdgeIndexQuery(MemgraphCypher::EdgeIndexQueryContext *ctx) { + MG_ASSERT(ctx->children.size() == 1, "EdgeIndexQuery should have exactly one child!"); + auto *index_query = std::any_cast<EdgeIndexQuery *>(ctx->children[0]->accept(this)); + query_ = index_query; + return index_query; +} + +antlrcpp::Any CypherMainVisitor::visitCreateEdgeIndex(MemgraphCypher::CreateEdgeIndexContext *ctx) { + auto *index_query = storage_->Create<EdgeIndexQuery>(); + index_query->action_ = EdgeIndexQuery::Action::CREATE; + index_query->edge_type_ = AddEdgeType(std::any_cast<std::string>(ctx->labelName()->accept(this))); + return index_query; +} + +antlrcpp::Any CypherMainVisitor::visitDropEdgeIndex(MemgraphCypher::DropEdgeIndexContext *ctx) { + auto *index_query = storage_->Create<EdgeIndexQuery>(); + index_query->action_ = EdgeIndexQuery::Action::DROP; + index_query->edge_type_ = AddEdgeType(std::any_cast<std::string>(ctx->labelName()->accept(this))); + return index_query; +} + antlrcpp::Any CypherMainVisitor::visitAuthQuery(MemgraphCypher::AuthQueryContext *ctx) { MG_ASSERT(ctx->children.size() == 1, "AuthQuery should have exactly one child!"); auto *auth_query = std::any_cast<AuthQuery *>(ctx->children[0]->accept(this)); diff --git a/src/query/frontend/ast/cypher_main_visitor.hpp b/src/query/frontend/ast/cypher_main_visitor.hpp index 6d66e6d7e..8c65345c8 100644 --- a/src/query/frontend/ast/cypher_main_visitor.hpp +++ b/src/query/frontend/ast/cypher_main_visitor.hpp @@ -148,6 +148,11 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor { */ antlrcpp::Any visitIndexQuery(MemgraphCypher::IndexQueryContext *ctx) override; + /** + * @return IndexQuery* + */ + antlrcpp::Any visitEdgeIndexQuery(MemgraphCypher::EdgeIndexQueryContext *ctx) override; + /** * @return ExplainQuery* */ @@ -499,6 +504,16 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor { */ antlrcpp::Any visitDropIndex(MemgraphCypher::DropIndexContext *ctx) override; + /** + * @return EdgeIndexQuery* + */ + antlrcpp::Any visitCreateEdgeIndex(MemgraphCypher::CreateEdgeIndexContext *ctx) override; + + /** + * @return DropEdgeIndex* + */ + antlrcpp::Any visitDropEdgeIndex(MemgraphCypher::DropEdgeIndexContext *ctx) override; + /** * @return AuthQuery* */ diff --git a/src/query/frontend/opencypher/grammar/MemgraphCypher.g4 b/src/query/frontend/opencypher/grammar/MemgraphCypher.g4 index d24480b0a..0147bba04 100644 --- a/src/query/frontend/opencypher/grammar/MemgraphCypher.g4 +++ b/src/query/frontend/opencypher/grammar/MemgraphCypher.g4 @@ -133,6 +133,7 @@ symbolicName : UnescapedSymbolicName query : cypherQuery | indexQuery + | edgeIndexQuery | explainQuery | profileQuery | databaseInfoQuery @@ -527,3 +528,9 @@ showDatabase : SHOW DATABASE ; showDatabases : SHOW DATABASES ; edgeImportModeQuery : EDGE IMPORT MODE ( ACTIVE | INACTIVE ) ; + +createEdgeIndex : CREATE EDGE INDEX ON ':' labelName ; + +dropEdgeIndex : DROP EDGE INDEX ON ':' labelName ; + +edgeIndexQuery : createEdgeIndex | dropEdgeIndex ; diff --git a/src/query/frontend/semantic/required_privileges.cpp b/src/query/frontend/semantic/required_privileges.cpp index ef66a75ac..15726e3e2 100644 --- a/src/query/frontend/semantic/required_privileges.cpp +++ b/src/query/frontend/semantic/required_privileges.cpp @@ -27,6 +27,8 @@ class PrivilegeExtractor : public QueryVisitor<void>, public HierarchicalTreeVis void Visit(IndexQuery & /*unused*/) override { AddPrivilege(AuthQuery::Privilege::INDEX); } + void Visit(EdgeIndexQuery & /*unused*/) override { AddPrivilege(AuthQuery::Privilege::INDEX); } + void Visit(AnalyzeGraphQuery & /*unused*/) override { AddPrivilege(AuthQuery::Privilege::INDEX); } void Visit(AuthQuery & /*unused*/) override { AddPrivilege(AuthQuery::Privilege::AUTH); } diff --git a/src/query/frontend/semantic/symbol.hpp b/src/query/frontend/semantic/symbol.hpp index 77557b6fe..0cfb86608 100644 --- a/src/query/frontend/semantic/symbol.hpp +++ b/src/query/frontend/semantic/symbol.hpp @@ -53,6 +53,8 @@ class Symbol { bool user_declared() const { return user_declared_; } int token_position() const { return token_position_; } + bool IsSymbolAnonym() const { return name_.substr(0U, 4U) == "anon"; } + std::string name_; int64_t position_; bool user_declared_{true}; diff --git a/src/query/interpreter.cpp b/src/query/interpreter.cpp index ecec4fccb..ce74586d3 100644 --- a/src/query/interpreter.cpp +++ b/src/query/interpreter.cpp @@ -2679,6 +2679,75 @@ PreparedQuery PrepareIndexQuery(ParsedQuery parsed_query, bool in_explicit_trans RWType::W}; } +PreparedQuery PrepareEdgeIndexQuery(ParsedQuery parsed_query, bool in_explicit_transaction, + std::vector<Notification> *notifications, CurrentDB ¤t_db) { + if (in_explicit_transaction) { + throw IndexInMulticommandTxException(); + } + + auto *index_query = utils::Downcast<EdgeIndexQuery>(parsed_query.query); + std::function<void(Notification &)> handler; + + MG_ASSERT(current_db.db_acc_, "Index query expects a current DB"); + auto &db_acc = *current_db.db_acc_; + + MG_ASSERT(current_db.db_transactional_accessor_, "Index query expects a current DB transaction"); + auto *dba = &*current_db.execution_db_accessor_; + + auto invalidate_plan_cache = [plan_cache = db_acc->plan_cache()] { + plan_cache->WithLock([&](auto &cache) { cache.reset(); }); + }; + + auto *storage = db_acc->storage(); + auto edge_type = storage->NameToEdgeType(index_query->edge_type_.name); + + Notification index_notification(SeverityLevel::INFO); + switch (index_query->action_) { + case EdgeIndexQuery::Action::CREATE: { + index_notification.code = NotificationCode::CREATE_INDEX; + index_notification.title = fmt::format("Created index on edge-type {}.", index_query->edge_type_.name); + + handler = [dba, edge_type, label_name = index_query->edge_type_.name, + invalidate_plan_cache = std::move(invalidate_plan_cache)](Notification &index_notification) { + auto maybe_index_error = dba->CreateIndex(edge_type); + utils::OnScopeExit invalidator(invalidate_plan_cache); + + if (maybe_index_error.HasError()) { + index_notification.code = NotificationCode::EXISTENT_INDEX; + index_notification.title = fmt::format("Index on edge-type {} already exists.", label_name); + } + }; + break; + } + case EdgeIndexQuery::Action::DROP: { + index_notification.code = NotificationCode::DROP_INDEX; + index_notification.title = fmt::format("Dropped index on edge-type {}.", index_query->edge_type_.name); + handler = [dba, edge_type, label_name = index_query->edge_type_.name, + invalidate_plan_cache = std::move(invalidate_plan_cache)](Notification &index_notification) { + auto maybe_index_error = dba->DropIndex(edge_type); + utils::OnScopeExit invalidator(invalidate_plan_cache); + + if (maybe_index_error.HasError()) { + index_notification.code = NotificationCode::NONEXISTENT_INDEX; + index_notification.title = fmt::format("Index on edge-type {} doesn't exist.", label_name); + } + }; + break; + } + } + + return PreparedQuery{ + {}, + std::move(parsed_query.required_privileges), + [handler = std::move(handler), notifications, index_notification = std::move(index_notification)]( + AnyStream * /*stream*/, std::optional<int> /*unused*/) mutable { + handler(index_notification); + notifications->push_back(index_notification); + return QueryHandlerResult::COMMIT; + }, + RWType::W}; +} + PreparedQuery PrepareAuthQuery(ParsedQuery parsed_query, bool in_explicit_transaction, InterpreterContext *interpreter_context, Interpreter &interpreter) { if (in_explicit_transaction) { @@ -3483,6 +3552,7 @@ PreparedQuery PrepareDatabaseInfoQuery(ParsedQuery parsed_query, bool in_explici auto *storage = database->storage(); const std::string_view label_index_mark{"label"}; const std::string_view label_property_index_mark{"label+property"}; + const std::string_view edge_type_index_mark{"edge-type"}; auto info = dba->ListAllIndices(); auto storage_acc = database->Access(); std::vector<std::vector<TypedValue>> results; @@ -3497,6 +3567,10 @@ PreparedQuery PrepareDatabaseInfoQuery(ParsedQuery parsed_query, bool in_explici TypedValue(storage->PropertyToName(item.second)), TypedValue(static_cast<int>(storage_acc->ApproximateVertexCount(item.first, item.second)))}); } + for (const auto &item : info.edge_type) { + results.push_back({TypedValue(edge_type_index_mark), TypedValue(storage->EdgeTypeToName(item)), TypedValue(), + TypedValue(static_cast<int>(storage_acc->ApproximateEdgeCount(item)))}); + } std::sort(results.begin(), results.end(), [&label_index_mark](const auto &record_1, const auto &record_2) { const auto type_1 = record_1[0].ValueString(); const auto type_2 = record_2[0].ValueString(); @@ -4283,13 +4357,14 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string, utils::Downcast<CypherQuery>(parsed_query.query) || utils::Downcast<ExplainQuery>(parsed_query.query) || 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<DatabaseInfoQuery>(parsed_query.query) || - utils::Downcast<ConstraintQuery>(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); if (!in_explicit_transaction_ && requires_db_transaction) { // TODO: ATM only a single database, will change when we have multiple database transactions bool could_commit = utils::Downcast<CypherQuery>(parsed_query.query) != nullptr; bool unique = utils::Downcast<IndexQuery>(parsed_query.query) != nullptr || + utils::Downcast<EdgeIndexQuery>(parsed_query.query) != nullptr || utils::Downcast<ConstraintQuery>(parsed_query.query) != nullptr || upper_case_query.find(kSchemaAssert) != std::string::npos; SetupDatabaseTransaction(could_commit, unique); @@ -4326,6 +4401,9 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string, } else if (utils::Downcast<IndexQuery>(parsed_query.query)) { prepared_query = PrepareIndexQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->notifications, current_db_); + } else if (utils::Downcast<EdgeIndexQuery>(parsed_query.query)) { + prepared_query = PrepareEdgeIndexQuery(std::move(parsed_query), in_explicit_transaction_, + &query_execution->notifications, current_db_); } else if (utils::Downcast<AnalyzeGraphQuery>(parsed_query.query)) { prepared_query = PrepareAnalyzeGraphQuery(std::move(parsed_query), in_explicit_transaction_, current_db_); } else if (utils::Downcast<AuthQuery>(parsed_query.query)) { diff --git a/src/query/plan/hint_provider.hpp b/src/query/plan/hint_provider.hpp index b70de9aaf..3c8510561 100644 --- a/src/query/plan/hint_provider.hpp +++ b/src/query/plan/hint_provider.hpp @@ -114,6 +114,9 @@ class PlanHintsProvider final : public HierarchicalLogicalOperatorVisitor { bool PreVisit(ScanAllById & /*unused*/) override { return true; } bool PostVisit(ScanAllById & /*unused*/) override { return true; } + bool PreVisit(ScanAllByEdgeType & /*unused*/) override { return true; } + bool PostVisit(ScanAllByEdgeType & /*unused*/) override { return true; } + bool PreVisit(ConstructNamedPath & /*unused*/) override { return true; } bool PostVisit(ConstructNamedPath & /*unused*/) override { return true; } diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp index ba421b653..7cd506050 100644 --- a/src/query/plan/operator.cpp +++ b/src/query/plan/operator.cpp @@ -105,6 +105,7 @@ extern const Event ScanAllByLabelPropertyRangeOperator; extern const Event ScanAllByLabelPropertyValueOperator; extern const Event ScanAllByLabelPropertyOperator; extern const Event ScanAllByIdOperator; +extern const Event ScanAllByEdgeTypeOperator; extern const Event ExpandOperator; extern const Event ExpandVariableOperator; extern const Event ConstructNamedPathOperator; @@ -517,6 +518,60 @@ class ScanAllCursor : public Cursor { const char *op_name_; }; +template <typename TEdgesFun> +class ScanAllByEdgeTypeCursor : public Cursor { + public: + explicit ScanAllByEdgeTypeCursor(const ScanAllByEdgeType &self, Symbol output_symbol, UniqueCursorPtr input_cursor, + storage::View view, TEdgesFun get_edges, const char *op_name) + : self_(self), + output_symbol_(std::move(output_symbol)), + input_cursor_(std::move(input_cursor)), + view_(view), + get_edges_(std::move(get_edges)), + op_name_(op_name) {} + + bool Pull(Frame &frame, ExecutionContext &context) override { + OOMExceptionEnabler oom_exception; + SCOPED_PROFILE_OP_BY_REF(self_); + + AbortCheck(context); + + while (!vertices_ || vertices_it_.value() == vertices_end_it_.value()) { + if (!input_cursor_->Pull(frame, context)) return false; + auto next_vertices = get_edges_(frame, context); + if (!next_vertices) continue; + + vertices_.emplace(std::move(next_vertices.value())); + vertices_it_.emplace(vertices_.value().begin()); + vertices_end_it_.emplace(vertices_.value().end()); + } + + frame[output_symbol_] = *vertices_it_.value(); + ++vertices_it_.value(); + return true; + } + + void Shutdown() override { input_cursor_->Shutdown(); } + + void Reset() override { + input_cursor_->Reset(); + vertices_ = std::nullopt; + vertices_it_ = std::nullopt; + vertices_end_it_ = std::nullopt; + } + + private: + const ScanAllByEdgeType &self_; + const Symbol output_symbol_; + const UniqueCursorPtr input_cursor_; + storage::View view_; + TEdgesFun get_edges_; + std::optional<typename std::result_of<TEdgesFun(Frame &, ExecutionContext &)>::type::value_type> vertices_; + std::optional<decltype(vertices_.value().begin())> vertices_it_; + std::optional<decltype(vertices_.value().end())> vertices_end_it_; + const char *op_name_; +}; + ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::View view) : input_(input ? input : std::make_shared<Once>()), output_symbol_(std::move(output_symbol)), view_(view) {} @@ -556,6 +611,33 @@ UniqueCursorPtr ScanAllByLabel::MakeCursor(utils::MemoryResource *mem) const { view_, std::move(vertices), "ScanAllByLabel"); } +ScanAllByEdgeType::ScanAllByEdgeType(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, + storage::EdgeTypeId edge_type, storage::View view) + : input_(input ? input : std::make_shared<Once>()), + output_symbol_(std::move(output_symbol)), + view_(view), + edge_type_(edge_type) {} + +ACCEPT_WITH_INPUT(ScanAllByEdgeType) + +UniqueCursorPtr ScanAllByEdgeType::MakeCursor(utils::MemoryResource *mem) const { + memgraph::metrics::IncrementCounter(memgraph::metrics::ScanAllByEdgeTypeOperator); + + auto edges = [this](Frame &, ExecutionContext &context) { + auto *db = context.db_accessor; + return std::make_optional(db->Edges(view_, edge_type_)); + }; + + return MakeUniqueCursorPtr<ScanAllByEdgeTypeCursor<decltype(edges)>>( + mem, *this, output_symbol_, input_->MakeCursor(mem), view_, std::move(edges), "ScanAllByEdgeType"); +} + +std::vector<Symbol> ScanAllByEdgeType::ModifiedSymbols(const SymbolTable &table) const { + auto symbols = input_->ModifiedSymbols(table); + symbols.emplace_back(output_symbol_); + return symbols; +} + // TODO(buda): Implement ScanAllByLabelProperty operator to iterate over // vertices that have the label and some value for the given property. diff --git a/src/query/plan/operator.hpp b/src/query/plan/operator.hpp index cdaca2875..6563c2bb0 100644 --- a/src/query/plan/operator.hpp +++ b/src/query/plan/operator.hpp @@ -99,6 +99,7 @@ class ScanAllByLabelPropertyRange; class ScanAllByLabelPropertyValue; class ScanAllByLabelProperty; class ScanAllById; +class ScanAllByEdgeType; class Expand; class ExpandVariable; class ConstructNamedPath; @@ -134,10 +135,10 @@ class RollUpApply; using LogicalOperatorCompositeVisitor = utils::CompositeVisitor<Once, CreateNode, CreateExpand, ScanAll, ScanAllByLabel, ScanAllByLabelPropertyRange, - ScanAllByLabelPropertyValue, ScanAllByLabelProperty, ScanAllById, Expand, ExpandVariable, - ConstructNamedPath, Filter, Produce, Delete, SetProperty, SetProperties, SetLabels, - RemoveProperty, RemoveLabels, EdgeUniquenessFilter, Accumulate, Aggregate, Skip, Limit, - OrderBy, Merge, Optional, Unwind, Distinct, Union, Cartesian, CallProcedure, LoadCsv, + ScanAllByLabelPropertyValue, ScanAllByLabelProperty, ScanAllById, ScanAllByEdgeType, Expand, + ExpandVariable, ConstructNamedPath, Filter, Produce, Delete, SetProperty, SetProperties, + SetLabels, RemoveProperty, RemoveLabels, EdgeUniquenessFilter, Accumulate, Aggregate, Skip, + Limit, OrderBy, Merge, Optional, Unwind, Distinct, Union, Cartesian, CallProcedure, LoadCsv, Foreach, EmptyResult, EvaluatePatternFilter, Apply, IndexedJoin, HashJoin, RollUpApply>; using LogicalOperatorLeafVisitor = utils::LeafVisitor<Once>; @@ -592,6 +593,42 @@ class ScanAllByLabel : public memgraph::query::plan::ScanAll { } }; +class ScanAllByEdgeType : public memgraph::query::plan::LogicalOperator { + public: + static const utils::TypeInfo kType; + const utils::TypeInfo &GetTypeInfo() const override { return kType; } + + ScanAllByEdgeType() = default; + ScanAllByEdgeType(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::EdgeTypeId edge_type, + storage::View view = storage::View::OLD); + bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override; + UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override; + std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override; + + bool HasSingleInput() const override { return true; } + std::shared_ptr<LogicalOperator> input() const override { return input_; } + void set_input(std::shared_ptr<LogicalOperator> input) override { input_ = input; } + + std::string ToString() const override { + return fmt::format("ScanAllByEdgeType ({} :{})", output_symbol_.name(), dba_->EdgeTypeToName(edge_type_)); + } + + std::shared_ptr<memgraph::query::plan::LogicalOperator> input_; + Symbol output_symbol_; + storage::View view_; + + storage::EdgeTypeId edge_type_; + + std::unique_ptr<LogicalOperator> Clone(AstStorage *storage) const override { + auto object = std::make_unique<ScanAllByEdgeType>(); + object->input_ = input_ ? input_->Clone(storage) : nullptr; + object->output_symbol_ = output_symbol_; + object->view_ = view_; + object->edge_type_ = edge_type_; + return object; + } +}; + /// Behaves like @c ScanAll, but produces only vertices with given label and /// property value which is inside a range (inclusive or exlusive). /// diff --git a/src/query/plan/operator_type_info.cpp b/src/query/plan/operator_type_info.cpp index 168137552..6b0a28313 100644 --- a/src/query/plan/operator_type_info.cpp +++ b/src/query/plan/operator_type_info.cpp @@ -49,6 +49,8 @@ constexpr utils::TypeInfo query::plan::ScanAllByLabelProperty::kType{ constexpr utils::TypeInfo query::plan::ScanAllById::kType{utils::TypeId::SCAN_ALL_BY_ID, "ScanAllById", &query::plan::ScanAll::kType}; +constexpr utils::TypeInfo query::plan::ScanAllByEdgeType::kType{utils::TypeId::SCAN_ALL_BY_EDGE_TYPE, + "ScanAllByEdgeType", &query::plan::ScanAll::kType}; constexpr utils::TypeInfo query::plan::ExpandCommon::kType{utils::TypeId::EXPAND_COMMON, "ExpandCommon", nullptr}; diff --git a/src/query/plan/planner.hpp b/src/query/plan/planner.hpp index e8ca80e39..3136e7271 100644 --- a/src/query/plan/planner.hpp +++ b/src/query/plan/planner.hpp @@ -23,6 +23,7 @@ #include "query/plan/operator.hpp" #include "query/plan/preprocess.hpp" #include "query/plan/pretty_print.hpp" +#include "query/plan/rewrite/edge_type_index_lookup.hpp" #include "query/plan/rewrite/index_lookup.hpp" #include "query/plan/rewrite/join.hpp" #include "query/plan/rule_based_planner.hpp" @@ -54,8 +55,11 @@ class PostProcessor final { std::unique_ptr<LogicalOperator> Rewrite(std::unique_ptr<LogicalOperator> plan, TPlanningContext *context) { auto index_lookup_plan = RewriteWithIndexLookup(std::move(plan), context->symbol_table, context->ast_storage, context->db, index_hints_); - return RewriteWithJoinRewriter(std::move(index_lookup_plan), context->symbol_table, context->ast_storage, - context->db); + auto join_plan = + RewriteWithJoinRewriter(std::move(index_lookup_plan), context->symbol_table, context->ast_storage, context->db); + auto edge_index_plan = RewriteWithEdgeTypeIndexRewriter(std::move(join_plan), context->symbol_table, + context->ast_storage, context->db); + return edge_index_plan; } template <class TVertexCounts> diff --git a/src/query/plan/pretty_print.cpp b/src/query/plan/pretty_print.cpp index 7938f9c73..eeb0c15b5 100644 --- a/src/query/plan/pretty_print.cpp +++ b/src/query/plan/pretty_print.cpp @@ -76,6 +76,13 @@ bool PlanPrinter::PreVisit(ScanAllById &op) { return true; } +bool PlanPrinter::PreVisit(query::plan::ScanAllByEdgeType &op) { + op.dba_ = dba_; + WithPrintLn([&op](auto &out) { out << "* " << op.ToString(); }); + op.dba_ = nullptr; + return true; +} + bool PlanPrinter::PreVisit(query::plan::Expand &op) { op.dba_ = dba_; WithPrintLn([&op](auto &out) { out << "* " << op.ToString(); }); @@ -464,6 +471,19 @@ bool PlanToJsonVisitor::PreVisit(ScanAllById &op) { return false; } +bool PlanToJsonVisitor::PreVisit(ScanAllByEdgeType &op) { + json self; + self["name"] = "ScanAllByEdgeType"; + self["edge_type"] = ToJson(op.edge_type_, *dba_); + self["output_symbol"] = ToJson(op.output_symbol_); + + op.input_->Accept(*this); + self["input"] = PopOutput(); + + output_ = std::move(self); + return false; +} + bool PlanToJsonVisitor::PreVisit(CreateNode &op) { json self; self["name"] = "CreateNode"; diff --git a/src/query/plan/pretty_print.hpp b/src/query/plan/pretty_print.hpp index af8429b85..d62ae6bf2 100644 --- a/src/query/plan/pretty_print.hpp +++ b/src/query/plan/pretty_print.hpp @@ -67,6 +67,7 @@ class PlanPrinter : public virtual HierarchicalLogicalOperatorVisitor { bool PreVisit(ScanAllByLabelPropertyRange &) override; bool PreVisit(ScanAllByLabelProperty &) override; bool PreVisit(ScanAllById &) override; + bool PreVisit(ScanAllByEdgeType &) override; bool PreVisit(Expand &) override; bool PreVisit(ExpandVariable &) override; @@ -204,6 +205,7 @@ class PlanToJsonVisitor : public virtual HierarchicalLogicalOperatorVisitor { bool PreVisit(ScanAllByLabelPropertyValue &) override; bool PreVisit(ScanAllByLabelProperty &) override; bool PreVisit(ScanAllById &) override; + bool PreVisit(ScanAllByEdgeType &) override; bool PreVisit(EmptyResult &) override; bool PreVisit(Produce &) override; diff --git a/src/query/plan/rewrite/edge_type_index_lookup.hpp b/src/query/plan/rewrite/edge_type_index_lookup.hpp new file mode 100644 index 000000000..ed8666513 --- /dev/null +++ b/src/query/plan/rewrite/edge_type_index_lookup.hpp @@ -0,0 +1,534 @@ +// 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 +// 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. + +/// @file +/// This file provides a plan rewriter which replaces `ScanAll` and `Expand` +/// operations with `ScanAllByEdgeType` if possible. The public entrypoint is +/// `RewriteWithEdgeTypeIndexRewriter`. + +#pragma once + +#include <algorithm> +#include <memory> +#include <optional> +#include <unordered_map> +#include <unordered_set> +#include <vector> + +#include <gflags/gflags.h> + +#include "query/plan/operator.hpp" +#include "query/plan/preprocess.hpp" +#include "query/plan/rewrite/index_lookup.hpp" +#include "utils/algorithm.hpp" + +namespace memgraph::query::plan { + +namespace impl { + +template <class TDbAccessor> +class EdgeTypeIndexRewriter final : public HierarchicalLogicalOperatorVisitor { + public: + EdgeTypeIndexRewriter(SymbolTable *symbol_table, AstStorage *ast_storage, TDbAccessor *db) + : symbol_table_(symbol_table), ast_storage_(ast_storage), db_(db) {} + + using HierarchicalLogicalOperatorVisitor::PostVisit; + using HierarchicalLogicalOperatorVisitor::PreVisit; + using HierarchicalLogicalOperatorVisitor::Visit; + + bool Visit(Once &) override { return true; } + + bool PreVisit(Filter &op) override { + prev_ops_.push_back(&op); + return true; + } + + bool PostVisit(Filter & /*op*/) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ScanAll &op) override { + prev_ops_.push_back(&op); + + if (op.input()->GetTypeInfo() == Once::kType) { + const bool is_node_anon = op.output_symbol_.IsSymbolAnonym(); + once_under_scanall_ = is_node_anon; + } + + return true; + } + + bool PostVisit(ScanAll &op) override { + prev_ops_.pop_back(); + + if (EdgeTypeIndexingPossible()) { + SetOnParent(op.input()); + } + + return true; + } + + bool PreVisit(Expand &op) override { + prev_ops_.push_back(&op); + + if (op.input()->GetTypeInfo() == ScanAll::kType) { + const bool only_one_edge_type = (op.common_.edge_types.size() == 1U); + const bool expansion_is_named = !(op.common_.edge_symbol.IsSymbolAnonym()); + const bool expdanded_node_not_named = op.common_.node_symbol.IsSymbolAnonym(); + + edge_type_index_exist = only_one_edge_type ? db_->EdgeTypeIndexExists(op.common_.edge_types.front()) : false; + + scanall_under_expand_ = only_one_edge_type && expansion_is_named && expdanded_node_not_named; + } + + return true; + } + + bool PostVisit(Expand &op) override { + prev_ops_.pop_back(); + + if (EdgeTypeIndexingPossible()) { + auto indexed_scan = GenEdgeTypeScan(op); + SetOnParent(std::move(indexed_scan)); + } + + return true; + } + + bool PreVisit(ExpandVariable &op) override { + prev_ops_.push_back(&op); + return true; + } + + bool PostVisit(ExpandVariable &expand) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Merge &op) override { + prev_ops_.push_back(&op); + op.input()->Accept(*this); + RewriteBranch(&op.merge_match_); + return false; + } + + bool PostVisit(Merge &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Optional &op) override { + prev_ops_.push_back(&op); + op.input()->Accept(*this); + RewriteBranch(&op.optional_); + return false; + } + + bool PostVisit(Optional &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Cartesian &op) override { + prev_ops_.push_back(&op); + return true; + } + + bool PostVisit(Cartesian &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(IndexedJoin &op) override { + prev_ops_.push_back(&op); + RewriteBranch(&op.main_branch_); + RewriteBranch(&op.sub_branch_); + return false; + } + + bool PostVisit(IndexedJoin &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(HashJoin &op) override { + prev_ops_.push_back(&op); + return true; + } + + bool PostVisit(HashJoin &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Union &op) override { + prev_ops_.push_back(&op); + RewriteBranch(&op.left_op_); + RewriteBranch(&op.right_op_); + return false; + } + + bool PostVisit(Union &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(CreateNode &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(CreateNode &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(CreateExpand &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(CreateExpand &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ScanAllByLabel &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(ScanAllByLabel &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ScanAllByLabelPropertyRange &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(ScanAllByLabelPropertyRange &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ScanAllByLabelPropertyValue &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(ScanAllByLabelPropertyValue &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ScanAllByLabelProperty &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(ScanAllByLabelProperty &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ScanAllById &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(ScanAllById &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ScanAllByEdgeType &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(ScanAllByEdgeType &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(ConstructNamedPath &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(ConstructNamedPath &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Produce &op) override { + prev_ops_.push_back(&op); + + if (op.input()->GetTypeInfo() == Expand::kType) { + expand_under_produce_ = true; + } + + return true; + } + bool PostVisit(Produce &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(EmptyResult &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(EmptyResult &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Delete &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(Delete &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(SetProperty &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(SetProperty &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(SetProperties &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(SetProperties &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(SetLabels &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(SetLabels &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(RemoveProperty &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(RemoveProperty &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(RemoveLabels &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(RemoveLabels &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(EdgeUniquenessFilter &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(EdgeUniquenessFilter &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Accumulate &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(Accumulate &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Aggregate &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(Aggregate &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Skip &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(Skip &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Limit &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(Limit &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(OrderBy &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(OrderBy &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Unwind &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(Unwind &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Distinct &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(Distinct &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(CallProcedure &op) override { + prev_ops_.push_back(&op); + return true; + } + bool PostVisit(CallProcedure &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Foreach &op) override { + prev_ops_.push_back(&op); + op.input()->Accept(*this); + RewriteBranch(&op.update_clauses_); + return false; + } + + bool PostVisit(Foreach &) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(EvaluatePatternFilter &op) override { + prev_ops_.push_back(&op); + return true; + } + + bool PostVisit(EvaluatePatternFilter & /*op*/) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(Apply &op) override { + prev_ops_.push_back(&op); + op.input()->Accept(*this); + RewriteBranch(&op.subquery_); + return false; + } + + bool PostVisit(Apply & /*op*/) override { + prev_ops_.pop_back(); + return true; + } + + bool PreVisit(LoadCsv &op) override { + prev_ops_.push_back(&op); + return true; + } + + bool PostVisit(LoadCsv & /*op*/) override { + prev_ops_.pop_back(); + return true; + } + + std::shared_ptr<LogicalOperator> new_root_; + + private: + SymbolTable *symbol_table_; + AstStorage *ast_storage_; + TDbAccessor *db_; + // Collected filters, pending for examination if they can be used for advanced + // lookup operations (by index, node ID, ...). + Filters filters_; + // Expressions which no longer need a plain Filter operator. + std::unordered_set<Expression *> filter_exprs_for_removal_; + std::vector<LogicalOperator *> prev_ops_; + std::unordered_set<Symbol> cartesian_symbols_; + + bool EdgeTypeIndexingPossible() const { + return expand_under_produce_ && scanall_under_expand_ && once_under_scanall_ && edge_type_index_exist; + } + bool expand_under_produce_ = false; + bool scanall_under_expand_ = false; + bool once_under_scanall_ = false; + bool edge_type_index_exist = false; + + bool DefaultPreVisit() override { + throw utils::NotYetImplemented("Operator not yet covered by EdgeTypeIndexRewriter"); + } + + std::unique_ptr<ScanAllByEdgeType> GenEdgeTypeScan(const Expand &expand) { + const auto &input = expand.input(); + const auto &output_symbol = expand.common_.edge_symbol; + const auto &view = expand.view_; + + // Extract edge_type from symbol + auto edge_type = expand.common_.edge_types.front(); + return std::make_unique<ScanAllByEdgeType>(input, output_symbol, edge_type, view); + } + + void SetOnParent(const std::shared_ptr<LogicalOperator> &input) { + MG_ASSERT(input); + if (prev_ops_.empty()) { + MG_ASSERT(!new_root_); + new_root_ = input; + return; + } + prev_ops_.back()->set_input(input); + } + + void RewriteBranch(std::shared_ptr<LogicalOperator> *branch) { + EdgeTypeIndexRewriter<TDbAccessor> rewriter(symbol_table_, ast_storage_, db_); + (*branch)->Accept(rewriter); + if (rewriter.new_root_) { + *branch = rewriter.new_root_; + } + } +}; + +} // namespace impl + +template <class TDbAccessor> +std::unique_ptr<LogicalOperator> RewriteWithEdgeTypeIndexRewriter(std::unique_ptr<LogicalOperator> root_op, + SymbolTable *symbol_table, AstStorage *ast_storage, + TDbAccessor *db) { + impl::EdgeTypeIndexRewriter<TDbAccessor> rewriter(symbol_table, ast_storage, db); + root_op->Accept(rewriter); + return root_op; +} + +} // namespace memgraph::query::plan diff --git a/src/query/plan/vertex_count_cache.hpp b/src/query/plan/vertex_count_cache.hpp index 4cfb2486b..802f4e09f 100644 --- a/src/query/plan/vertex_count_cache.hpp +++ b/src/query/plan/vertex_count_cache.hpp @@ -78,6 +78,8 @@ class VertexCountCache { return db_->LabelPropertyIndexExists(label, property); } + bool EdgeTypeIndexExists(storage::EdgeTypeId edge_type) { return db_->EdgeTypeIndexExists(edge_type); } + std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const { return db_->GetIndexStats(label); } diff --git a/src/query/procedure/module.hpp b/src/query/procedure/module.hpp index 41cda0ca6..f5027dafa 100644 --- a/src/query/procedure/module.hpp +++ b/src/query/procedure/module.hpp @@ -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 diff --git a/src/query/procedure/py_module.hpp b/src/query/procedure/py_module.hpp index 9cb22fe2c..fe93b5c51 100644 --- a/src/query/procedure/py_module.hpp +++ b/src/query/procedure/py_module.hpp @@ -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 diff --git a/src/storage/v2/CMakeLists.txt b/src/storage/v2/CMakeLists.txt index 150a02cc7..ec5108d63 100644 --- a/src/storage/v2/CMakeLists.txt +++ b/src/storage/v2/CMakeLists.txt @@ -21,8 +21,10 @@ add_library(mg-storage-v2 STATIC storage.cpp indices/indices.cpp all_vertices_iterable.cpp + edges_iterable.cpp vertices_iterable.cpp inmemory/storage.cpp + inmemory/edge_type_index.cpp inmemory/label_index.cpp inmemory/label_property_index.cpp inmemory/unique_constraints.cpp @@ -30,6 +32,7 @@ add_library(mg-storage-v2 STATIC disk/edge_import_mode_cache.cpp disk/storage.cpp disk/rocksdb_storage.cpp + disk/edge_type_index.cpp disk/label_index.cpp disk/label_property_index.cpp disk/unique_constraints.cpp diff --git a/src/storage/v2/disk/edge_type_index.cpp b/src/storage/v2/disk/edge_type_index.cpp new file mode 100644 index 000000000..d11eb6caf --- /dev/null +++ b/src/storage/v2/disk/edge_type_index.cpp @@ -0,0 +1,49 @@ +// 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 +// 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 "edge_type_index.hpp" + +#include "utils/exceptions.hpp" + +namespace memgraph::storage { + +bool DiskEdgeTypeIndex::DropIndex(EdgeTypeId /*edge_type*/) { + spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode."); + return true; +} + +bool DiskEdgeTypeIndex::IndexExists(EdgeTypeId /*edge_type*/) const { + spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode."); + return false; +} + +std::vector<EdgeTypeId> DiskEdgeTypeIndex::ListIndices() const { + spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode."); + return {}; +} + +uint64_t DiskEdgeTypeIndex::ApproximateEdgeCount(EdgeTypeId /*edge_type*/) const { + spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode."); + return 0U; +} + +void DiskEdgeTypeIndex::UpdateOnEdgeCreation(Vertex * /*from*/, Vertex * /*to*/, EdgeRef /*edge_ref*/, + EdgeTypeId /*edge_type*/, const Transaction & /*tx*/) { + spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode."); +} + +void DiskEdgeTypeIndex::UpdateOnEdgeModification(Vertex * /*old_from*/, Vertex * /*old_to*/, Vertex * /*new_from*/, + Vertex * /*new_to*/, EdgeRef /*edge_ref*/, EdgeTypeId /*edge_type*/, + const Transaction & /*tx*/) { + spdlog::warn("Edge-type index related operations are not yet supported using on-disk storage mode."); +} + +} // namespace memgraph::storage diff --git a/src/storage/v2/disk/edge_type_index.hpp b/src/storage/v2/disk/edge_type_index.hpp new file mode 100644 index 000000000..fe79b2690 --- /dev/null +++ b/src/storage/v2/disk/edge_type_index.hpp @@ -0,0 +1,35 @@ +// 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 +// 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. + +#pragma once + +#include "storage/v2/indices/edge_type_index.hpp" + +namespace memgraph::storage { + +class DiskEdgeTypeIndex : public storage::EdgeTypeIndex { + public: + bool DropIndex(EdgeTypeId edge_type) override; + + bool IndexExists(EdgeTypeId edge_type) const override; + + std::vector<EdgeTypeId> ListIndices() const override; + + uint64_t ApproximateEdgeCount(EdgeTypeId edge_type) const override; + + void UpdateOnEdgeCreation(Vertex *from, Vertex *to, EdgeRef edge_ref, EdgeTypeId edge_type, + const Transaction &tx) override; + + void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, EdgeRef edge_ref, + EdgeTypeId edge_type, const Transaction &tx) override; +}; + +} // namespace memgraph::storage diff --git a/src/storage/v2/disk/storage.cpp b/src/storage/v2/disk/storage.cpp index f9cd2ac13..21ae7755e 100644 --- a/src/storage/v2/disk/storage.cpp +++ b/src/storage/v2/disk/storage.cpp @@ -41,6 +41,7 @@ #include "storage/v2/edge_accessor.hpp" #include "storage/v2/edge_import_mode.hpp" #include "storage/v2/edge_ref.hpp" +#include "storage/v2/edges_iterable.hpp" #include "storage/v2/id_types.hpp" #include "storage/v2/modified_edge.hpp" #include "storage/v2/mvcc.hpp" @@ -807,11 +808,21 @@ void DiskStorage::LoadVerticesFromDiskLabelPropertyIndexForIntervalSearch( } } +EdgesIterable DiskStorage::DiskAccessor::Edges(EdgeTypeId /*edge_type*/, View /*view*/) { + throw utils::NotYetImplemented( + "Edge-type index related operations are not yet supported using on-disk storage mode."); +} + uint64_t DiskStorage::DiskAccessor::ApproximateVertexCount() const { auto *disk_storage = static_cast<DiskStorage *>(storage_); return disk_storage->vertex_count_.load(std::memory_order_acquire); } +uint64_t DiskStorage::DiskAccessor::ApproximateEdgeCount(EdgeTypeId /*edge_type*/) const { + spdlog::info("Edge-type index related operations are not yet supported using on-disk storage mode."); + return 0U; +} + uint64_t DiskStorage::GetDiskSpaceUsage() const { uint64_t main_disk_storage_size = utils::GetDirDiskUsage(config_.disk.main_storage_directory); uint64_t index_disk_storage_size = utils::GetDirDiskUsage(config_.disk.label_index_directory) + @@ -1629,6 +1640,9 @@ utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::Co return StorageManipulationError{PersistenceError{}}; } } break; + case MetadataDelta::Action::EDGE_INDEX_CREATE: { + throw utils::NotYetImplemented("Edge-type indexing is not yet implemented on on-disk storage mode."); + } case MetadataDelta::Action::LABEL_INDEX_DROP: { if (!disk_storage->durable_metadata_.PersistLabelIndexDeletion(md_delta.label)) { return StorageManipulationError{PersistenceError{}}; @@ -1641,6 +1655,9 @@ utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::Co return StorageManipulationError{PersistenceError{}}; } } break; + case MetadataDelta::Action::EDGE_INDEX_DROP: { + throw utils::NotYetImplemented("Edge-type indexing is not yet implemented on on-disk storage mode."); + } case MetadataDelta::Action::LABEL_INDEX_STATS_SET: { throw utils::NotYetImplemented("SetIndexStats(stats) is not implemented for DiskStorage."); } break; @@ -1917,6 +1934,11 @@ utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor: return {}; } +utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor::CreateIndex(EdgeTypeId /*edge_type*/) { + throw utils::NotYetImplemented( + "Edge-type index related operations are not yet supported using on-disk storage mode."); +} + utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor::DropIndex(LabelId label) { MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!"); auto *on_disk = static_cast<DiskStorage *>(storage_); @@ -1945,6 +1967,11 @@ utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor: return {}; } +utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor::DropIndex(EdgeTypeId /*edge_type*/) { + throw utils::NotYetImplemented( + "Edge-type index related operations are not yet supported using on-disk storage mode."); +} + utils::BasicResult<StorageExistenceConstraintDefinitionError, void> DiskStorage::DiskAccessor::CreateExistenceConstraint(LabelId label, PropertyId property) { MG_ASSERT(unique_guard_.owns_lock(), "Create existence constraint requires a unique access to the storage!"); @@ -2053,6 +2080,12 @@ std::unique_ptr<Storage::Accessor> DiskStorage::UniqueAccess( return std::unique_ptr<DiskAccessor>( new DiskAccessor{Storage::Accessor::unique_access, this, isolation_level, storage_mode_}); } + +bool DiskStorage::DiskAccessor::EdgeTypeIndexExists(EdgeTypeId /*edge_type*/) const { + spdlog::info("Edge-type index related operations are not yet supported using on-disk storage mode."); + return false; +} + IndicesInfo DiskStorage::DiskAccessor::ListAllIndices() const { auto *on_disk = static_cast<DiskStorage *>(storage_); auto *disk_label_index = static_cast<DiskLabelIndex *>(on_disk->indices_.label_index_.get()); diff --git a/src/storage/v2/disk/storage.hpp b/src/storage/v2/disk/storage.hpp index 4d71fd10b..349a7454a 100644 --- a/src/storage/v2/disk/storage.hpp +++ b/src/storage/v2/disk/storage.hpp @@ -72,6 +72,8 @@ class DiskStorage final : public Storage { const std::optional<utils::Bound<PropertyValue>> &lower_bound, const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) override; + EdgesIterable Edges(EdgeTypeId edge_type, View view) override; + uint64_t ApproximateVertexCount() const override; uint64_t ApproximateVertexCount(LabelId /*label*/) const override { return 10; } @@ -89,6 +91,8 @@ class DiskStorage final : public Storage { return 10; } + uint64_t ApproximateEdgeCount(EdgeTypeId edge_type) const override; + std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId & /*label*/) const override { return {}; } @@ -140,6 +144,8 @@ class DiskStorage final : public Storage { return disk_storage->indices_.label_property_index_->IndexExists(label, property); } + bool EdgeTypeIndexExists(EdgeTypeId edge_type) const override; + IndicesInfo ListAllIndices() const override; ConstraintsInfo ListAllConstraints() const override; @@ -158,10 +164,14 @@ class DiskStorage final : public Storage { utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) override; + utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(EdgeTypeId edge_type) override; + utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) override; utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) override; + utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(EdgeTypeId edge_type) override; + utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint( LabelId label, PropertyId property) override; diff --git a/src/storage/v2/durability/durability.cpp b/src/storage/v2/durability/durability.cpp index a83313820..fbbedbee5 100644 --- a/src/storage/v2/durability/durability.cpp +++ b/src/storage/v2/durability/durability.cpp @@ -31,6 +31,7 @@ #include "storage/v2/durability/paths.hpp" #include "storage/v2/durability/snapshot.hpp" #include "storage/v2/durability/wal.hpp" +#include "storage/v2/inmemory/edge_type_index.hpp" #include "storage/v2/inmemory/label_index.hpp" #include "storage/v2/inmemory/label_property_index.hpp" #include "storage/v2/inmemory/unique_constraints.hpp" @@ -199,9 +200,18 @@ void RecoverIndicesAndStats(const RecoveredIndicesAndConstraints::IndicesMetadat } spdlog::info("Label+property indices statistics are recreated."); - spdlog::info("Indices are recreated."); + // Recover edge-type indices. + spdlog::info("Recreating {} edge-type indices from metadata.", indices_metadata.edge.size()); + auto *mem_edge_type_index = static_cast<InMemoryEdgeTypeIndex *>(indices->edge_type_index_.get()); + for (const auto &item : indices_metadata.edge) { + if (!mem_edge_type_index->CreateIndex(item, vertices->access())) { + throw RecoveryFailure("The edge-type index must be created here!"); + } + spdlog::info("Index on :{} is recreated from metadata", name_id_mapper->IdToName(item.AsUint())); + } + spdlog::info("Edge-type indices are recreated."); - spdlog::info("Recreating constraints from metadata."); + spdlog::info("Indices are recreated."); } void RecoverExistenceConstraints(const RecoveredIndicesAndConstraints::ConstraintsMetadata &constraints_metadata, diff --git a/src/storage/v2/durability/marker.hpp b/src/storage/v2/durability/marker.hpp index 8f00d435d..ac0cc074d 100644 --- a/src/storage/v2/durability/marker.hpp +++ b/src/storage/v2/durability/marker.hpp @@ -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 @@ -37,6 +37,8 @@ enum class Marker : uint8_t { SECTION_CONSTRAINTS = 0x25, SECTION_DELTA = 0x26, SECTION_EPOCH_HISTORY = 0x27, + SECTION_EDGE_INDICES = 0x28, + SECTION_OFFSETS = 0x42, DELTA_VERTEX_CREATE = 0x50, @@ -60,6 +62,8 @@ enum class Marker : uint8_t { DELTA_LABEL_INDEX_STATS_CLEAR = 0x62, DELTA_LABEL_PROPERTY_INDEX_STATS_SET = 0x63, DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR = 0x64, + DELTA_EDGE_TYPE_INDEX_CREATE = 0x65, + DELTA_EDGE_TYPE_INDEX_DROP = 0x66, VALUE_FALSE = 0x00, VALUE_TRUE = 0xff, @@ -85,6 +89,7 @@ static const Marker kMarkersAll[] = { Marker::SECTION_CONSTRAINTS, Marker::SECTION_DELTA, Marker::SECTION_EPOCH_HISTORY, + Marker::SECTION_EDGE_INDICES, Marker::SECTION_OFFSETS, Marker::DELTA_VERTEX_CREATE, Marker::DELTA_VERTEX_DELETE, @@ -103,6 +108,8 @@ static const Marker kMarkersAll[] = { Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR, Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE, Marker::DELTA_LABEL_PROPERTY_INDEX_DROP, + Marker::DELTA_EDGE_TYPE_INDEX_CREATE, + Marker::DELTA_EDGE_TYPE_INDEX_DROP, Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE, Marker::DELTA_EXISTENCE_CONSTRAINT_DROP, Marker::DELTA_UNIQUE_CONSTRAINT_CREATE, diff --git a/src/storage/v2/durability/metadata.hpp b/src/storage/v2/durability/metadata.hpp index 42e24e723..c8ee27b2f 100644 --- a/src/storage/v2/durability/metadata.hpp +++ b/src/storage/v2/durability/metadata.hpp @@ -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 @@ -43,6 +43,7 @@ struct RecoveredIndicesAndConstraints { std::vector<std::pair<LabelId, PropertyId>> label_property; std::vector<std::pair<LabelId, LabelIndexStats>> label_stats; std::vector<std::pair<LabelId, std::pair<PropertyId, LabelPropertyIndexStats>>> label_property_stats; + std::vector<EdgeTypeId> edge; } indices; struct ConstraintsMetadata { diff --git a/src/storage/v2/durability/serialization.cpp b/src/storage/v2/durability/serialization.cpp index 6b13d9d00..28ba64943 100644 --- a/src/storage/v2/durability/serialization.cpp +++ b/src/storage/v2/durability/serialization.cpp @@ -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 @@ -332,6 +332,7 @@ std::optional<PropertyValue> Decoder::ReadPropertyValue() { case Marker::SECTION_CONSTRAINTS: case Marker::SECTION_DELTA: case Marker::SECTION_EPOCH_HISTORY: + case Marker::SECTION_EDGE_INDICES: case Marker::SECTION_OFFSETS: case Marker::DELTA_VERTEX_CREATE: case Marker::DELTA_VERTEX_DELETE: @@ -350,6 +351,8 @@ std::optional<PropertyValue> Decoder::ReadPropertyValue() { case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR: case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE: case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP: + case Marker::DELTA_EDGE_TYPE_INDEX_CREATE: + case Marker::DELTA_EDGE_TYPE_INDEX_DROP: case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE: case Marker::DELTA_EXISTENCE_CONSTRAINT_DROP: case Marker::DELTA_UNIQUE_CONSTRAINT_CREATE: @@ -435,6 +438,7 @@ bool Decoder::SkipPropertyValue() { case Marker::SECTION_CONSTRAINTS: case Marker::SECTION_DELTA: case Marker::SECTION_EPOCH_HISTORY: + case Marker::SECTION_EDGE_INDICES: case Marker::SECTION_OFFSETS: case Marker::DELTA_VERTEX_CREATE: case Marker::DELTA_VERTEX_DELETE: @@ -453,6 +457,8 @@ bool Decoder::SkipPropertyValue() { case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR: case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE: case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP: + case Marker::DELTA_EDGE_TYPE_INDEX_CREATE: + case Marker::DELTA_EDGE_TYPE_INDEX_DROP: case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE: case Marker::DELTA_EXISTENCE_CONSTRAINT_DROP: case Marker::DELTA_UNIQUE_CONSTRAINT_CREATE: diff --git a/src/storage/v2/durability/snapshot.cpp b/src/storage/v2/durability/snapshot.cpp index eee099870..5fea3dfa5 100644 --- a/src/storage/v2/durability/snapshot.cpp +++ b/src/storage/v2/durability/snapshot.cpp @@ -153,6 +153,11 @@ SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path) { info.offset_edges = read_offset(); info.offset_vertices = read_offset(); info.offset_indices = read_offset(); + if (*version >= 17) { + info.offset_edge_indices = read_offset(); + } else { + info.offset_edge_indices = 0U; + } info.offset_constraints = read_offset(); info.offset_mapper = read_offset(); info.offset_epoch_history = read_offset(); @@ -1379,10 +1384,11 @@ RecoveredSnapshot LoadSnapshotVersion15(const std::filesystem::path &path, utils return {info, recovery_info, std::move(indices_constraints)}; } -RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices, - utils::SkipList<Edge> *edges, - std::deque<std::pair<std::string, uint64_t>> *epoch_history, - NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, const Config &config) { +RecoveredSnapshot LoadSnapshotVersion16(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices, + utils::SkipList<Edge> *edges, + std::deque<std::pair<std::string, uint64_t>> *epoch_history, + NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, + const Config &config) { RecoveryInfo recovery_info; RecoveredIndicesAndConstraints indices_constraints; @@ -1391,13 +1397,7 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipLis if (!version) throw RecoveryFailure("Couldn't read snapshot magic and/or version!"); if (!IsVersionSupported(*version)) throw RecoveryFailure(fmt::format("Invalid snapshot version {}", *version)); - if (*version == 14U) { - return LoadSnapshotVersion14(path, vertices, edges, epoch_history, name_id_mapper, edge_count, - config.salient.items); - } - if (*version == 15U) { - return LoadSnapshotVersion15(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config); - } + if (*version != 16U) throw RecoveryFailure(fmt::format("Expected snapshot version is 16, but got {}", *version)); // Cleanup of loaded data in case of failure. bool success = false; @@ -1727,6 +1727,380 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipLis return {info, recovery_info, std::move(indices_constraints)}; } +RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices, + utils::SkipList<Edge> *edges, + std::deque<std::pair<std::string, uint64_t>> *epoch_history, + NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, const Config &config) { + RecoveryInfo recovery_info; + RecoveredIndicesAndConstraints indices_constraints; + + Decoder snapshot; + const auto version = snapshot.Initialize(path, kSnapshotMagic); + if (!version) throw RecoveryFailure("Couldn't read snapshot magic and/or version!"); + + if (!IsVersionSupported(*version)) throw RecoveryFailure(fmt::format("Invalid snapshot version {}", *version)); + if (*version == 14U) { + return LoadSnapshotVersion14(path, vertices, edges, epoch_history, name_id_mapper, edge_count, + config.salient.items); + } + if (*version == 15U) { + return LoadSnapshotVersion15(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config); + } + if (*version == 16U) { + return LoadSnapshotVersion16(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config); + } + + // Cleanup of loaded data in case of failure. + bool success = false; + utils::OnScopeExit cleanup([&] { + if (!success) { + edges->clear(); + vertices->clear(); + epoch_history->clear(); + } + }); + + // Read snapshot info. + const auto info = ReadSnapshotInfo(path); + spdlog::info("Recovering {} vertices and {} edges.", info.vertices_count, info.edges_count); + // Check for edges. + bool snapshot_has_edges = info.offset_edges != 0; + + // Recover mapper. + std::unordered_map<uint64_t, uint64_t> snapshot_id_map; + { + spdlog::info("Recovering mapper metadata."); + if (!snapshot.SetPosition(info.offset_mapper)) throw RecoveryFailure("Couldn't read data from snapshot!"); + + auto marker = snapshot.ReadMarker(); + if (!marker || *marker != Marker::SECTION_MAPPER) throw RecoveryFailure("Failed to read section mapper!"); + + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Failed to read name-id mapper size!"); + + for (uint64_t i = 0; i < *size; ++i) { + auto id = snapshot.ReadUint(); + if (!id) throw RecoveryFailure("Failed to read id for name-id mapper!"); + auto name = snapshot.ReadString(); + if (!name) throw RecoveryFailure("Failed to read name for name-id mapper!"); + auto my_id = name_id_mapper->NameToId(*name); + snapshot_id_map.emplace(*id, my_id); + SPDLOG_TRACE("Mapping \"{}\"from snapshot id {} to actual id {}.", *name, *id, my_id); + } + } + auto get_label_from_id = [&snapshot_id_map](uint64_t label_id) { + auto it = snapshot_id_map.find(label_id); + if (it == snapshot_id_map.end()) throw RecoveryFailure("Couldn't find label id in snapshot_id_map!"); + return LabelId::FromUint(it->second); + }; + auto get_property_from_id = [&snapshot_id_map](uint64_t property_id) { + auto it = snapshot_id_map.find(property_id); + if (it == snapshot_id_map.end()) throw RecoveryFailure("Couldn't find property id in snapshot_id_map!"); + return PropertyId::FromUint(it->second); + }; + auto get_edge_type_from_id = [&snapshot_id_map](uint64_t edge_type_id) { + auto it = snapshot_id_map.find(edge_type_id); + if (it == snapshot_id_map.end()) throw RecoveryFailure("Couldn't find edge type id in snapshot_id_map!"); + return EdgeTypeId::FromUint(it->second); + }; + + // Reset current edge count. + edge_count->store(0, std::memory_order_release); + + { + spdlog::info("Recovering edges."); + // Recover edges. + if (snapshot_has_edges) { + // We don't need to check whether we store properties on edge or not, because `LoadPartialEdges` will always + // iterate over the edges in the snapshot (if they exist) and the current configuration of properties on edge only + // affect what it does: + // 1. If properties are allowed on edges, then it loads the edges. + // 2. If properties are not allowed on edges, then it checks that none of the edges have any properties. + if (!snapshot.SetPosition(info.offset_edge_batches)) { + throw RecoveryFailure("Couldn't read data from snapshot!"); + } + const auto edge_batches = ReadBatchInfos(snapshot); + + RecoverOnMultipleThreads( + config.durability.recovery_thread_count, + [path, edges, items = config.salient.items, &get_property_from_id](const size_t /*batch_index*/, + const BatchInfo &batch) { + LoadPartialEdges(path, *edges, batch.offset, batch.count, items, get_property_from_id); + }, + edge_batches); + } + spdlog::info("Edges are recovered."); + + // Recover vertices (labels and properties). + spdlog::info("Recovering vertices.", info.vertices_count); + uint64_t last_vertex_gid{0}; + + if (!snapshot.SetPosition(info.offset_vertex_batches)) { + throw RecoveryFailure("Couldn't read data from snapshot!"); + } + + const auto vertex_batches = ReadBatchInfos(snapshot); + RecoverOnMultipleThreads( + config.durability.recovery_thread_count, + [path, vertices, &vertex_batches, &get_label_from_id, &get_property_from_id, &last_vertex_gid]( + const size_t batch_index, const BatchInfo &batch) { + const auto last_vertex_gid_in_batch = + LoadPartialVertices(path, *vertices, batch.offset, batch.count, get_label_from_id, get_property_from_id); + if (batch_index == vertex_batches.size() - 1) { + last_vertex_gid = last_vertex_gid_in_batch; + } + }, + vertex_batches); + + spdlog::info("Vertices are recovered."); + + // Recover vertices (in/out edges). + spdlog::info("Recover connectivity."); + recovery_info.vertex_batches.reserve(vertex_batches.size()); + for (const auto batch : vertex_batches) { + recovery_info.vertex_batches.emplace_back(Gid::FromUint(0), batch.count); + } + std::atomic<uint64_t> highest_edge_gid{0}; + + RecoverOnMultipleThreads( + config.durability.recovery_thread_count, + [path, vertices, edges, edge_count, items = config.salient.items, snapshot_has_edges, &get_edge_type_from_id, + &highest_edge_gid, &recovery_info](const size_t batch_index, const BatchInfo &batch) { + const auto result = LoadPartialConnectivity(path, *vertices, *edges, batch.offset, batch.count, items, + snapshot_has_edges, get_edge_type_from_id); + edge_count->fetch_add(result.edge_count); + auto known_highest_edge_gid = highest_edge_gid.load(); + while (known_highest_edge_gid < result.highest_edge_id) { + highest_edge_gid.compare_exchange_weak(known_highest_edge_gid, result.highest_edge_id); + } + recovery_info.vertex_batches[batch_index].first = result.first_vertex_gid; + }, + vertex_batches); + + spdlog::info("Connectivity is recovered."); + + // Set initial values for edge/vertex ID generators. + recovery_info.next_edge_id = highest_edge_gid + 1; + recovery_info.next_vertex_id = last_vertex_gid + 1; + } + + // Recover indices. + { + spdlog::info("Recovering metadata of indices."); + if (!snapshot.SetPosition(info.offset_indices)) throw RecoveryFailure("Couldn't read data from snapshot!"); + + auto marker = snapshot.ReadMarker(); + if (!marker || *marker != Marker::SECTION_INDICES) throw RecoveryFailure("Couldn't read section indices!"); + + // Recover label indices. + { + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Couldn't read the number of label indices"); + spdlog::info("Recovering metadata of {} label indices.", *size); + for (uint64_t i = 0; i < *size; ++i) { + auto label = snapshot.ReadUint(); + if (!label) throw RecoveryFailure("Couldn't read label of label index!"); + AddRecoveredIndexConstraint(&indices_constraints.indices.label, get_label_from_id(*label), + "The label index already exists!"); + SPDLOG_TRACE("Recovered metadata of label index for :{}", name_id_mapper->IdToName(snapshot_id_map.at(*label))); + } + spdlog::info("Metadata of label indices are recovered."); + } + + // Recover label indices statistics. + { + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Couldn't read the number of entries for label index statistics!"); + spdlog::info("Recovering metadata of {} label indices statistics.", *size); + for (uint64_t i = 0; i < *size; ++i) { + const auto label = snapshot.ReadUint(); + if (!label) throw RecoveryFailure("Couldn't read label while recovering label index statistics!"); + const auto count = snapshot.ReadUint(); + if (!count) throw RecoveryFailure("Couldn't read count for label index statistics!"); + const auto avg_degree = snapshot.ReadDouble(); + if (!avg_degree) throw RecoveryFailure("Couldn't read average degree for label index statistics"); + const auto label_id = get_label_from_id(*label); + indices_constraints.indices.label_stats.emplace_back(label_id, LabelIndexStats{*count, *avg_degree}); + SPDLOG_TRACE("Recovered metadata of label index statistics for :{}", + name_id_mapper->IdToName(snapshot_id_map.at(*label))); + } + spdlog::info("Metadata of label indices are recovered."); + } + + // Recover label+property indices. + { + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Couldn't recover the number of label property indices!"); + spdlog::info("Recovering metadata of {} label+property indices.", *size); + for (uint64_t i = 0; i < *size; ++i) { + auto label = snapshot.ReadUint(); + if (!label) throw RecoveryFailure("Couldn't read label for label property index!"); + auto property = snapshot.ReadUint(); + if (!property) throw RecoveryFailure("Couldn't read property for label property index"); + AddRecoveredIndexConstraint(&indices_constraints.indices.label_property, + {get_label_from_id(*label), get_property_from_id(*property)}, + "The label+property index already exists!"); + SPDLOG_TRACE("Recovered metadata of label+property index for :{}({})", + name_id_mapper->IdToName(snapshot_id_map.at(*label)), + name_id_mapper->IdToName(snapshot_id_map.at(*property))); + } + spdlog::info("Metadata of label+property indices are recovered."); + } + + // Recover label+property indices statistics. + { + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Couldn't recover the number of entries for label property statistics!"); + spdlog::info("Recovering metadata of {} label+property indices statistics.", *size); + for (uint64_t i = 0; i < *size; ++i) { + const auto label = snapshot.ReadUint(); + if (!label) throw RecoveryFailure("Couldn't read label for label property index statistics!"); + const auto property = snapshot.ReadUint(); + if (!property) throw RecoveryFailure("Couldn't read property for label property index statistics!"); + const auto count = snapshot.ReadUint(); + if (!count) throw RecoveryFailure("Couldn't read count for label property index statistics!!"); + const auto distinct_values_count = snapshot.ReadUint(); + if (!distinct_values_count) + throw RecoveryFailure("Couldn't read distinct values count for label property index statistics!"); + const auto statistic = snapshot.ReadDouble(); + if (!statistic) throw RecoveryFailure("Couldn't read statistics value for label-property index statistics!"); + const auto avg_group_size = snapshot.ReadDouble(); + if (!avg_group_size) + throw RecoveryFailure("Couldn't read average group size for label property index statistics!"); + const auto avg_degree = snapshot.ReadDouble(); + if (!avg_degree) throw RecoveryFailure("Couldn't read average degree for label property index statistics!"); + const auto label_id = get_label_from_id(*label); + const auto property_id = get_property_from_id(*property); + indices_constraints.indices.label_property_stats.emplace_back( + label_id, std::make_pair(property_id, LabelPropertyIndexStats{*count, *distinct_values_count, *statistic, + *avg_group_size, *avg_degree})); + SPDLOG_TRACE("Recovered metadata of label+property index statistics for :{}({})", + name_id_mapper->IdToName(snapshot_id_map.at(*label)), + name_id_mapper->IdToName(snapshot_id_map.at(*property))); + } + spdlog::info("Metadata of label+property indices are recovered."); + } + + // Recover edge-type indices. + spdlog::info("Recovering metadata of indices."); + if (!snapshot.SetPosition(info.offset_edge_indices)) throw RecoveryFailure("Couldn't read data from snapshot!"); + + marker = snapshot.ReadMarker(); + if (!marker || *marker != Marker::SECTION_EDGE_INDICES) + throw RecoveryFailure("Couldn't read section edge-indices!"); + + { + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Couldn't read the number of edge-type indices"); + spdlog::info("Recovering metadata of {} edge-type indices.", *size); + for (uint64_t i = 0; i < *size; ++i) { + auto edge_type = snapshot.ReadUint(); + if (!edge_type) throw RecoveryFailure("Couldn't read edge-type of edge-type index!"); + AddRecoveredIndexConstraint(&indices_constraints.indices.edge, get_edge_type_from_id(*edge_type), + "The edge-type index already exists!"); + SPDLOG_TRACE("Recovered metadata of edge-type index for :{}", + name_id_mapper->IdToName(snapshot_id_map.at(*edge_type))); + } + spdlog::info("Metadata of edge-type indices are recovered."); + } + + spdlog::info("Metadata of indices are recovered."); + } + + // Recover constraints. + { + spdlog::info("Recovering metadata of constraints."); + if (!snapshot.SetPosition(info.offset_constraints)) throw RecoveryFailure("Couldn't read data from snapshot!"); + + auto marker = snapshot.ReadMarker(); + if (!marker || *marker != Marker::SECTION_CONSTRAINTS) + throw RecoveryFailure("Couldn't read section constraints marker!"); + + // Recover existence constraints. + { + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Couldn't read the number of existence constraints!"); + spdlog::info("Recovering metadata of {} existence constraints.", *size); + for (uint64_t i = 0; i < *size; ++i) { + auto label = snapshot.ReadUint(); + if (!label) throw RecoveryFailure("Couldn't read label of existence constraints!"); + auto property = snapshot.ReadUint(); + if (!property) throw RecoveryFailure("Couldn't read property of existence constraints!"); + AddRecoveredIndexConstraint(&indices_constraints.constraints.existence, + {get_label_from_id(*label), get_property_from_id(*property)}, + "The existence constraint already exists!"); + SPDLOG_TRACE("Recovered metadata of existence constraint for :{}({})", + name_id_mapper->IdToName(snapshot_id_map.at(*label)), + name_id_mapper->IdToName(snapshot_id_map.at(*property))); + } + spdlog::info("Metadata of existence constraints are recovered."); + } + + // Recover unique constraints. + // Snapshot version should be checked since unique constraints were + // implemented in later versions of snapshot. + if (*version >= kUniqueConstraintVersion) { + auto size = snapshot.ReadUint(); + if (!size) throw RecoveryFailure("Couldn't read the number of unique constraints!"); + spdlog::info("Recovering metadata of {} unique constraints.", *size); + for (uint64_t i = 0; i < *size; ++i) { + auto label = snapshot.ReadUint(); + if (!label) throw RecoveryFailure("Couldn't read label of unique constraints!"); + auto properties_count = snapshot.ReadUint(); + if (!properties_count) throw RecoveryFailure("Couldn't read the number of properties in unique constraint!"); + std::set<PropertyId> properties; + for (uint64_t j = 0; j < *properties_count; ++j) { + auto property = snapshot.ReadUint(); + if (!property) throw RecoveryFailure("Couldn't read property of unique constraint!"); + properties.insert(get_property_from_id(*property)); + } + AddRecoveredIndexConstraint(&indices_constraints.constraints.unique, {get_label_from_id(*label), properties}, + "The unique constraint already exists!"); + SPDLOG_TRACE("Recovered metadata of unique constraints for :{}", + name_id_mapper->IdToName(snapshot_id_map.at(*label))); + } + spdlog::info("Metadata of unique constraints are recovered."); + } + spdlog::info("Metadata of constraints are recovered."); + } + + spdlog::info("Recovering metadata."); + // Recover epoch history + { + if (!snapshot.SetPosition(info.offset_epoch_history)) throw RecoveryFailure("Couldn't read data from snapshot!"); + + const auto marker = snapshot.ReadMarker(); + if (!marker || *marker != Marker::SECTION_EPOCH_HISTORY) + throw RecoveryFailure("Couldn't read section epoch history marker!"); + + const auto history_size = snapshot.ReadUint(); + if (!history_size) { + throw RecoveryFailure("Couldn't read history size!"); + } + + for (int i = 0; i < *history_size; ++i) { + auto maybe_epoch_id = snapshot.ReadString(); + if (!maybe_epoch_id) { + throw RecoveryFailure("Couldn't read maybe epoch id!"); + } + const auto maybe_last_commit_timestamp = snapshot.ReadUint(); + if (!maybe_last_commit_timestamp) { + throw RecoveryFailure("Couldn't read maybe last commit timestamp!"); + } + epoch_history->emplace_back(std::move(*maybe_epoch_id), *maybe_last_commit_timestamp); + } + } + + spdlog::info("Metadata recovered."); + // Recover timestamp. + recovery_info.next_timestamp = info.start_timestamp + 1; + + // Set success flag (to disable cleanup). + success = true; + + return {info, recovery_info, std::move(indices_constraints)}; +} + using OldSnapshotFiles = std::vector<std::pair<uint64_t, std::filesystem::path>>; void EnsureNecessaryWalFilesExist(const std::filesystem::path &wal_directory, const std::string &uuid, OldSnapshotFiles old_snapshot_files, Transaction *transaction, @@ -1835,6 +2209,7 @@ void CreateSnapshot(Storage *storage, Transaction *transaction, const std::files uint64_t offset_edges = 0; uint64_t offset_vertices = 0; uint64_t offset_indices = 0; + uint64_t offset_edge_indices = 0; uint64_t offset_constraints = 0; uint64_t offset_mapper = 0; uint64_t offset_metadata = 0; @@ -1847,6 +2222,7 @@ void CreateSnapshot(Storage *storage, Transaction *transaction, const std::files snapshot.WriteUint(offset_edges); snapshot.WriteUint(offset_vertices); snapshot.WriteUint(offset_indices); + snapshot.WriteUint(offset_edge_indices); snapshot.WriteUint(offset_constraints); snapshot.WriteUint(offset_mapper); snapshot.WriteUint(offset_epoch_history); @@ -2106,6 +2482,17 @@ void CreateSnapshot(Storage *storage, Transaction *transaction, const std::files snapshot.SetPosition(last_pos); } } + + // Write edge-type indices. + offset_edge_indices = snapshot.GetPosition(); + snapshot.WriteMarker(Marker::SECTION_EDGE_INDICES); + { + auto edge_type = storage->indices_.edge_type_index_->ListIndices(); + snapshot.WriteUint(edge_type.size()); + for (const auto &item : edge_type) { + write_mapping(item); + } + } } // Write constraints. @@ -2196,6 +2583,7 @@ void CreateSnapshot(Storage *storage, Transaction *transaction, const std::files snapshot.WriteUint(offset_edges); snapshot.WriteUint(offset_vertices); snapshot.WriteUint(offset_indices); + snapshot.WriteUint(offset_edge_indices); snapshot.WriteUint(offset_constraints); snapshot.WriteUint(offset_mapper); snapshot.WriteUint(offset_epoch_history); diff --git a/src/storage/v2/durability/snapshot.hpp b/src/storage/v2/durability/snapshot.hpp index 4c1aee1ce..b8c224b3f 100644 --- a/src/storage/v2/durability/snapshot.hpp +++ b/src/storage/v2/durability/snapshot.hpp @@ -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 @@ -34,6 +34,7 @@ struct SnapshotInfo { uint64_t offset_edges; uint64_t offset_vertices; uint64_t offset_indices; + uint64_t offset_edge_indices; uint64_t offset_constraints; uint64_t offset_mapper; uint64_t offset_epoch_history; diff --git a/src/storage/v2/durability/storage_global_operation.hpp b/src/storage/v2/durability/storage_global_operation.hpp index a4f1b043a..7dd635e9d 100644 --- a/src/storage/v2/durability/storage_global_operation.hpp +++ b/src/storage/v2/durability/storage_global_operation.hpp @@ -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 @@ -23,6 +23,8 @@ enum class StorageMetadataOperation { LABEL_PROPERTY_INDEX_DROP, LABEL_PROPERTY_INDEX_STATS_SET, LABEL_PROPERTY_INDEX_STATS_CLEAR, + EDGE_TYPE_INDEX_CREATE, + EDGE_TYPE_INDEX_DROP, EXISTENCE_CONSTRAINT_CREATE, EXISTENCE_CONSTRAINT_DROP, UNIQUE_CONSTRAINT_CREATE, diff --git a/src/storage/v2/durability/version.hpp b/src/storage/v2/durability/version.hpp index 25eb30904..58ca0364a 100644 --- a/src/storage/v2/durability/version.hpp +++ b/src/storage/v2/durability/version.hpp @@ -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 @@ -20,7 +20,7 @@ namespace memgraph::storage::durability { // The current version of snapshot and WAL encoding / decoding. // IMPORTANT: Please bump this version for every snapshot and/or WAL format // change!!! -const uint64_t kVersion{16}; +const uint64_t kVersion{17}; const uint64_t kOldestSupportedVersion{14}; const uint64_t kUniqueConstraintVersion{13}; diff --git a/src/storage/v2/durability/wal.cpp b/src/storage/v2/durability/wal.cpp index 52e916052..5c40ab1c5 100644 --- a/src/storage/v2/durability/wal.cpp +++ b/src/storage/v2/durability/wal.cpp @@ -95,6 +95,10 @@ Marker OperationToMarker(StorageMetadataOperation operation) { return Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_SET; case StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_CLEAR: return Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR; + case StorageMetadataOperation::EDGE_TYPE_INDEX_CREATE: + return Marker::DELTA_EDGE_TYPE_INDEX_CREATE; + case StorageMetadataOperation::EDGE_TYPE_INDEX_DROP: + return Marker::DELTA_EDGE_TYPE_INDEX_DROP; case StorageMetadataOperation::EXISTENCE_CONSTRAINT_CREATE: return Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE; case StorageMetadataOperation::EXISTENCE_CONSTRAINT_DROP: @@ -172,6 +176,10 @@ WalDeltaData::Type MarkerToWalDeltaDataType(Marker marker) { return WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET; case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR: return WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR; + case Marker::DELTA_EDGE_TYPE_INDEX_CREATE: + return WalDeltaData::Type::EDGE_INDEX_CREATE; + case Marker::DELTA_EDGE_TYPE_INDEX_DROP: + return WalDeltaData::Type::EDGE_INDEX_DROP; case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE: return WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE; case Marker::DELTA_EXISTENCE_CONSTRAINT_DROP: @@ -198,6 +206,7 @@ WalDeltaData::Type MarkerToWalDeltaDataType(Marker marker) { case Marker::SECTION_CONSTRAINTS: case Marker::SECTION_DELTA: case Marker::SECTION_EPOCH_HISTORY: + case Marker::SECTION_EDGE_INDICES: case Marker::SECTION_OFFSETS: case Marker::VALUE_FALSE: case Marker::VALUE_TRUE: @@ -280,6 +289,7 @@ WalDeltaData ReadSkipWalDeltaData(BaseDecoder *decoder) { } case WalDeltaData::Type::TRANSACTION_END: break; + // NOLINTNEXTLINE(bugprone-branch-clone) case WalDeltaData::Type::LABEL_INDEX_CREATE: case WalDeltaData::Type::LABEL_INDEX_DROP: case WalDeltaData::Type::LABEL_INDEX_STATS_CLEAR: @@ -295,6 +305,17 @@ WalDeltaData ReadSkipWalDeltaData(BaseDecoder *decoder) { } break; } + case WalDeltaData::Type::EDGE_INDEX_CREATE: + case WalDeltaData::Type::EDGE_INDEX_DROP: { + if constexpr (read_data) { + auto edge_type = decoder->ReadString(); + if (!edge_type) throw RecoveryFailure("Invalid WAL data!"); + delta.operation_edge_type.edge_type = std::move(*edge_type); + } else { + if (!decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!"); + } + break; + } case WalDeltaData::Type::LABEL_INDEX_STATS_SET: { if constexpr (read_data) { auto label = decoder->ReadString(); @@ -522,6 +543,9 @@ bool operator==(const WalDeltaData &a, const WalDeltaData &b) { case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP: return a.operation_label_properties.label == b.operation_label_properties.label && a.operation_label_properties.properties == b.operation_label_properties.properties; + case WalDeltaData::Type::EDGE_INDEX_CREATE: + case WalDeltaData::Type::EDGE_INDEX_DROP: + return a.operation_edge_type.edge_type == b.operation_edge_type.edge_type; } } bool operator!=(const WalDeltaData &a, const WalDeltaData &b) { return !(a == b); } @@ -703,6 +727,37 @@ void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, Storage } break; } + case StorageMetadataOperation::EDGE_TYPE_INDEX_CREATE: + case StorageMetadataOperation::EDGE_TYPE_INDEX_DROP: { + MG_ASSERT(false, "Invalid function call!"); + } + } +} + +void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageMetadataOperation operation, + EdgeTypeId edge_type, uint64_t timestamp) { + encoder->WriteMarker(Marker::SECTION_DELTA); + encoder->WriteUint(timestamp); + switch (operation) { + case StorageMetadataOperation::EDGE_TYPE_INDEX_CREATE: + case StorageMetadataOperation::EDGE_TYPE_INDEX_DROP: { + encoder->WriteMarker(OperationToMarker(operation)); + encoder->WriteString(name_id_mapper->IdToName(edge_type.AsUint())); + break; + } + case StorageMetadataOperation::LABEL_INDEX_CREATE: + case StorageMetadataOperation::LABEL_INDEX_DROP: + case StorageMetadataOperation::LABEL_INDEX_STATS_CLEAR: + case StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_CLEAR: + case StorageMetadataOperation::LABEL_INDEX_STATS_SET: + case StorageMetadataOperation::LABEL_PROPERTY_INDEX_CREATE: + case StorageMetadataOperation::LABEL_PROPERTY_INDEX_DROP: + case StorageMetadataOperation::EXISTENCE_CONSTRAINT_CREATE: + case StorageMetadataOperation::EXISTENCE_CONSTRAINT_DROP: + case StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_SET: + case StorageMetadataOperation::UNIQUE_CONSTRAINT_CREATE: + case StorageMetadataOperation::UNIQUE_CONSTRAINT_DROP: + MG_ASSERT(false, "Invalid function call!"); } } @@ -887,6 +942,18 @@ RecoveryInfo LoadWal(const std::filesystem::path &path, RecoveredIndicesAndConst "The label index doesn't exist!"); break; } + case WalDeltaData::Type::EDGE_INDEX_CREATE: { + auto edge_type_id = EdgeTypeId::FromUint(name_id_mapper->NameToId(delta.operation_edge_type.edge_type)); + AddRecoveredIndexConstraint(&indices_constraints->indices.edge, edge_type_id, + "The edge-type index already exists!"); + break; + } + case WalDeltaData::Type::EDGE_INDEX_DROP: { + auto edge_type_id = EdgeTypeId::FromUint(name_id_mapper->NameToId(delta.operation_edge_type.edge_type)); + RemoveRecoveredIndexConstraint(&indices_constraints->indices.edge, edge_type_id, + "The edge-type index doesn't exist!"); + break; + } case WalDeltaData::Type::LABEL_INDEX_STATS_SET: { auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_stats.label)); LabelIndexStats stats{}; @@ -1088,6 +1155,11 @@ void WalFile::AppendOperation(StorageMetadataOperation operation, LabelId label, UpdateStats(timestamp); } +void WalFile::AppendOperation(StorageMetadataOperation operation, EdgeTypeId edge_type, uint64_t timestamp) { + EncodeOperation(&wal_, name_id_mapper_, operation, edge_type, timestamp); + UpdateStats(timestamp); +} + void WalFile::Sync() { wal_.Sync(); } uint64_t WalFile::GetSize() { return wal_.GetSize(); } diff --git a/src/storage/v2/durability/wal.hpp b/src/storage/v2/durability/wal.hpp index 20d88b040..516487e0d 100644 --- a/src/storage/v2/durability/wal.hpp +++ b/src/storage/v2/durability/wal.hpp @@ -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 @@ -67,6 +67,8 @@ struct WalDeltaData { LABEL_PROPERTY_INDEX_DROP, LABEL_PROPERTY_INDEX_STATS_SET, LABEL_PROPERTY_INDEX_STATS_CLEAR, + EDGE_INDEX_CREATE, + EDGE_INDEX_DROP, EXISTENCE_CONSTRAINT_CREATE, EXISTENCE_CONSTRAINT_DROP, UNIQUE_CONSTRAINT_CREATE, @@ -111,6 +113,10 @@ struct WalDeltaData { std::set<std::string, std::less<>> properties; } operation_label_properties; + struct { + std::string edge_type; + } operation_edge_type; + struct { std::string label; std::string stats; @@ -155,6 +161,8 @@ constexpr bool IsWalDeltaDataTypeTransactionEndVersion15(const WalDeltaData::Typ case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP: case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET: case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR: + case WalDeltaData::Type::EDGE_INDEX_CREATE: + case WalDeltaData::Type::EDGE_INDEX_DROP: case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: case WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP: case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE: @@ -164,7 +172,7 @@ constexpr bool IsWalDeltaDataTypeTransactionEndVersion15(const WalDeltaData::Typ } constexpr bool IsWalDeltaDataTypeTransactionEnd(const WalDeltaData::Type type, const uint64_t version = kVersion) { - if (version < 16U) { + if (version < 17U) { return IsWalDeltaDataTypeTransactionEndVersion15(type); } // All deltas are now handled in a transactional scope @@ -208,6 +216,9 @@ void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, Storage LabelId label, const std::set<PropertyId> &properties, const LabelIndexStats &stats, const LabelPropertyIndexStats &property_stats, uint64_t timestamp); +void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageMetadataOperation operation, + EdgeTypeId edge_type, uint64_t timestamp); + /// Function used to load the WAL data into the storage. /// @throw RecoveryFailure RecoveryInfo LoadWal(const std::filesystem::path &path, RecoveredIndicesAndConstraints *indices_constraints, @@ -240,6 +251,8 @@ class WalFile { void AppendOperation(StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties, const LabelIndexStats &stats, const LabelPropertyIndexStats &property_stats, uint64_t timestamp); + void AppendOperation(StorageMetadataOperation operation, EdgeTypeId edge_type, uint64_t timestamp); + void Sync(); uint64_t GetSize(); diff --git a/src/storage/v2/edges_iterable.cpp b/src/storage/v2/edges_iterable.cpp new file mode 100644 index 000000000..6acae34e3 --- /dev/null +++ b/src/storage/v2/edges_iterable.cpp @@ -0,0 +1,149 @@ +// 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 +// 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 "storage/v2/edges_iterable.hpp" + +namespace memgraph::storage { + +EdgesIterable::EdgesIterable(InMemoryEdgeTypeIndex::Iterable edges) : type_(Type::BY_EDGE_TYPE_IN_MEMORY) { + new (&in_memory_edges_by_edge_type_) InMemoryEdgeTypeIndex::Iterable(std::move(edges)); +} + +EdgesIterable::EdgesIterable(EdgesIterable &&other) noexcept : type_(other.type_) { + switch (other.type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + new (&in_memory_edges_by_edge_type_) + InMemoryEdgeTypeIndex::Iterable(std::move(other.in_memory_edges_by_edge_type_)); + break; + } +} + +EdgesIterable &EdgesIterable::operator=(EdgesIterable &&other) noexcept { + Destroy(); + type_ = other.type_; + switch (other.type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + new (&in_memory_edges_by_edge_type_) + InMemoryEdgeTypeIndex::Iterable(std::move(other.in_memory_edges_by_edge_type_)); + break; + } + return *this; +} + +EdgesIterable::~EdgesIterable() { Destroy(); } + +void EdgesIterable::Destroy() noexcept { + switch (type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + in_memory_edges_by_edge_type_.InMemoryEdgeTypeIndex::Iterable::~Iterable(); + break; + } +} + +EdgesIterable::Iterator EdgesIterable::begin() { + switch (type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + return Iterator(in_memory_edges_by_edge_type_.begin()); + } +} + +EdgesIterable::Iterator EdgesIterable::end() { + switch (type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + return Iterator(in_memory_edges_by_edge_type_.end()); + } +} + +EdgesIterable::Iterator::Iterator(InMemoryEdgeTypeIndex::Iterable::Iterator it) : type_(Type::BY_EDGE_TYPE_IN_MEMORY) { + // NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg) + new (&in_memory_edges_by_edge_type_) InMemoryEdgeTypeIndex::Iterable::Iterator(std::move(it)); +} + +EdgesIterable::Iterator::Iterator(const EdgesIterable::Iterator &other) : type_(other.type_) { + switch (other.type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + new (&in_memory_edges_by_edge_type_) + InMemoryEdgeTypeIndex::Iterable::Iterator(other.in_memory_edges_by_edge_type_); + break; + } +} + +// NOLINTNEXTLINE(cert-oop54-cpp) +EdgesIterable::Iterator &EdgesIterable::Iterator::operator=(const EdgesIterable::Iterator &other) { + Destroy(); + type_ = other.type_; + switch (other.type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + new (&in_memory_edges_by_edge_type_) + InMemoryEdgeTypeIndex::Iterable::Iterator(other.in_memory_edges_by_edge_type_); + break; + } + return *this; +} + +EdgesIterable::Iterator::Iterator(EdgesIterable::Iterator &&other) noexcept : type_(other.type_) { + switch (other.type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + new (&in_memory_edges_by_edge_type_) + // NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg) + InMemoryEdgeTypeIndex::Iterable::Iterator(std::move(other.in_memory_edges_by_edge_type_)); + break; + } +} + +EdgesIterable::Iterator &EdgesIterable::Iterator::operator=(EdgesIterable::Iterator &&other) noexcept { + Destroy(); + type_ = other.type_; + switch (other.type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + new (&in_memory_edges_by_edge_type_) + // NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg) + InMemoryEdgeTypeIndex::Iterable::Iterator(std::move(other.in_memory_edges_by_edge_type_)); + break; + } + return *this; +} + +EdgesIterable::Iterator::~Iterator() { Destroy(); } + +void EdgesIterable::Iterator::Destroy() noexcept { + switch (type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + in_memory_edges_by_edge_type_.InMemoryEdgeTypeIndex::Iterable::Iterator::~Iterator(); + break; + } +} + +EdgeAccessor const &EdgesIterable::Iterator::operator*() const { + switch (type_) { + ; + case Type::BY_EDGE_TYPE_IN_MEMORY: + return *in_memory_edges_by_edge_type_; + } +} + +EdgesIterable::Iterator &EdgesIterable::Iterator::operator++() { + switch (type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + ++in_memory_edges_by_edge_type_; + break; + } + return *this; +} + +bool EdgesIterable::Iterator::operator==(const Iterator &other) const { + switch (type_) { + case Type::BY_EDGE_TYPE_IN_MEMORY: + return in_memory_edges_by_edge_type_ == other.in_memory_edges_by_edge_type_; + } +} + +} // namespace memgraph::storage diff --git a/src/storage/v2/edges_iterable.hpp b/src/storage/v2/edges_iterable.hpp new file mode 100644 index 000000000..9c9326705 --- /dev/null +++ b/src/storage/v2/edges_iterable.hpp @@ -0,0 +1,73 @@ +// 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 +// 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. + +#pragma once + +#include "storage/v2/all_vertices_iterable.hpp" +#include "storage/v2/inmemory/edge_type_index.hpp" + +namespace memgraph::storage { + +class InMemoryEdgeTypeIndex; + +class EdgesIterable final { + enum class Type { BY_EDGE_TYPE_IN_MEMORY }; + + Type type_; + union { + InMemoryEdgeTypeIndex::Iterable in_memory_edges_by_edge_type_; + }; + + void Destroy() noexcept; + + public: + explicit EdgesIterable(InMemoryEdgeTypeIndex::Iterable); + + EdgesIterable(const EdgesIterable &) = delete; + EdgesIterable &operator=(const EdgesIterable &) = delete; + + EdgesIterable(EdgesIterable &&) noexcept; + EdgesIterable &operator=(EdgesIterable &&) noexcept; + + ~EdgesIterable(); + + class Iterator final { + Type type_; + union { + InMemoryEdgeTypeIndex::Iterable::Iterator in_memory_edges_by_edge_type_; + }; + + void Destroy() noexcept; + + public: + explicit Iterator(InMemoryEdgeTypeIndex::Iterable::Iterator); + + Iterator(const Iterator &); + Iterator &operator=(const Iterator &); + + Iterator(Iterator &&) noexcept; + Iterator &operator=(Iterator &&) noexcept; + + ~Iterator(); + + EdgeAccessor const &operator*() const; + + Iterator &operator++(); + + bool operator==(const Iterator &other) const; + bool operator!=(const Iterator &other) const { return !(*this == other); } + }; + + Iterator begin(); + Iterator end(); +}; + +} // namespace memgraph::storage diff --git a/src/storage/v2/indices/edge_type_index.hpp b/src/storage/v2/indices/edge_type_index.hpp new file mode 100644 index 000000000..788ccb225 --- /dev/null +++ b/src/storage/v2/indices/edge_type_index.hpp @@ -0,0 +1,46 @@ +// 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 +// 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. + +#pragma once + +#include <vector> + +#include "storage/v2/transaction.hpp" + +namespace memgraph::storage { + +class EdgeTypeIndex { + public: + EdgeTypeIndex() = default; + + EdgeTypeIndex(const EdgeTypeIndex &) = delete; + EdgeTypeIndex(EdgeTypeIndex &&) = delete; + EdgeTypeIndex &operator=(const EdgeTypeIndex &) = delete; + EdgeTypeIndex &operator=(EdgeTypeIndex &&) = delete; + + virtual ~EdgeTypeIndex() = default; + + virtual bool DropIndex(EdgeTypeId edge_type) = 0; + + virtual bool IndexExists(EdgeTypeId edge_type) const = 0; + + virtual std::vector<EdgeTypeId> ListIndices() const = 0; + + virtual uint64_t ApproximateEdgeCount(EdgeTypeId edge_type) const = 0; + + virtual void UpdateOnEdgeCreation(Vertex *from, Vertex *to, EdgeRef edge_ref, EdgeTypeId edge_type, + const Transaction &tx) = 0; + + 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; +}; + +} // namespace memgraph::storage diff --git a/src/storage/v2/indices/indices.cpp b/src/storage/v2/indices/indices.cpp index c86ec8442..6068f888f 100644 --- a/src/storage/v2/indices/indices.cpp +++ b/src/storage/v2/indices/indices.cpp @@ -10,8 +10,10 @@ // licenses/APL.txt. #include "storage/v2/indices/indices.hpp" +#include "storage/v2/disk/edge_type_index.hpp" #include "storage/v2/disk/label_index.hpp" #include "storage/v2/disk/label_property_index.hpp" +#include "storage/v2/inmemory/edge_type_index.hpp" #include "storage/v2/inmemory/label_index.hpp" #include "storage/v2/inmemory/label_property_index.hpp" @@ -35,6 +37,8 @@ void Indices::AbortEntries(LabelId label, std::span<std::pair<PropertyValue, Ver void Indices::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp, std::stop_token token) const { static_cast<InMemoryLabelIndex *>(label_index_.get())->RemoveObsoleteEntries(oldest_active_start_timestamp, token); static_cast<InMemoryLabelPropertyIndex *>(label_property_index_.get()) + ->RemoveObsoleteEntries(oldest_active_start_timestamp, token); + static_cast<InMemoryEdgeTypeIndex *>(edge_type_index_.get()) ->RemoveObsoleteEntries(oldest_active_start_timestamp, std::move(token)); } @@ -53,14 +57,21 @@ void Indices::UpdateOnSetProperty(PropertyId property, const PropertyValue &valu label_property_index_->UpdateOnSetProperty(property, value, vertex, tx); } +void Indices::UpdateOnEdgeCreation(Vertex *from, Vertex *to, EdgeRef edge_ref, EdgeTypeId edge_type, + const Transaction &tx) const { + edge_type_index_->UpdateOnEdgeCreation(from, to, edge_ref, edge_type, tx); +} + Indices::Indices(const Config &config, StorageMode storage_mode) { std::invoke([this, config, storage_mode]() { if (storage_mode == StorageMode::IN_MEMORY_TRANSACTIONAL || storage_mode == StorageMode::IN_MEMORY_ANALYTICAL) { label_index_ = std::make_unique<InMemoryLabelIndex>(); label_property_index_ = std::make_unique<InMemoryLabelPropertyIndex>(); + edge_type_index_ = std::make_unique<InMemoryEdgeTypeIndex>(); } else { label_index_ = std::make_unique<DiskLabelIndex>(config); label_property_index_ = std::make_unique<DiskLabelPropertyIndex>(config); + edge_type_index_ = std::make_unique<DiskEdgeTypeIndex>(); } }); } diff --git a/src/storage/v2/indices/indices.hpp b/src/storage/v2/indices/indices.hpp index d95187bbb..40cff577f 100644 --- a/src/storage/v2/indices/indices.hpp +++ b/src/storage/v2/indices/indices.hpp @@ -15,6 +15,7 @@ #include <span> #include "storage/v2/id_types.hpp" +#include "storage/v2/indices/edge_type_index.hpp" #include "storage/v2/indices/label_index.hpp" #include "storage/v2/indices/label_property_index.hpp" #include "storage/v2/storage_mode.hpp" @@ -64,8 +65,12 @@ struct Indices { void UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex, const Transaction &tx) const; + void UpdateOnEdgeCreation(Vertex *from, Vertex *to, EdgeRef edge_ref, EdgeTypeId edge_type, + const Transaction &tx) const; + std::unique_ptr<LabelIndex> label_index_; std::unique_ptr<LabelPropertyIndex> label_property_index_; + std::unique_ptr<EdgeTypeIndex> edge_type_index_; }; } // namespace memgraph::storage diff --git a/src/storage/v2/inmemory/edge_type_index.cpp b/src/storage/v2/inmemory/edge_type_index.cpp new file mode 100644 index 000000000..e439628b4 --- /dev/null +++ b/src/storage/v2/inmemory/edge_type_index.cpp @@ -0,0 +1,318 @@ +// 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 +// 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 "storage/v2/inmemory/edge_type_index.hpp" + +#include "storage/v2/constraints/constraints.hpp" +#include "storage/v2/indices/indices_utils.hpp" +#include "utils/counter.hpp" + +namespace { + +using Delta = memgraph::storage::Delta; +using Vertex = memgraph::storage::Vertex; +using Edge = memgraph::storage::Edge; +using EdgeRef = memgraph::storage::EdgeRef; +using EdgeTypeId = memgraph::storage::EdgeTypeId; +using Transaction = memgraph::storage::Transaction; +using View = memgraph::storage::View; + +bool IsIndexEntryVisible(Edge *edge, const Transaction *transaction, View view) { + bool exists = true; + bool deleted = true; + Delta *delta = nullptr; + { + auto guard = std::shared_lock{edge->lock}; + deleted = edge->deleted; + delta = edge->delta; + } + ApplyDeltasForRead(transaction, delta, view, [&](const Delta &delta) { + switch (delta.action) { + case Delta::Action::ADD_LABEL: + case Delta::Action::REMOVE_LABEL: + case Delta::Action::SET_PROPERTY: + case Delta::Action::ADD_IN_EDGE: + case Delta::Action::ADD_OUT_EDGE: + case Delta::Action::REMOVE_IN_EDGE: + case Delta::Action::REMOVE_OUT_EDGE: + break; + case Delta::Action::RECREATE_OBJECT: { + deleted = false; + break; + } + case Delta::Action::DELETE_DESERIALIZED_OBJECT: + case Delta::Action::DELETE_OBJECT: { + exists = false; + break; + } + } + }); + return exists && !deleted; +} + +using ReturnType = std::optional<std::tuple<EdgeTypeId, Vertex *, EdgeRef>>; +ReturnType VertexDeletedConnectedEdges(Vertex *vertex, Edge *edge, const Transaction *transaction, View view) { + ReturnType link; + Delta *delta = nullptr; + { + auto guard = std::shared_lock{vertex->lock}; + delta = vertex->delta; + } + ApplyDeltasForRead(transaction, delta, view, [&](const Delta &delta) { + switch (delta.action) { + case Delta::Action::ADD_LABEL: + case Delta::Action::REMOVE_LABEL: + case Delta::Action::SET_PROPERTY: + break; + case Delta::Action::ADD_IN_EDGE: { + if (edge == delta.vertex_edge.edge.ptr) { + link = {delta.vertex_edge.edge_type, delta.vertex_edge.vertex, delta.vertex_edge.edge}; + auto it = std::find(vertex->in_edges.begin(), vertex->in_edges.end(), link); + MG_ASSERT(it == vertex->in_edges.end(), "Invalid database state!"); + break; + } + } + case Delta::Action::ADD_OUT_EDGE: { + if (edge == delta.vertex_edge.edge.ptr) { + link = {delta.vertex_edge.edge_type, delta.vertex_edge.vertex, delta.vertex_edge.edge}; + auto it = std::find(vertex->out_edges.begin(), vertex->out_edges.end(), link); + MG_ASSERT(it == vertex->out_edges.end(), "Invalid database state!"); + break; + } + } + case Delta::Action::REMOVE_IN_EDGE: + case Delta::Action::REMOVE_OUT_EDGE: + case Delta::Action::RECREATE_OBJECT: + case Delta::Action::DELETE_DESERIALIZED_OBJECT: + case Delta::Action::DELETE_OBJECT: + break; + } + }); + return link; +} + +} // namespace + +namespace memgraph::storage { + +bool InMemoryEdgeTypeIndex::CreateIndex(EdgeTypeId edge_type, utils::SkipList<Vertex>::Accessor vertices) { + auto [it, emplaced] = index_.try_emplace(edge_type); + if (!emplaced) { + return false; + } + + utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception; + try { + auto edge_acc = it->second.access(); + for (auto &from_vertex : vertices) { + if (from_vertex.deleted) { + continue; + } + + for (auto &edge : from_vertex.out_edges) { + const auto type = std::get<kEdgeTypeIdPos>(edge); + if (type == edge_type) { + auto *to_vertex = std::get<kVertexPos>(edge); + if (to_vertex->deleted) { + continue; + } + edge_acc.insert({&from_vertex, to_vertex, std::get<kEdgeRefPos>(edge).ptr, 0}); + } + } + } + } catch (const utils::OutOfMemoryException &) { + utils::MemoryTracker::OutOfMemoryExceptionBlocker oom_exception_blocker; + index_.erase(it); + throw; + } + + return true; +} + +bool InMemoryEdgeTypeIndex::DropIndex(EdgeTypeId edge_type) { return index_.erase(edge_type) > 0; } + +bool InMemoryEdgeTypeIndex::IndexExists(EdgeTypeId edge_type) const { return index_.find(edge_type) != index_.end(); } + +std::vector<EdgeTypeId> InMemoryEdgeTypeIndex::ListIndices() const { + std::vector<EdgeTypeId> ret; + ret.reserve(index_.size()); + for (const auto &item : index_) { + ret.push_back(item.first); + } + return ret; +} + +void InMemoryEdgeTypeIndex::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp, std::stop_token token) { + auto maybe_stop = utils::ResettableCounter<2048>(); + + for (auto &label_storage : index_) { + if (token.stop_requested()) return; + + auto edges_acc = label_storage.second.access(); + for (auto it = edges_acc.begin(); it != edges_acc.end();) { + if (maybe_stop() && token.stop_requested()) return; + + auto next_it = it; + ++next_it; + + if (it->timestamp >= oldest_active_start_timestamp) { + it = next_it; + continue; + } + + if (next_it != edges_acc.end() || it->from_vertex->deleted || it->to_vertex->deleted || + !std::ranges::all_of(it->from_vertex->out_edges, [&](const auto &edge) { + auto *to_vertex = std::get<InMemoryEdgeTypeIndex::kVertexPos>(edge); + return to_vertex != it->to_vertex; + })) { + edges_acc.remove(*it); + } + + it = next_it; + } + } +} + +uint64_t InMemoryEdgeTypeIndex::ApproximateEdgeCount(EdgeTypeId edge_type) const { + if (auto it = index_.find(edge_type); it != index_.end()) { + return it->second.size(); + } + return 0; +} + +void InMemoryEdgeTypeIndex::UpdateOnEdgeCreation(Vertex *from, Vertex *to, EdgeRef edge_ref, EdgeTypeId edge_type, + const Transaction &tx) { + auto it = index_.find(edge_type); + if (it == index_.end()) { + return; + } + auto acc = it->second.access(); + acc.insert(Entry{from, to, edge_ref.ptr, tx.start_timestamp}); +} + +void InMemoryEdgeTypeIndex::UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, + EdgeRef edge_ref, EdgeTypeId edge_type, const Transaction &tx) { + auto it = index_.find(edge_type); + if (it == index_.end()) { + return; + } + auto acc = it->second.access(); + + auto entry_to_update = std::ranges::find_if(acc, [&](const auto &entry) { + return entry.from_vertex == old_from && entry.to_vertex == old_to && entry.edge == edge_ref.ptr; + }); + + acc.remove(Entry{entry_to_update->from_vertex, entry_to_update->to_vertex, entry_to_update->edge, + entry_to_update->timestamp}); + acc.insert(Entry{new_from, new_to, edge_ref.ptr, tx.start_timestamp}); +} + +InMemoryEdgeTypeIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, EdgeTypeId edge_type, + View view, Storage *storage, Transaction *transaction) + : index_accessor_(std::move(index_accessor)), + edge_type_(edge_type), + view_(view), + storage_(storage), + transaction_(transaction) {} + +InMemoryEdgeTypeIndex::Iterable::Iterator::Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator) + : self_(self), + index_iterator_(index_iterator), + current_edge_accessor_(EdgeRef{nullptr}, EdgeTypeId::FromInt(0), nullptr, nullptr, self_->storage_, nullptr), + current_edge_(nullptr) { + AdvanceUntilValid(); +} + +InMemoryEdgeTypeIndex::Iterable::Iterator &InMemoryEdgeTypeIndex::Iterable::Iterator::operator++() { + ++index_iterator_; + AdvanceUntilValid(); + return *this; +} + +void InMemoryEdgeTypeIndex::Iterable::Iterator::AdvanceUntilValid() { + for (; index_iterator_ != self_->index_accessor_.end(); ++index_iterator_) { + auto *from_vertex = index_iterator_->from_vertex; + auto *to_vertex = index_iterator_->to_vertex; + + if (!IsIndexEntryVisible(index_iterator_->edge, self_->transaction_, self_->view_) || from_vertex->deleted || + to_vertex->deleted) { + continue; + } + + const bool edge_was_deleted = index_iterator_->edge->deleted; + auto [edge_ref, edge_type, deleted_from_vertex, deleted_to_vertex] = GetEdgeInfo(); + MG_ASSERT(edge_ref != EdgeRef(nullptr), "Invalid database state!"); + + if (edge_was_deleted) { + from_vertex = deleted_from_vertex; + to_vertex = deleted_to_vertex; + } + + auto accessor = EdgeAccessor{edge_ref, edge_type, from_vertex, to_vertex, self_->storage_, self_->transaction_}; + if (!accessor.IsVisible(self_->view_)) { + continue; + } + + current_edge_accessor_ = accessor; + current_edge_ = edge_ref; + break; + } +} + +std::tuple<EdgeRef, EdgeTypeId, Vertex *, Vertex *> InMemoryEdgeTypeIndex::Iterable::Iterator::GetEdgeInfo() { + auto *from_vertex = index_iterator_->from_vertex; + auto *to_vertex = index_iterator_->to_vertex; + + if (index_iterator_->edge->deleted) { + const auto missing_in_edge = + VertexDeletedConnectedEdges(from_vertex, index_iterator_->edge, self_->transaction_, self_->view_); + const auto missing_out_edge = + VertexDeletedConnectedEdges(to_vertex, index_iterator_->edge, self_->transaction_, self_->view_); + if (missing_in_edge && missing_out_edge && + std::get<kEdgeRefPos>(*missing_in_edge) == std::get<kEdgeRefPos>(*missing_out_edge)) { + return std::make_tuple(std::get<kEdgeRefPos>(*missing_in_edge), std::get<kEdgeTypeIdPos>(*missing_in_edge), + to_vertex, from_vertex); + } + } + + const auto &from_edges = from_vertex->out_edges; + const auto &to_edges = to_vertex->in_edges; + + auto it = std::find_if(from_edges.begin(), from_edges.end(), [&](const auto &from_entry) { + const auto &from_edge = std::get<kEdgeRefPos>(from_entry); + return std::any_of(to_edges.begin(), to_edges.end(), [&](const auto &to_entry) { + const auto &to_edge = std::get<kEdgeRefPos>(to_entry); + return index_iterator_->edge->gid == from_edge.ptr->gid && from_edge.ptr->gid == to_edge.ptr->gid; + }); + }); + + if (it != from_edges.end()) { + const auto &from_edge = std::get<kEdgeRefPos>(*it); + return std::make_tuple(from_edge, std::get<kEdgeTypeIdPos>(*it), from_vertex, to_vertex); + } + + return {EdgeRef(nullptr), EdgeTypeId::FromUint(0U), nullptr, nullptr}; +} + +void InMemoryEdgeTypeIndex::RunGC() { + for (auto &index_entry : index_) { + index_entry.second.run_gc(); + } +} + +InMemoryEdgeTypeIndex::Iterable InMemoryEdgeTypeIndex::Edges(EdgeTypeId edge_type, View view, Storage *storage, + Transaction *transaction) { + const auto it = index_.find(edge_type); + MG_ASSERT(it != index_.end(), "Index for edge-type {} doesn't exist", edge_type.AsUint()); + return {it->second.access(), edge_type, view, storage, transaction}; +} + +} // namespace memgraph::storage diff --git a/src/storage/v2/inmemory/edge_type_index.hpp b/src/storage/v2/inmemory/edge_type_index.hpp new file mode 100644 index 000000000..db8f7843f --- /dev/null +++ b/src/storage/v2/inmemory/edge_type_index.hpp @@ -0,0 +1,113 @@ +// 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 +// 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. + +#pragma once + +#include <map> +#include <utility> + +#include "storage/v2/constraints/constraints.hpp" +#include "storage/v2/edge_accessor.hpp" +#include "storage/v2/id_types.hpp" +#include "storage/v2/indices/edge_type_index.hpp" +#include "storage/v2/indices/label_index_stats.hpp" +#include "utils/rw_lock.hpp" +#include "utils/synchronized.hpp" + +namespace memgraph::storage { + +class InMemoryEdgeTypeIndex : public storage::EdgeTypeIndex { + private: + struct Entry { + Vertex *from_vertex; + Vertex *to_vertex; + + Edge *edge; + + uint64_t timestamp; + + bool operator<(const Entry &rhs) const { return edge->gid < rhs.edge->gid; } + bool operator==(const Entry &rhs) const { return edge->gid == rhs.edge->gid; } + }; + + public: + InMemoryEdgeTypeIndex() = default; + + /// @throw std::bad_alloc + bool CreateIndex(EdgeTypeId edge_type, utils::SkipList<Vertex>::Accessor vertices); + + /// Returns false if there was no index to drop + bool DropIndex(EdgeTypeId edge_type) override; + + bool IndexExists(EdgeTypeId edge_type) const override; + + std::vector<EdgeTypeId> ListIndices() const override; + + void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp, std::stop_token token); + + uint64_t ApproximateEdgeCount(EdgeTypeId edge_type) const override; + + void UpdateOnEdgeCreation(Vertex *from, Vertex *to, EdgeRef edge_ref, EdgeTypeId edge_type, + const Transaction &tx) override; + + void UpdateOnEdgeModification(Vertex *old_from, Vertex *old_to, Vertex *new_from, Vertex *new_to, EdgeRef edge_ref, + EdgeTypeId edge_type, const Transaction &tx) override; + + static constexpr std::size_t kEdgeTypeIdPos = 0U; + static constexpr std::size_t kVertexPos = 1U; + static constexpr std::size_t kEdgeRefPos = 2U; + + class Iterable { + public: + Iterable(utils::SkipList<Entry>::Accessor index_accessor, EdgeTypeId edge_type, View view, Storage *storage, + Transaction *transaction); + + class Iterator { + public: + Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator); + + EdgeAccessor const &operator*() const { return current_edge_accessor_; } + + bool operator==(const Iterator &other) const { return index_iterator_ == other.index_iterator_; } + bool operator!=(const Iterator &other) const { return index_iterator_ != other.index_iterator_; } + + Iterator &operator++(); + + private: + void AdvanceUntilValid(); + std::tuple<EdgeRef, EdgeTypeId, Vertex *, Vertex *> GetEdgeInfo(); + + Iterable *self_; + utils::SkipList<Entry>::Iterator index_iterator_; + EdgeAccessor current_edge_accessor_; + EdgeRef current_edge_{nullptr}; + }; + + Iterator begin() { return {this, index_accessor_.begin()}; } + Iterator end() { return {this, index_accessor_.end()}; } + + private: + utils::SkipList<Entry>::Accessor index_accessor_; + EdgeTypeId edge_type_; + View view_; + Storage *storage_; + Transaction *transaction_; + }; + + void RunGC(); + + Iterable Edges(EdgeTypeId edge_type, View view, Storage *storage, Transaction *transaction); + + private: + std::map<EdgeTypeId, utils::SkipList<Entry>> index_; +}; + +} // namespace memgraph::storage diff --git a/src/storage/v2/inmemory/storage.cpp b/src/storage/v2/inmemory/storage.cpp index 3a4fa9b91..1ea909450 100644 --- a/src/storage/v2/inmemory/storage.cpp +++ b/src/storage/v2/inmemory/storage.cpp @@ -20,6 +20,7 @@ #include "storage/v2/durability/snapshot.hpp" #include "storage/v2/edge_direction.hpp" #include "storage/v2/id_types.hpp" +#include "storage/v2/inmemory/edge_type_index.hpp" #include "storage/v2/metadata_delta.hpp" /// REPLICATION /// @@ -350,6 +351,9 @@ Result<EdgeAccessor> InMemoryStorage::InMemoryAccessor::CreateEdge(VertexAccesso transaction_.manyDeltasCache.Invalidate(from_vertex, edge_type, EdgeDirection::OUT); transaction_.manyDeltasCache.Invalidate(to_vertex, edge_type, EdgeDirection::IN); + // Update indices if they exist. + storage_->indices_.UpdateOnEdgeCreation(from_vertex, to_vertex, edge, edge_type, transaction_); + // Increment edge count. storage_->edge_count_.fetch_add(1, std::memory_order_acq_rel); }}; @@ -553,6 +557,11 @@ Result<EdgeAccessor> InMemoryStorage::InMemoryAccessor::EdgeSetFrom(EdgeAccessor CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, new_from_vertex, edge_ref); to_vertex->in_edges.emplace_back(edge_type, new_from_vertex, edge_ref); + auto *in_memory = static_cast<InMemoryStorage *>(storage_); + auto *mem_edge_type_index = static_cast<InMemoryEdgeTypeIndex *>(in_memory->indices_.edge_type_index_.get()); + mem_edge_type_index->UpdateOnEdgeModification(old_from_vertex, to_vertex, new_from_vertex, to_vertex, edge_ref, + edge_type, transaction_); + transaction_.manyDeltasCache.Invalidate(new_from_vertex, edge_type, EdgeDirection::OUT); transaction_.manyDeltasCache.Invalidate(old_from_vertex, edge_type, EdgeDirection::OUT); transaction_.manyDeltasCache.Invalidate(to_vertex, edge_type, EdgeDirection::IN); @@ -659,6 +668,11 @@ Result<EdgeAccessor> InMemoryStorage::InMemoryAccessor::EdgeSetTo(EdgeAccessor * CreateAndLinkDelta(&transaction_, new_to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex, edge_ref); new_to_vertex->in_edges.emplace_back(edge_type, from_vertex, edge_ref); + auto *in_memory = static_cast<InMemoryStorage *>(storage_); + auto *mem_edge_type_index = static_cast<InMemoryEdgeTypeIndex *>(in_memory->indices_.edge_type_index_.get()); + mem_edge_type_index->UpdateOnEdgeModification(from_vertex, old_to_vertex, from_vertex, new_to_vertex, edge_ref, + edge_type, transaction_); + transaction_.manyDeltasCache.Invalidate(from_vertex, edge_type, EdgeDirection::OUT); transaction_.manyDeltasCache.Invalidate(old_to_vertex, edge_type, EdgeDirection::IN); transaction_.manyDeltasCache.Invalidate(new_to_vertex, edge_type, EdgeDirection::IN); @@ -1264,6 +1278,18 @@ utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryA return {}; } +utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryAccessor::CreateIndex( + EdgeTypeId edge_type) { + MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!"); + auto *in_memory = static_cast<InMemoryStorage *>(storage_); + auto *mem_edge_type_index = static_cast<InMemoryEdgeTypeIndex *>(in_memory->indices_.edge_type_index_.get()); + if (!mem_edge_type_index->CreateIndex(edge_type, in_memory->vertices_.access())) { + return StorageIndexDefinitionError{IndexDefinitionError{}}; + } + transaction_.md_deltas.emplace_back(MetadataDelta::edge_index_create, edge_type); + return {}; +} + utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryAccessor::DropIndex(LabelId label) { MG_ASSERT(unique_guard_.owns_lock(), "Dropping label index requires a unique access to the storage!"); auto *in_memory = static_cast<InMemoryStorage *>(storage_); @@ -1292,6 +1318,18 @@ utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryA return {}; } +utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryAccessor::DropIndex( + EdgeTypeId edge_type) { + MG_ASSERT(unique_guard_.owns_lock(), "Drop index requires a unique access to the storage!"); + auto *in_memory = static_cast<InMemoryStorage *>(storage_); + auto *mem_edge_type_index = static_cast<InMemoryEdgeTypeIndex *>(in_memory->indices_.edge_type_index_.get()); + if (!mem_edge_type_index->DropIndex(edge_type)) { + return StorageIndexDefinitionError{IndexDefinitionError{}}; + } + transaction_.md_deltas.emplace_back(MetadataDelta::edge_index_drop, edge_type); + return {}; +} + utils::BasicResult<StorageExistenceConstraintDefinitionError, void> InMemoryStorage::InMemoryAccessor::CreateExistenceConstraint(LabelId label, PropertyId property) { MG_ASSERT(unique_guard_.owns_lock(), "Creating existence requires a unique access to the storage!"); @@ -1383,6 +1421,11 @@ VerticesIterable InMemoryStorage::InMemoryAccessor::Vertices( mem_label_property_index->Vertices(label, property, lower_bound, upper_bound, view, storage_, &transaction_)); } +EdgesIterable InMemoryStorage::InMemoryAccessor::Edges(EdgeTypeId edge_type, View view) { + auto *mem_edge_type_index = static_cast<InMemoryEdgeTypeIndex *>(storage_->indices_.edge_type_index_.get()); + return EdgesIterable(mem_edge_type_index->Edges(edge_type, view, storage_, &transaction_)); +} + Transaction InMemoryStorage::CreateTransaction( IsolationLevel isolation_level, StorageMode storage_mode, memgraph::replication_coordination_glue::ReplicationRole replication_role) { @@ -2017,6 +2060,10 @@ bool InMemoryStorage::AppendToWal(const Transaction &transaction, uint64_t final AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_INDEX_CREATE, md_delta.label, final_commit_timestamp); } break; + case MetadataDelta::Action::EDGE_INDEX_CREATE: { + AppendToWalDataDefinition(durability::StorageMetadataOperation::EDGE_TYPE_INDEX_CREATE, md_delta.edge_type, + final_commit_timestamp); + } break; case MetadataDelta::Action::LABEL_PROPERTY_INDEX_CREATE: { const auto &info = md_delta.label_property; AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_CREATE, info.label, @@ -2026,6 +2073,10 @@ bool InMemoryStorage::AppendToWal(const Transaction &transaction, uint64_t final AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_INDEX_DROP, md_delta.label, final_commit_timestamp); } break; + case MetadataDelta::Action::EDGE_INDEX_DROP: { + AppendToWalDataDefinition(durability::StorageMetadataOperation::EDGE_TYPE_INDEX_DROP, md_delta.edge_type, + final_commit_timestamp); + } break; case MetadataDelta::Action::LABEL_PROPERTY_INDEX_DROP: { const auto &info = md_delta.label_property; AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_DROP, info.label, @@ -2091,6 +2142,12 @@ void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOpera repl_storage_state_.AppendOperation(operation, label, properties, stats, property_stats, final_commit_timestamp); } +void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOperation operation, EdgeTypeId edge_type, + uint64_t final_commit_timestamp) { + wal_file_->AppendOperation(operation, edge_type, final_commit_timestamp); + repl_storage_state_.AppendOperation(operation, edge_type, final_commit_timestamp); +} + void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties, LabelPropertyIndexStats property_stats, @@ -2240,7 +2297,8 @@ IndicesInfo InMemoryStorage::InMemoryAccessor::ListAllIndices() const { auto *mem_label_index = static_cast<InMemoryLabelIndex *>(in_memory->indices_.label_index_.get()); auto *mem_label_property_index = static_cast<InMemoryLabelPropertyIndex *>(in_memory->indices_.label_property_index_.get()); - return {mem_label_index->ListIndices(), mem_label_property_index->ListIndices()}; + auto *mem_edge_type_index = static_cast<InMemoryEdgeTypeIndex *>(in_memory->indices_.edge_type_index_.get()); + return {mem_label_index->ListIndices(), mem_label_property_index->ListIndices(), mem_edge_type_index->ListIndices()}; } ConstraintsInfo InMemoryStorage::InMemoryAccessor::ListAllConstraints() const { const auto *mem_storage = static_cast<InMemoryStorage *>(storage_); diff --git a/src/storage/v2/inmemory/storage.hpp b/src/storage/v2/inmemory/storage.hpp index c0e46d0c9..6d10e0fbd 100644 --- a/src/storage/v2/inmemory/storage.hpp +++ b/src/storage/v2/inmemory/storage.hpp @@ -16,6 +16,7 @@ #include <memory> #include <utility> #include "storage/v2/indices/label_index_stats.hpp" +#include "storage/v2/inmemory/edge_type_index.hpp" #include "storage/v2/inmemory/label_index.hpp" #include "storage/v2/inmemory/label_property_index.hpp" #include "storage/v2/inmemory/replication/recovery.hpp" @@ -53,6 +54,7 @@ class InMemoryStorage final : public Storage { const InMemoryStorage *storage); friend class InMemoryLabelIndex; friend class InMemoryLabelPropertyIndex; + friend class InMemoryEdgeTypeIndex; public: enum class CreateSnapshotError : uint8_t { DisabledForReplica, ReachedMaxNumTries }; @@ -107,6 +109,8 @@ class InMemoryStorage final : public Storage { const std::optional<utils::Bound<PropertyValue>> &lower_bound, const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) override; + EdgesIterable Edges(EdgeTypeId edge_type, View view) override; + /// Return approximate number of all vertices in the database. /// Note that this is always an over-estimate and never an under-estimate. uint64_t ApproximateVertexCount() const override { @@ -145,6 +149,10 @@ class InMemoryStorage final : public Storage { label, property, lower, upper); } + uint64_t ApproximateEdgeCount(EdgeTypeId id) const override { + return static_cast<InMemoryStorage *>(storage_)->indices_.edge_type_index_->ApproximateEdgeCount(id); + } + template <typename TResult, typename TIndex, typename TIndexKey> std::optional<TResult> GetIndexStatsForIndex(TIndex *index, TIndexKey &&key) const { return index->GetIndexStats(key); @@ -204,6 +212,10 @@ class InMemoryStorage final : public Storage { return static_cast<InMemoryStorage *>(storage_)->indices_.label_property_index_->IndexExists(label, property); } + bool EdgeTypeIndexExists(EdgeTypeId edge_type) const override { + return static_cast<InMemoryStorage *>(storage_)->indices_.edge_type_index_->IndexExists(edge_type); + } + IndicesInfo ListAllIndices() const override; ConstraintsInfo ListAllConstraints() const override; @@ -239,6 +251,14 @@ class InMemoryStorage final : public Storage { /// @throw std::bad_alloc utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) override; + /// Create an index. + /// Returns void if the index has been created. + /// Returns `StorageIndexDefinitionError` if an error occures. Error can be: + /// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction. + /// * `IndexDefinitionError`: the index already exists. + /// @throw std::bad_alloc + utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(EdgeTypeId edge_type) override; + /// Drop an existing index. /// Returns void if the index has been dropped. /// Returns `StorageIndexDefinitionError` if an error occures. Error can be: @@ -253,6 +273,13 @@ class InMemoryStorage final : public Storage { /// * `IndexDefinitionError`: the index does not exist. utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) override; + /// Drop an existing index. + /// Returns void if the index has been dropped. + /// Returns `StorageIndexDefinitionError` if an error occures. Error can be: + /// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction. + /// * `IndexDefinitionError`: the index does not exist. + utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(EdgeTypeId edge_type) override; + /// Returns void if the existence constraint has been created. /// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be: /// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction. @@ -374,20 +401,17 @@ class InMemoryStorage final : public Storage { /// Return true in all cases excepted if any sync replicas have not sent confirmation. [[nodiscard]] bool AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp, DatabaseAccessProtector db_acc); - /// Return true in all cases excepted if any sync replicas have not sent confirmation. void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label, uint64_t final_commit_timestamp); - /// Return true in all cases excepted if any sync replicas have not sent confirmation. + void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, EdgeTypeId edge_type, + uint64_t final_commit_timestamp); void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties, uint64_t final_commit_timestamp); - /// Return true in all cases excepted if any sync replicas have not sent confirmation. void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label, LabelIndexStats stats, uint64_t final_commit_timestamp); - /// Return true in all cases excepted if any sync replicas have not sent confirmation. void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties, LabelPropertyIndexStats property_stats, uint64_t final_commit_timestamp); - /// Return true in all cases excepted if any sync replicas have not sent confirmation. void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties, LabelIndexStats stats, LabelPropertyIndexStats property_stats, uint64_t final_commit_timestamp); diff --git a/src/storage/v2/metadata_delta.hpp b/src/storage/v2/metadata_delta.hpp index 94d806c19..b34966a62 100644 --- a/src/storage/v2/metadata_delta.hpp +++ b/src/storage/v2/metadata_delta.hpp @@ -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 @@ -35,6 +35,8 @@ struct MetadataDelta { LABEL_PROPERTY_INDEX_DROP, LABEL_PROPERTY_INDEX_STATS_SET, LABEL_PROPERTY_INDEX_STATS_CLEAR, + EDGE_INDEX_CREATE, + EDGE_INDEX_DROP, EXISTENCE_CONSTRAINT_CREATE, EXISTENCE_CONSTRAINT_DROP, UNIQUE_CONSTRAINT_CREATE, @@ -57,6 +59,10 @@ struct MetadataDelta { } label_property_index_stats_set; static constexpr struct LabelPropertyIndexStatsClear { } label_property_index_stats_clear; + static constexpr struct EdgeIndexCreate { + } edge_index_create; + static constexpr struct EdgeIndexDrop { + } edge_index_drop; static constexpr struct ExistenceConstraintCreate { } existence_constraint_create; static constexpr struct ExistenceConstraintDrop { @@ -87,6 +93,11 @@ struct MetadataDelta { MetadataDelta(LabelPropertyIndexStatsClear /*tag*/, LabelId label) : action(Action::LABEL_PROPERTY_INDEX_STATS_CLEAR), label{label} {} + MetadataDelta(EdgeIndexCreate /*tag*/, EdgeTypeId edge_type) + : action(Action::EDGE_INDEX_CREATE), edge_type(edge_type) {} + + MetadataDelta(EdgeIndexDrop /*tag*/, EdgeTypeId edge_type) : action(Action::EDGE_INDEX_DROP), edge_type(edge_type) {} + MetadataDelta(ExistenceConstraintCreate /*tag*/, LabelId label, PropertyId property) : action(Action::EXISTENCE_CONSTRAINT_CREATE), label_property{label, property} {} @@ -114,6 +125,8 @@ struct MetadataDelta { case Action::LABEL_PROPERTY_INDEX_DROP: case Action::LABEL_PROPERTY_INDEX_STATS_SET: case Action::LABEL_PROPERTY_INDEX_STATS_CLEAR: + case Action::EDGE_INDEX_CREATE: + case Action::EDGE_INDEX_DROP: case Action::EXISTENCE_CONSTRAINT_CREATE: case Action::EXISTENCE_CONSTRAINT_DROP: break; @@ -129,6 +142,8 @@ struct MetadataDelta { union { LabelId label; + EdgeTypeId edge_type; + struct { LabelId label; PropertyId property; diff --git a/src/storage/v2/replication/replication_client.cpp b/src/storage/v2/replication/replication_client.cpp index fb332672a..3c1081206 100644 --- a/src/storage/v2/replication/replication_client.cpp +++ b/src/storage/v2/replication/replication_client.cpp @@ -407,6 +407,12 @@ void ReplicaStream::AppendOperation(durability::StorageMetadataOperation operati timestamp); } +void ReplicaStream::AppendOperation(durability::StorageMetadataOperation operation, EdgeTypeId edge_type, + uint64_t timestamp) { + replication::Encoder encoder(stream_.GetBuilder()); + EncodeOperation(&encoder, storage_->name_id_mapper_.get(), operation, edge_type, timestamp); +} + replication::AppendDeltasRes ReplicaStream::Finalize() { return stream_.AwaitResponse(); } } // namespace memgraph::storage diff --git a/src/storage/v2/replication/replication_client.hpp b/src/storage/v2/replication/replication_client.hpp index 063501111..77a9ba40b 100644 --- a/src/storage/v2/replication/replication_client.hpp +++ b/src/storage/v2/replication/replication_client.hpp @@ -65,6 +65,9 @@ class ReplicaStream { const std::set<PropertyId> &properties, const LabelIndexStats &stats, const LabelPropertyIndexStats &property_stats, uint64_t timestamp); + /// @throw rpc::RpcFailedException + void AppendOperation(durability::StorageMetadataOperation operation, EdgeTypeId edge_type, uint64_t timestamp); + /// @throw rpc::RpcFailedException replication::AppendDeltasRes Finalize(); diff --git a/src/storage/v2/replication/replication_storage_state.cpp b/src/storage/v2/replication/replication_storage_state.cpp index 25cf484c9..b8f3fef62 100644 --- a/src/storage/v2/replication/replication_storage_state.cpp +++ b/src/storage/v2/replication/replication_storage_state.cpp @@ -53,6 +53,16 @@ void ReplicationStorageState::AppendOperation(durability::StorageMetadataOperati }); } +void ReplicationStorageState::AppendOperation(durability::StorageMetadataOperation operation, EdgeTypeId edge_type, + uint64_t final_commit_timestamp) { + replication_clients_.WithLock([&](auto &clients) { + for (auto &client : clients) { + client->IfStreamingTransaction( + [&](auto &stream) { stream.AppendOperation(operation, edge_type, final_commit_timestamp); }); + } + }); +} + bool ReplicationStorageState::FinalizeTransaction(uint64_t timestamp, Storage *storage, DatabaseAccessProtector db_acc) { return replication_clients_.WithLock([=, db_acc = std::move(db_acc)](auto &clients) mutable { diff --git a/src/storage/v2/replication/replication_storage_state.hpp b/src/storage/v2/replication/replication_storage_state.hpp index 91cec563c..f99807c13 100644 --- a/src/storage/v2/replication/replication_storage_state.hpp +++ b/src/storage/v2/replication/replication_storage_state.hpp @@ -46,6 +46,8 @@ struct ReplicationStorageState { void AppendOperation(durability::StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties, const LabelIndexStats &stats, const LabelPropertyIndexStats &property_stats, uint64_t final_commit_timestamp); + void AppendOperation(durability::StorageMetadataOperation operation, EdgeTypeId edge_type, + uint64_t final_commit_timestamp); bool FinalizeTransaction(uint64_t timestamp, Storage *storage, DatabaseAccessProtector db_acc); // Getters diff --git a/src/storage/v2/storage.hpp b/src/storage/v2/storage.hpp index 5868d70a3..58936bd56 100644 --- a/src/storage/v2/storage.hpp +++ b/src/storage/v2/storage.hpp @@ -30,6 +30,7 @@ #include "storage/v2/durability/paths.hpp" #include "storage/v2/durability/wal.hpp" #include "storage/v2/edge_accessor.hpp" +#include "storage/v2/edges_iterable.hpp" #include "storage/v2/indices/indices.hpp" #include "storage/v2/mvcc.hpp" #include "storage/v2/replication/enums.hpp" @@ -61,6 +62,7 @@ class EdgeAccessor; struct IndicesInfo { std::vector<LabelId> label; std::vector<std::pair<LabelId, PropertyId>> label_property; + std::vector<EdgeTypeId> edge_type; }; struct ConstraintsInfo { @@ -172,6 +174,8 @@ class Storage { const std::optional<utils::Bound<PropertyValue>> &lower_bound, const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) = 0; + virtual EdgesIterable Edges(EdgeTypeId edge_type, View view) = 0; + virtual Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex); virtual Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex( @@ -192,6 +196,8 @@ class Storage { const std::optional<utils::Bound<PropertyValue>> &lower, const std::optional<utils::Bound<PropertyValue>> &upper) const = 0; + virtual uint64_t ApproximateEdgeCount(EdgeTypeId id) const = 0; + virtual std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const = 0; virtual std::optional<storage::LabelPropertyIndexStats> GetIndexStats( @@ -224,6 +230,8 @@ class Storage { virtual bool LabelPropertyIndexExists(LabelId label, PropertyId property) const = 0; + virtual bool EdgeTypeIndexExists(EdgeTypeId edge_type) const = 0; + virtual IndicesInfo ListAllIndices() const = 0; virtual ConstraintsInfo ListAllConstraints() const = 0; @@ -268,10 +276,14 @@ class Storage { virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) = 0; + virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(EdgeTypeId edge_type) = 0; + virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) = 0; virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) = 0; + virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(EdgeTypeId edge_type) = 0; + virtual utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint( LabelId label, PropertyId property) = 0; diff --git a/src/storage/v2/vertices_iterable.cpp b/src/storage/v2/vertices_iterable.cpp index f6ff46da6..9753052ae 100644 --- a/src/storage/v2/vertices_iterable.cpp +++ b/src/storage/v2/vertices_iterable.cpp @@ -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 @@ -10,7 +10,6 @@ // licenses/APL.txt. #include "storage/v2/vertices_iterable.hpp" - namespace memgraph::storage { VerticesIterable::VerticesIterable(AllVerticesIterable vertices) : type_(Type::ALL) { diff --git a/src/storage/v2/vertices_iterable.hpp b/src/storage/v2/vertices_iterable.hpp index e057e8a38..6075a68a2 100644 --- a/src/storage/v2/vertices_iterable.hpp +++ b/src/storage/v2/vertices_iterable.hpp @@ -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 diff --git a/src/utils/atomic_memory_block.hpp b/src/utils/atomic_memory_block.hpp index c15424549..31a3cf3a9 100644 --- a/src/utils/atomic_memory_block.hpp +++ b/src/utils/atomic_memory_block.hpp @@ -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 diff --git a/src/utils/event_counter.cpp b/src/utils/event_counter.cpp index a7f4d30fb..54ff4ed5c 100644 --- a/src/utils/event_counter.cpp +++ b/src/utils/event_counter.cpp @@ -26,6 +26,7 @@ M(ScanAllByLabelPropertyValueOperator, Operator, "Number of times ScanAllByLabelPropertyValue operator was used.") \ M(ScanAllByLabelPropertyOperator, Operator, "Number of times ScanAllByLabelProperty operator was used.") \ M(ScanAllByIdOperator, Operator, "Number of times ScanAllById operator was used.") \ + M(ScanAllByEdgeTypeOperator, Operator, "Number of times ScanAllByEdgeTypeOperator operator was used.") \ M(ExpandOperator, Operator, "Number of times Expand operator was used.") \ M(ExpandVariableOperator, Operator, "Number of times ExpandVariable operator was used.") \ M(ConstructNamedPathOperator, Operator, "Number of times ConstructNamedPath operator was used.") \ diff --git a/src/utils/settings.cpp b/src/utils/settings.cpp index 4768edc42..5e0954b4b 100644 --- a/src/utils/settings.cpp +++ b/src/utils/settings.cpp @@ -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 diff --git a/src/utils/typeinfo.hpp b/src/utils/typeinfo.hpp index 3ed6128fc..77910f731 100644 --- a/src/utils/typeinfo.hpp +++ b/src/utils/typeinfo.hpp @@ -32,6 +32,7 @@ enum class TypeId : uint64_t { SCAN_ALL_BY_LABEL_PROPERTY_VALUE, SCAN_ALL_BY_LABEL_PROPERTY, SCAN_ALL_BY_ID, + SCAN_ALL_BY_EDGE_TYPE, EXPAND_COMMON, EXPAND, EXPANSION_LAMBDA, @@ -185,6 +186,7 @@ enum class TypeId : uint64_t { AST_EXPLAIN_QUERY, AST_PROFILE_QUERY, AST_INDEX_QUERY, + AST_EDGE_INDEX_QUERY, AST_CREATE, AST_CALL_PROCEDURE, AST_MATCH, diff --git a/tests/integration/durability/tests/v17/test_all/create_dataset.cypher b/tests/integration/durability/tests/v17/test_all/create_dataset.cypher new file mode 100644 index 000000000..9ee350d9a --- /dev/null +++ b/tests/integration/durability/tests/v17/test_all/create_dataset.cypher @@ -0,0 +1,22 @@ +// --storage-items-per-batch is set to 10 +CREATE INDEX ON :`label2`(`prop2`); +CREATE INDEX ON :`label2`(`prop`); +CREATE INDEX ON :`label`; +CREATE INDEX ON :__mg_vertex__(__mg_id__); +CREATE EDGE INDEX ON :`edge_type`; +CREATE (:`edge_index_from`), (:`edge_index_to`); +MATCH (n:`edge_index_from`), (m:`edge_index_to`) CREATE (n)-[r:`edge_type`]->(m); +CREATE (:__mg_vertex__:`label2` {__mg_id__: 0, `prop2`: ["kaj", 2, Null, {`prop4`: -1.341}], `ext`: 2, `prop`: "joj"}); +CREATE (:__mg_vertex__:`label2`:`label` {__mg_id__: 1, `ext`: 2, `prop`: "joj"}); +CREATE (:__mg_vertex__:`label2` {__mg_id__: 2, `prop2`: 2, `prop`: 1}); +CREATE (:__mg_vertex__:`label2` {__mg_id__: 3, `prop2`: 2, `prop`: 2}); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 0 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 1 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 2 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 3 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); +CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`); +CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`prop2`, u.`prop` IS UNIQUE; +ANALYZE GRAPH; +DROP INDEX ON :__mg_vertex__(__mg_id__); +DROP EDGE INDEX ON :`edge_type`; +MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; diff --git a/tests/integration/durability/tests/v17/test_all/expected_snapshot.cypher b/tests/integration/durability/tests/v17/test_all/expected_snapshot.cypher new file mode 100644 index 000000000..fb2d74667 --- /dev/null +++ b/tests/integration/durability/tests/v17/test_all/expected_snapshot.cypher @@ -0,0 +1,19 @@ + CREATE (:__mg_vertex__:`edge_index_from` {__mg_id__: 0}); + CREATE (:__mg_vertex__:`edge_index_to` {__mg_id__: 1}); + CREATE (:__mg_vertex__:`label2` {__mg_id__: 2, `prop2`: ["kaj", 2, Null, {`prop4`: -1.341}], `ext`: 2, `prop`: "joj"}); + CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop2`: 2, `prop`: 1}); + CREATE (:__mg_vertex__:`label2` {__mg_id__: 5, `prop2`: 2, `prop`: 2}); + CREATE (:__mg_vertex__:`label`:`label2` {__mg_id__: 3, `ext`: 2, `prop`: "joj"}); + CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`prop2`, u.`prop` IS UNIQUE; + CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`); + CREATE INDEX ON :__mg_vertex__(__mg_id__); + CREATE INDEX ON :`label2`(`prop2`); + CREATE INDEX ON :`label2`(`prop`); + CREATE INDEX ON :`label`; + DROP INDEX ON :__mg_vertex__(__mg_id__); + MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge_type`]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 2 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 3 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 4 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 5 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v); diff --git a/tests/integration/durability/tests/v17/test_all/expected_wal.cypher b/tests/integration/durability/tests/v17/test_all/expected_wal.cypher new file mode 100644 index 000000000..33efec9e2 --- /dev/null +++ b/tests/integration/durability/tests/v17/test_all/expected_wal.cypher @@ -0,0 +1,19 @@ + CREATE (:__mg_vertex__:`edge_index_from` {__mg_id__: 0}); + CREATE (:__mg_vertex__:`edge_index_to` {__mg_id__: 1}); + CREATE (:__mg_vertex__:`label2` {__mg_id__: 2, `prop2`: ["kaj", 2, Null, {`prop4`: -1.341}], `prop`: "joj", `ext`: 2}); + CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop2`: 2, `prop`: 1}); + CREATE (:__mg_vertex__:`label2` {__mg_id__: 5, `prop2`: 2, `prop`: 2}); + CREATE (:__mg_vertex__:`label`:`label2` {__mg_id__: 3, `prop`: "joj", `ext`: 2}); + CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`prop2`, u.`prop` IS UNIQUE; + CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`); + CREATE INDEX ON :__mg_vertex__(__mg_id__); + CREATE INDEX ON :`label2`(`prop2`); + CREATE INDEX ON :`label2`(`prop`); + CREATE INDEX ON :`label`; + DROP INDEX ON :__mg_vertex__(__mg_id__); + MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge_type`]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 2 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 3 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 4 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v); + MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 3 AND v.__mg_id__ = 5 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v); diff --git a/tests/integration/durability/tests/v17/test_all/snapshot.bin b/tests/integration/durability/tests/v17/test_all/snapshot.bin new file mode 100644 index 0000000000000000000000000000000000000000..9cc54b4809a1a843cc80955e0b5634a4f70154e1 GIT binary patch literal 2067 zcmcIlJ#Q015M91O2tn2xwnJwKA`&`@i*E_89MMowQljX5w`avRwi8^0Cci8azk&)O zQBl)G3KbB0-kXRsqKMe)Y3I$GH?w<tuV1~NteNF{X$OMrw@XW8JJ$p}xGrE5U?1S^ z4G9eZz5tvGXxk;l7=pHy8pd9hAc40(W!YO2^Xp*wHRJCi*8z+?;kZqqCb+Qll|Rk0 z#)YM?{x9i-Q`e!($>@Qs;D{fiS#stk(Jh>Sy$SoxH0}^Uh*cBLQKiY|&{52<V_4oH zI4XvoBL3dFOCp!p%nqir+_ygeI2s<$Pd+M{8pG*OuN>p*2=y(B+S_9CTqBI($|{>3 zF$AD}{L2F}RFxhu6g5mTIozR5hPXo=hPZ=Svhl<+#_2TML`YAF)Ai~WENzXJ=mgq{ zr4^<jMr}&3tjkeQfUW$!?glm0CG(kvoz~Kl+fhq22@bE0a_Fo{W(jawZx-uQPimb4 zeDAGLZgePFUwfi+f^@U9$^6GFn#R+tA)19X4PoaYKQB(t;T*?BKaP2?9Gs5#CL;=w zeD*DG(VxS6ve4M#TjcJDr&tZ*?%t$|hr9Es1>2#AMR@*U7A@XIcej~YA57oTKi`aa zIOoC9^qIXUwD-tuh+2_vhdP93X%@3Q&9k~ri>xZrGVAA|)2+&o<s|1X%I&P|mT6su o-{d;hX+OkH+U;lYdC?6;43&K<%KAB=I}S~M>=?C%_^U_OFSaa&o&W#< literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_all/wal.bin b/tests/integration/durability/tests/v17/test_all/wal.bin new file mode 100644 index 0000000000000000000000000000000000000000..61a33372da4fa550b0bbaf5b614b319121e6eabc GIT binary patch literal 3582 zcmbVO%Wl&^6b(-S;?WwnMu4CSRfQ0)nrD+P*|0*Y(o`t|WV!Yv%A-k?xTSz#!*3z+ zA$$i5RxnQH+&gwAiAolcJ#)^v=ia$9o;R;Q_Es~~-z)2>da}N&hiCStdMfUet5s1c zyFnpebc<!b>Q=*&aCa+yAm4=iD-pOH6mnsqnR9o=an&tKid$)hq1&wZmEC+*ggb?t zrGztE)@Akh&c3dmdj7HK<?U<gbvPOf^Nc&%vaZgfU?Qe4Ofn$-mQ*J}ctT=skm?=v z<asA8?@Wv$aqfBcO?|W-dLC0G`<9v&gcD&)c+sb!;7N(Qt#JEV2{2ZpG5a3mVv%+~ zln$Ds!HLelq#!(2voK~Og^7tNKN>Jn8o8_Jha40cl5?6OVdhDR70LyR$YV8HSn`r6 zrxA%|nsxv~>MWM%3TF)~?$}q9nXcb)2=1&n>r@k+T6FIJ`tz;xBl`W*7zVm=C$V;C z&@rM>xJia_YdBOnBOYGGdBk2BDN76fz6C6g)o5&9X~EWz>Cze?sE^tjK-4+hf~pwM zhN@sKI2lbHnWm}&Wmy%Q8V*i)tVYMeysCh$A=6a>q|Rcf%H$Z*SIdBft4Y^rd&U4Y z6o)qv@G|9fN*P9YtY%?sUdmu=$aKnp)L9IwGbtOe$&@(-1JsZ*mYj2vDL+stA5sNv zMCrBrU1~?#<xFYJQl@M1SN)7oxQH8X4JkOQ8SU&q<8~bzk3H6srXP}N2+T2!U`%OH zB1B1>hPnoY&V@}wOPYR6rlCd-l9fjNnbM$TAWGUa)HNu=T-Y?Ur0FMQ8m`De(lpwb zDGlD%;HFJOU4#C&X=q8)8_M*D<OPJjTCyMvlK8B`d#PIyw+gb06Ub8q;x}eXp-m;^ zrvg7x<6pAD;ItoQ_i`^cvi@1i3q@-rM0T(AmB~TM1!+x#chg!0sTH=zQM(^R-kIM! z6=P3@$;+v6<VW#rklowaE~I|SMr$-U9eU&TXK5w3T}pb(E%K(XtUY6`INw_AI$^D3 z^pf&hNLtG$tl=7#xcblyk**8&QM!T)Y{pZ%O_Ry_o@7gGsbKHTUyWINOp)y8irpj} zT^D&$;$A3R9pCQ^5h!nGMaSV_U5q>6Enuv!u1H7m?59VIe85N9Qbn8v%RHh9%_Y`7 R5^{-0?Dyr0gj}e2^bhc)#{2*P literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_constraints/create_dataset.cypher b/tests/integration/durability/tests/v17/test_constraints/create_dataset.cypher new file mode 100644 index 000000000..96bb4bac4 --- /dev/null +++ b/tests/integration/durability/tests/v17/test_constraints/create_dataset.cypher @@ -0,0 +1,6 @@ +CREATE CONSTRAINT ON (u:`label2`) ASSERT EXISTS (u.`ext2`); +CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`); +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`a` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`b` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`c` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`a`, u.`b` IS UNIQUE; diff --git a/tests/integration/durability/tests/v17/test_constraints/expected_snapshot.cypher b/tests/integration/durability/tests/v17/test_constraints/expected_snapshot.cypher new file mode 100644 index 000000000..fbe2c28ab --- /dev/null +++ b/tests/integration/durability/tests/v17/test_constraints/expected_snapshot.cypher @@ -0,0 +1,6 @@ +CREATE CONSTRAINT ON (u:`label2`) ASSERT EXISTS (u.`ext2`); +CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`); +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`c` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`b` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`a` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`b`, u.`a` IS UNIQUE; diff --git a/tests/integration/durability/tests/v17/test_constraints/expected_wal.cypher b/tests/integration/durability/tests/v17/test_constraints/expected_wal.cypher new file mode 100644 index 000000000..9260455ed --- /dev/null +++ b/tests/integration/durability/tests/v17/test_constraints/expected_wal.cypher @@ -0,0 +1,6 @@ +CREATE CONSTRAINT ON (u:`label2`) ASSERT EXISTS (u.`ext2`); +CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`); +CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`a`, u.`b` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`a` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`b` IS UNIQUE; +CREATE CONSTRAINT ON (u:`label`) ASSERT u.`c` IS UNIQUE; diff --git a/tests/integration/durability/tests/v17/test_constraints/snapshot.bin b/tests/integration/durability/tests/v17/test_constraints/snapshot.bin new file mode 100644 index 0000000000000000000000000000000000000000..76986ab9af9feb2f1a955fdddbacb9fa3a1ad874 GIT binary patch literal 625 zcmaKpJx&8L5QR4+kSaw+J^=|LjivRh_pePu#Tk(8b)bmSBn5|{=6clJ0z~FbA_c{b zKh1mdJhq>o_Pcp;Tx)t2`UgHBeFD~_BheS&8}M=?(wo4Nq^r0BE7cxI`p)c<U7Vyb zCYb>A5N%iK79N!}P(7E@m99Gpyr(ImQ8I{5ga9o?!f{#(gqs*Zen03v4KlVN<}D%H zAm`W1`f>&{{F(Vtg@swln1gGECRD90x$3O<iEZpRZ2Q)^&TslUH>~E6a}Kdfc1WGK cp3Bn4wi>MCvQFOB+~+_uJb>(Hzjjgi0aYC*O8@`> literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_constraints/wal.bin b/tests/integration/durability/tests/v17/test_constraints/wal.bin new file mode 100644 index 0000000000000000000000000000000000000000..f2d54e5fd2ccb96b8792d650d271c29e2c1a47ae GIT binary patch literal 460 zcma)&JqyAx5Qgg~A`TL=J1E@(%~zYQE>7-Uisag4>L}>$A2-lTqmyQO!#&U4ao5XZ z*k|KAlLIF5!f>P~OoesURHb5Np-ZNu)XYLJnJ`9M=xW|qVuGC(-UGORc+ONW4Kqpr z^Sl;DI__%d2sX}gGRN4`HKxIW8w6eA*FEndVNFhAp{1)&gD#Sgpuoqzw{M=zkf$-& k#0G7H<)0cPHfY;_wJxzi+f7<rcRa&%pS0za!r-ViZvzA{R{#J2 literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_edges/create_dataset.cypher b/tests/integration/durability/tests/v17/test_edges/create_dataset.cypher new file mode 100644 index 000000000..ab3b3af6d --- /dev/null +++ b/tests/integration/durability/tests/v17/test_edges/create_dataset.cypher @@ -0,0 +1,60 @@ +// --storage-items-per-batch is set to 7 +CREATE INDEX ON :__mg_vertex__(__mg_id__); +CREATE (:__mg_vertex__ {__mg_id__: 0}); +CREATE (:__mg_vertex__ {__mg_id__: 1}); +CREATE (:__mg_vertex__ {__mg_id__: 2}); +CREATE (:__mg_vertex__ {__mg_id__: 3}); +CREATE (:__mg_vertex__ {__mg_id__: 4}); +CREATE (:__mg_vertex__ {__mg_id__: 5}); +CREATE (:__mg_vertex__ {__mg_id__: 6}); +CREATE (:__mg_vertex__ {__mg_id__: 7}); +CREATE (:__mg_vertex__ {__mg_id__: 8}); +CREATE (:__mg_vertex__ {__mg_id__: 9}); +CREATE (:__mg_vertex__ {__mg_id__: 10}); +CREATE (:__mg_vertex__ {__mg_id__: 11}); +CREATE (:__mg_vertex__ {__mg_id__: 12}); +CREATE (:__mg_vertex__ {__mg_id__: 13}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 14}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 15}); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 2 AND v.__mg_id__ = 3 CREATE (u)-[:`edge` {`prop`: 11}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 4 AND v.__mg_id__ = 5 CREATE (u)-[:`edge` {`prop`: true}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 6 AND v.__mg_id__ = 7 CREATE (u)-[:`edge2`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 8 AND v.__mg_id__ = 9 CREATE (u)-[:`edge2` {`prop`: -3.141}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 10 AND v.__mg_id__ = 11 CREATE (u)-[:`edgelink` {`prop`: {`prop`: 1, `prop2`: {`prop4`: 9}}}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 12 AND v.__mg_id__ = 13 CREATE (u)-[:`edgelink` {`prop`: [1, Null, false, "\n\n\n\n\\\"\"\n\t"]}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v); +ANALYZE GRAPH; +DROP INDEX ON :__mg_vertex__(__mg_id__); +MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; diff --git a/tests/integration/durability/tests/v17/test_edges/expected_snapshot.cypher b/tests/integration/durability/tests/v17/test_edges/expected_snapshot.cypher new file mode 100644 index 000000000..596753ba5 --- /dev/null +++ b/tests/integration/durability/tests/v17/test_edges/expected_snapshot.cypher @@ -0,0 +1,58 @@ +CREATE INDEX ON :__mg_vertex__(__mg_id__); +CREATE (:__mg_vertex__ {__mg_id__: 0}); +CREATE (:__mg_vertex__ {__mg_id__: 1}); +CREATE (:__mg_vertex__ {__mg_id__: 2}); +CREATE (:__mg_vertex__ {__mg_id__: 3}); +CREATE (:__mg_vertex__ {__mg_id__: 4}); +CREATE (:__mg_vertex__ {__mg_id__: 5}); +CREATE (:__mg_vertex__ {__mg_id__: 6}); +CREATE (:__mg_vertex__ {__mg_id__: 7}); +CREATE (:__mg_vertex__ {__mg_id__: 8}); +CREATE (:__mg_vertex__ {__mg_id__: 9}); +CREATE (:__mg_vertex__ {__mg_id__: 10}); +CREATE (:__mg_vertex__ {__mg_id__: 11}); +CREATE (:__mg_vertex__ {__mg_id__: 12}); +CREATE (:__mg_vertex__ {__mg_id__: 13}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 14}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 15}); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 2 AND v.__mg_id__ = 3 CREATE (u)-[:`edge` {`prop`: 11}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 4 AND v.__mg_id__ = 5 CREATE (u)-[:`edge` {`prop`: true}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 6 AND v.__mg_id__ = 7 CREATE (u)-[:`edge2`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 8 AND v.__mg_id__ = 9 CREATE (u)-[:`edge2` {`prop`: -3.141}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 10 AND v.__mg_id__ = 11 CREATE (u)-[:`edgelink` {`prop`: {`prop`: 1, `prop2`: {`prop4`: 9}}}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 12 AND v.__mg_id__ = 13 CREATE (u)-[:`edgelink` {`prop`: [1, Null, false, "\n\n\n\n\\\"\"\n\t"]}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v); +DROP INDEX ON :__mg_vertex__(__mg_id__); +MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; diff --git a/tests/integration/durability/tests/v17/test_edges/expected_wal.cypher b/tests/integration/durability/tests/v17/test_edges/expected_wal.cypher new file mode 100644 index 000000000..596753ba5 --- /dev/null +++ b/tests/integration/durability/tests/v17/test_edges/expected_wal.cypher @@ -0,0 +1,58 @@ +CREATE INDEX ON :__mg_vertex__(__mg_id__); +CREATE (:__mg_vertex__ {__mg_id__: 0}); +CREATE (:__mg_vertex__ {__mg_id__: 1}); +CREATE (:__mg_vertex__ {__mg_id__: 2}); +CREATE (:__mg_vertex__ {__mg_id__: 3}); +CREATE (:__mg_vertex__ {__mg_id__: 4}); +CREATE (:__mg_vertex__ {__mg_id__: 5}); +CREATE (:__mg_vertex__ {__mg_id__: 6}); +CREATE (:__mg_vertex__ {__mg_id__: 7}); +CREATE (:__mg_vertex__ {__mg_id__: 8}); +CREATE (:__mg_vertex__ {__mg_id__: 9}); +CREATE (:__mg_vertex__ {__mg_id__: 10}); +CREATE (:__mg_vertex__ {__mg_id__: 11}); +CREATE (:__mg_vertex__ {__mg_id__: 12}); +CREATE (:__mg_vertex__ {__mg_id__: 13}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 14}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 15}); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 2 AND v.__mg_id__ = 3 CREATE (u)-[:`edge` {`prop`: 11}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 4 AND v.__mg_id__ = 5 CREATE (u)-[:`edge` {`prop`: true}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 6 AND v.__mg_id__ = 7 CREATE (u)-[:`edge2`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 8 AND v.__mg_id__ = 9 CREATE (u)-[:`edge2` {`prop`: -3.141}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 10 AND v.__mg_id__ = 11 CREATE (u)-[:`edgelink` {`prop`: {`prop`: 1, `prop2`: {`prop4`: 9}}}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 12 AND v.__mg_id__ = 13 CREATE (u)-[:`edgelink` {`prop`: [1, Null, false, "\n\n\n\n\\\"\"\n\t"]}]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v); +MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v); +DROP INDEX ON :__mg_vertex__(__mg_id__); +MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; diff --git a/tests/integration/durability/tests/v17/test_edges/snapshot.bin b/tests/integration/durability/tests/v17/test_edges/snapshot.bin new file mode 100644 index 0000000000000000000000000000000000000000..070bbe530945bfc947eaa95233be64f6d54aa10b GIT binary patch literal 4297 zcmb7|yKYlK5Qfd21Omy%$>Gi(=SE`*Ut*uMG*ony2+6UJ0wM|#iA2F;P*T(K3>3Tx z4+1ec|F?$UjzSub{@K}?+4*O7PhLDfI#{Zm{yhtOl@t~(pGslpb|t))@J7NX3CoMs znuiiTOZZj^d8l@-A>`pgg(ac#TFb(XN|%R=vsW$s%EKk8tKV84w9AxzmtX$;`sGKQ z`<{m@G9(PyWvlX87j*dH{b9y_b=i*LcDAY={=&Oin5$gm;p%K>EAo-{&la<#T2`0& z{K?;|Mq_?157)F$^?Q=-b%$g)an|#`N=H1g>ZY$+aOjpp6%{6{TUCA%6453+O4RR3 zLd(8=-JunSRvpS5y5rDYhweFa-=PN%J#^@iLysMb4&@Ft99nZ|-JuPKHvfepRhS~E zClUwY0es`Q;R7o5NZJ`hEgWgEw5STP=6#;l$Ue1#M@H2>GP#Ip)CfvG$ex7U2Q+dY z<cdJh65_d1#GZg;A82up3^sdYN;RgjDQHUe%nbKId$<n|S_71vW17eAo)k%1GZXv3 z=<>+uQjbi&Vw%u`rex17bRV?6`yhMLV2bM)VrnXPj~N2DiR=RbA;43nk4Gk7F?Eih zDcNK1J{V`L!9CeiGE|B_a#1F)#6i;7bK%K8aMgHZvDzqjWO5Nx=Lkx1z&P$M_mR)` zLH5K-2dRjWw@&g(9Au9@T`v0|vqS<;xG_62xrnKA1Zl%s8i?G<p8eWvA7oFgbdZo3 zfZRRVYjuhhgr|W=#w~bO?J*-*N=`r^JUX!As5fRG1N5Zt*%@<4O&F*U2m=H%a8jt? z8GVC_Q3<I~Q4G!(6)BSiD$bWIP?0+b0v$ybsKmh38mM3*S)hUnWP$38!trV(3t7c9 zG^h$SpoAc7#7SwRuz3!0@OENn%z3;J1}fehgn^1n-^syU?d0IHbaHUVIXSpwoE+Q@ zPL8N!A!dYJDkn!nDkzQ>v&hN8FmQ6vy8lzSb#XL(;5hOGXafnMk=`6C9v_hVgWYl; zeO*Uaj`qs^w+HWdEMYL#w0RBj-8zQ3UD@WiJUafPQTbUv{-E6=!nL|ity}C2yQ9&l zS?;z*&F)~k-7H#Vzgax(_B)+cuiYNBSX`8YvTXN?X4&g^n%&)Q(cBpn!)DPNb&CFQ bd)OL{!beqcO9G<;mhVZ$8dUs)LRP;408MO2 literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_edges/wal.bin b/tests/integration/durability/tests/v17/test_edges/wal.bin new file mode 100644 index 0000000000000000000000000000000000000000..914f49154070a94eb847744868f9f0f10759e93d GIT binary patch literal 6616 zcma)=TW{l36ot#3mRsw|)Z8hVQXw=MC2rED6+&o)03kq?$B+stOfF6cgtltZf<Qd+ zGkE5I@W4OehrmwiwN^U59Qx$$?DegkbB>qYe)iE%ll7JBpZne$>6Q0;dU2<+on8fv z`hKh3+^q%uexufG_xiPVf4^NDcAM>1BT0sPjRQ8f8ui8?=nZPSgWX}R88jNT?y!DP zs|P`IzZ*1r2d#mZf>(CDN7Ku%E6=4@as1;bzDh1;$<J|ISzWG<2XQRP@s9U+`svXU z5w(!De6-Y&Ix}`YKU2RMKS|?9(ks<M549p=s3Ude7=t3R89z;9tek2gSF^|%>PVeA z##yo%uhIB1eq+=^549p=bVlmTF$P6$X8bITv9qd$JOM?<P)F*_G0u|B_<0&X!EcOO z=%H3*jLt}%ImV#K&5UtV9`Tr|7P3}k40WW=9Ag|ok<A!q{m6TY-x#%!wIXA5M(WHl z&XUcz!llLuRW0P1FEWNYQfH2FmTbnGG=@U8&_k`r80tu!kStPco>x9U6WNSkq%m#{ z)k5BEMaED^>dZ0DlFj&K8b6a>sTO*u6&XVvsWZnI6p_vNRT|@-R4wFPUSte)q|O}U zEZK}-r!gKes)ZhEMaED^>dY|)MPxJHqA|iiwUF^rWDNCEC(>fl?IjbjBC&Et%X-^r zRgU&TdZk+Ep;j~+)JvTYjfRd$tenx{BX1jRJ00i?<jG0}@th1s39Ec|N@D6As`2=| zsm8O7&ml=VBvNvIadz(WsH@eC-lh?dR4vBD%#43YI(qt|YhkqUuX@bk9XjB-SivgK z9RlV%^Hv&@-=!QT56KaTO92H<M^8(L7Zcu!{`vEdzqeNZ65gEd(FyZaz@85}8K3^Z zoLX8;$U0miuHeFKGqbSQ+rf>6+q@k3o4NJoa{j{6X0CH)JPETiBTabcOV;7y-<<h2 zmdod1^4I;16`wD%YxZ+et*)&KX5$1L2hT^2GfSqk#fid=%sW6bBQ#0C+LsWpRFWBC zj|AMRB?Rn^WJYL{fJaXW0f$;LBOH=|XGjSFYb}`(-X#IsS3<xElFSJ2k$_Y$A>i~# zW`y@iz@C*5u;G#!;XVn-$PxlJTrwkkKmuL}O9<F-$&BzJ2^;BE2>}}}nGrrF0cW;^ zfDM<-2%nH}%RK`&TrwkkN&;RJOJ=}^OJ;=6Nx1Ev0UIuv5spZ}8)L}~*l@{=5R%{* z&Hy3S+WxwR%m@()czMjuK!WlCG9!FR!b|QMln;;@;g|%xN0!W>e1OadUy<;Ndj{nL zWJWk40k4!LGbkS*Gr~6{yyl)k`2d*_z9j)~mnAbOA0RVAj|BYRri7q;fXoO361LK- z5`yvpG9wI0c+*W#K0s!K??|Y+3Cah^jPN}PciaT!17t>+kg)A0C?6m*!YK)Fxe3Y# z$c%7K!j7AOn^ZC*T%@{w{wq7*KKN&81;_E%YX9u=bXGlV?A_g}4#v~j__RNZuey`V zWEwAjHl1~6^N;)0!@9M0uSW6c;_UJ~o{oP>On+S(B%_POer?`s@3(%_WH#h19rEP= z4k?MLuTu2`=372Sou&WRTUXx$zN@*3NXG1n`ZtKkKR1z`xXHLjH;KqYH<6tVBJ$Bq zWT%6OymS-U=^!FM-9&aeh{#hnk(~}A^3_dbr-O)(-9&ccqTuu(f89iOI*7<)H<6tV iBJ$ZyWT%6Oymk}W=^!G%-9&aeh{$s{k)5T)<NpAH_`ARW literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_indices/create_dataset.cypher b/tests/integration/durability/tests/v17/test_indices/create_dataset.cypher new file mode 100644 index 000000000..739062f19 --- /dev/null +++ b/tests/integration/durability/tests/v17/test_indices/create_dataset.cypher @@ -0,0 +1,6 @@ +CREATE INDEX ON :`label2`; +CREATE INDEX ON :`label2`(`prop2`); +CREATE INDEX ON :`label`(`prop2`); +CREATE INDEX ON :`label`(`prop`); +CREATE EDGE INDEX ON :`edgetype`; +ANALYZE GRAPH; diff --git a/tests/integration/durability/tests/v17/test_indices/expected_snapshot.cypher b/tests/integration/durability/tests/v17/test_indices/expected_snapshot.cypher new file mode 100644 index 000000000..1e930697a --- /dev/null +++ b/tests/integration/durability/tests/v17/test_indices/expected_snapshot.cypher @@ -0,0 +1,5 @@ +CREATE INDEX ON :`label2`; +CREATE INDEX ON :`label`(`prop`); +CREATE INDEX ON :`label`(`prop2`); +CREATE INDEX ON :`label2`(`prop2`); +CREATE EDGE INDEX ON :`edgetype`; diff --git a/tests/integration/durability/tests/v17/test_indices/expected_wal.cypher b/tests/integration/durability/tests/v17/test_indices/expected_wal.cypher new file mode 100644 index 000000000..bfae88b0b --- /dev/null +++ b/tests/integration/durability/tests/v17/test_indices/expected_wal.cypher @@ -0,0 +1,5 @@ +CREATE INDEX ON :`label2`; +CREATE INDEX ON :`label2`(`prop2`); +CREATE INDEX ON :`label`(`prop2`); +CREATE INDEX ON :`label`(`prop`); +CREATE EDGE INDEX ON :`edgetype`; diff --git a/tests/integration/durability/tests/v17/test_indices/snapshot.bin b/tests/integration/durability/tests/v17/test_indices/snapshot.bin new file mode 100644 index 0000000000000000000000000000000000000000..99ad6e0ea29bd82dbabbc8cd429ce40afa116005 GIT binary patch literal 731 zcmbu7J#NB45QUupQ3@J{JwZYQ%}O47h1oVuM3E{d@Or&&EU+L=&XYTE2ilw_9r<Bq zjAR#);>Mrny?L`c^7GSvGl|a6BUD5^@P2jH;J|QXSYMO$!f<5xAeaG}BcKscPlcr^ z0}P0Ix+wV~M=H`KAc5Uuup~nXK2K5sN&aVh_dI`?zGyc!B>O|V4-UA45h1Wk-BP9W zRr&hXl(J66mS(qY<aUY^>fDvJOqKGFIg!cghSGoC$y|hcH6sOcvvk&=Mi-+tu5kMM wE7!62Hck-HXK8CT!C|s;IxKC_Cip_z%w&4yQ*=e1nk-lt^Bw7Z@jG=YU(Ba5i2wiq literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_indices/wal.bin b/tests/integration/durability/tests/v17/test_indices/wal.bin new file mode 100644 index 0000000000000000000000000000000000000000..661cba6c1b1f55ffefae349b245c01a11b6533b1 GIT binary patch literal 847 zcmc(cy-UPE5XBEs5wS6(vrwX3<w9<gc)ON1R`w?;!hQ@P5R#B=PE@S?$KAV|O^$F_ z>Q=M7_j_-WeR+N>FK4^+<Xz+BedBQJ-{C}A905xNp%JGfma!&6E;$LsLIY)sP%#+H ziYth;=7b5RNeoLxWSB@2!h;lyCsgv-!{FK6JHqkRKgFpO5=uHMtGRcKQaUmDCnz*+ z-B`r=EaKAA!L!XBZb&$rCC-Ktm%1Zd=!O_w7C1)$e20S0!MryxYSwmvHZfiP3Wd)= z)qQ1x$MF6p5bHeCkhcK#{x#BSO1mYuAL6HnvUJgzqEaSXi?W9<+pFn}FdJ2kt$ADb dP1Y43wwY9-`fV3ie&C<}?<)I`^|za>egKm>sSp4F literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_vertices/create_dataset.cypher b/tests/integration/durability/tests/v17/test_vertices/create_dataset.cypher new file mode 100644 index 000000000..061df375b --- /dev/null +++ b/tests/integration/durability/tests/v17/test_vertices/create_dataset.cypher @@ -0,0 +1,18 @@ +// --storage-items-per-batch is set to 5 +CREATE INDEX ON :__mg_vertex__(__mg_id__); +CREATE (:__mg_vertex__ {__mg_id__: 0}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 1}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 2, `prop`: false}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 3, `prop`: true}); +CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop`: 1}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 5, `prop2`: 3.141}); +CREATE (:__mg_vertex__:`label6` {__mg_id__: 6, `prop3`: true, `prop2`: -314000000}); +CREATE (:__mg_vertex__:`label3`:`label1`:`label2` {__mg_id__: 7}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 8, `prop3`: "str", `prop2`: 2, `prop`: 1}); +CREATE (:__mg_vertex__:`label2`:`label1` {__mg_id__: 9, `prop`: {`prop_nes`: "kaj je"}}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 10, `prop_array`: [1, false, Null, "str", {`prop2`: 2}]}); +CREATE (:__mg_vertex__:`label3`:`label` {__mg_id__: 11, `prop`: {`prop`: [1, false], `prop2`: {}, `prop3`: "test2", `prop4`: "test"}}); +CREATE (:__mg_vertex__ {__mg_id__: 12, `prop`: " \n\"\'\t\\%"}); +ANALYZE GRAPH; +DROP INDEX ON :__mg_vertex__(__mg_id__); +MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; diff --git a/tests/integration/durability/tests/v17/test_vertices/expected_snapshot.cypher b/tests/integration/durability/tests/v17/test_vertices/expected_snapshot.cypher new file mode 100644 index 000000000..ecdc1229e --- /dev/null +++ b/tests/integration/durability/tests/v17/test_vertices/expected_snapshot.cypher @@ -0,0 +1,16 @@ +CREATE INDEX ON :__mg_vertex__(__mg_id__); +CREATE (:__mg_vertex__ {__mg_id__: 0}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 1}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 2, `prop`: false}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 3, `prop`: true}); +CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop`: 1}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 5, `prop2`: 3.141}); +CREATE (:__mg_vertex__:`label6` {__mg_id__: 6, `prop3`: true, `prop2`: -314000000}); +CREATE (:__mg_vertex__:`label2`:`label3`:`label1` {__mg_id__: 7}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 8, `prop3`: "str", `prop2`: 2, `prop`: 1}); +CREATE (:__mg_vertex__:`label1`:`label2` {__mg_id__: 9, `prop`: {`prop_nes`: "kaj je"}}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 10, `prop_array`: [1, false, Null, "str", {`prop2`: 2}]}); +CREATE (:__mg_vertex__:`label`:`label3` {__mg_id__: 11, `prop`: {`prop`: [1, false], `prop2`: {}, `prop3`: "test2", `prop4`: "test"}}); +CREATE (:__mg_vertex__ {__mg_id__: 12, `prop`: " \n\"\'\t\\%"}); +DROP INDEX ON :__mg_vertex__(__mg_id__); +MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; diff --git a/tests/integration/durability/tests/v17/test_vertices/expected_wal.cypher b/tests/integration/durability/tests/v17/test_vertices/expected_wal.cypher new file mode 100644 index 000000000..d8f758737 --- /dev/null +++ b/tests/integration/durability/tests/v17/test_vertices/expected_wal.cypher @@ -0,0 +1,16 @@ +CREATE INDEX ON :__mg_vertex__(__mg_id__); +CREATE (:__mg_vertex__ {__mg_id__: 0}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 1}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 2, `prop`: false}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 3, `prop`: true}); +CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop`: 1}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 5, `prop2`: 3.141}); +CREATE (:__mg_vertex__:`label6` {__mg_id__: 6, `prop2`: -314000000, `prop3`: true}); +CREATE (:__mg_vertex__:`label2`:`label3`:`label1` {__mg_id__: 7}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 8, `prop`: 1, `prop2`: 2, `prop3`: "str"}); +CREATE (:__mg_vertex__:`label1`:`label2` {__mg_id__: 9, `prop`: {`prop_nes`: "kaj je"}}); +CREATE (:__mg_vertex__:`label` {__mg_id__: 10, `prop_array`: [1, false, Null, "str", {`prop2`: 2}]}); +CREATE (:__mg_vertex__:`label`:`label3` {__mg_id__: 11, `prop`: {`prop`: [1, false], `prop2`: {}, `prop3`: "test2", `prop4`: "test"}}); +CREATE (:__mg_vertex__ {__mg_id__: 12, `prop`: " \n\"\'\t\\%"}); +DROP INDEX ON :__mg_vertex__(__mg_id__); +MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__; diff --git a/tests/integration/durability/tests/v17/test_vertices/snapshot.bin b/tests/integration/durability/tests/v17/test_vertices/snapshot.bin new file mode 100644 index 0000000000000000000000000000000000000000..8a67d9a7d31453b35945fe69115087c3040990d0 GIT binary patch literal 1739 zcma)7y>8S%7`%l1a3M(Chm&nrNzg%I>Fo9Wb%-LQNtY4{d3`IS5QXUOnsmGeHKNT6 z(D5eJJVQ7qJF~IYM7Wy!c6N5Y+0VYy_lwJ3=l*#opF9d+WAnZwZvYIyF+hLIb25M* z05=|J@yeb50b15zgkT+&e+y|LDc;7)6EC28Enc)>Iar8A!BM^2xE(DUu*x&z3MIlh zQDa(s!ks{h&48m`p8fv*?U(MpZS&;s;eiq>l%$e;{dw}|cJ;_M!)omm%ON#(`V=8_ z0iyosQw!OQjINLr&rl(a;Fxq+QVBLLmUEs&poPA;l@&HYH_%f1C?&9bHUDz;@zO0+ zf<!-;=lb07C;}}z$O0-Y6Z_0_)L!Tg^wrF&7mdyy7y`XEl&hc|<*o&11QJ6)TOs7o zJ90??>Z8;i_J)gD`AnL2_Ti#YH;O-(Zn4bys}?R4y~BY~D(<O%f_*9j&<?;P~I zA6{`g!4c6T;5FA;6b`~PA=(6YeK#NFd|qCY#7Vg*t-HuL{mr`J+t-N22<EIwAMtC^ zpy&J?HR(CuC#GCG<wQ{A&SRh5A0|gW?7S#*I~`PMR#aJ9IO|etXOq;;?5HqiHk{@a t$um2)<*+oVvxQBIVqB$@!K6x!&5SGUU_7d(au{9b^QeMXA^b&w^A~t%WBC98 literal 0 HcmV?d00001 diff --git a/tests/integration/durability/tests/v17/test_vertices/wal.bin b/tests/integration/durability/tests/v17/test_vertices/wal.bin new file mode 100644 index 0000000000000000000000000000000000000000..304db455f08ea9363a01f9248e3ece3020f731db GIT binary patch literal 4355 zcma)8%Wl&^6b)}$9!iZ{BZMFdP&Qb~!)cSQ5F6MaRjwBmRI;385~YN;O5E}i5)z-m zn$KXzpRnc|$|RX{@3<a!$ZB%u+;h)8zMk>xmmkLE(&G2Rd8k&-ceQxrK31ziXLF-n z>u&l<FRA-MqZ|0$L2KYQx}89@I_+jX2?*S7)q<ek5`JIwyMEA0+I~kg>wY4Vda%(N z)LXrVqlioE&PBC+>)ukUIDS8jPsAh@ALH0v(xan(9COO<x^r3WZflH)g%mSWb9ThY zTse>S7%vN7rNS4~iesUJSW#hQYsf2#W}I}zn7Jb6GDt%m#!G3}NgBN6SV(>sNkg`V zJR=Q=h`9{X^0hTQ9?{X{@W@+{QvgmWf0^W8B4-f`9mI;{8<1z@0}(NoK|boR9u1$B zf65ps|0c-?ien+oYmt0pYe>w>d@veJ7BQJYf4;be$L8FA8Zf4oZQQ0dU?0aq2eG0y zkgXw+qUIZjh`9`1Ac<>hcx=wPXn1#`KR<u`t}H!gtZW0v+cv$h91AH{)CRIOWaepX zFj>T8hCYyG#Ao|(zWsXj_f-C@*rvClfn>8dskWVK^yVNIQmm+10}`_5&#cJ>G_nk& zZimPLMxlr$87#}UAUkwvDsf<NETluEs6%9HNO-TN5{QVo4ASzoHGI*#WuQj!#Q@W3 z$}MTMI=1P#IG(2IgyGVlH8KdgITkvI6&ZwV4Vihq^5NMaas&X0xE>16ulCs`?$MqR zpD}R_x10;(XNJWGV(Ph&_b%D5><i|SR@@~kaN=_;q+6!Q3S?_YJOVW<Kt#-CXft11 z!((&aPcoS#pS)F6@#tKZ7o=CtJaUgM0X#QDkSCqEM^4;OD~^Q@Vnt3MTSLNioC^jB zbSbldh`5}=EJ$H_8a_J{_~&9VGr(KLS4YM@7h)DBVLnTlxO}MqOT{#8(7#zy;8C5~ zChL>#-Y0i)d2=kJ&w(O$ksUG8U`(D_!)F|Dac)+Yt528qp0E}v^?{PwGposeYrAh$ z#+PdE@c1CDZq=L5D%JjInvM>7X?&85kHs|B_oiu*&bE8it(t@;C&PF+IXpg!r=!n8 z<}X5hF`NkbxAv7)Nr4qrZts5ua*E8Zs_Y%yFeNh$BhUTtFEeh3UPe13omTnhm|?`I z+8GIHVZ^0iXC$PB5u;^iB&3BA@0gvDkQPQ9Ms`L*S{U&P*%=9GVZ_B{XC$PB5!-e~ jLSn&}aAqEH8QB>LX<@{1Zf7K<g%RhVosp2|GVcBZa+C9o literal 0 HcmV?d00001 diff --git a/tests/manual/interactive_planning.cpp b/tests/manual/interactive_planning.cpp index 3f64c4f37..f0f60ba91 100644 --- a/tests/manual/interactive_planning.cpp +++ b/tests/manual/interactive_planning.cpp @@ -214,6 +214,8 @@ class InteractiveDbAccessor { return label_property_index_.at(key); } + bool EdgeTypeIndexExists(memgraph::storage::EdgeTypeId edge_type) { return true; } + std::optional<memgraph::storage::LabelIndexStats> GetIndexStats(const memgraph::storage::LabelId label) const { return dba_->GetIndexStats(label); } diff --git a/tests/unit/dbms_database.cpp b/tests/unit/dbms_database.cpp index 535c0c055..0fded2324 100644 --- a/tests/unit/dbms_database.cpp +++ b/tests/unit/dbms_database.cpp @@ -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 diff --git a/tests/unit/query_plan.cpp b/tests/unit/query_plan.cpp index bc4b2660c..5b574c1ff 100644 --- a/tests/unit/query_plan.cpp +++ b/tests/unit/query_plan.cpp @@ -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 @@ -808,13 +808,68 @@ TYPED_TEST(TestPlanner, MatchWhereBeforeExpand) { CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectFilter(), ExpectExpand(), ExpectProduce()); } +TYPED_TEST(TestPlanner, MatchEdgeTypeIndex) { + FakeDbAccessor dba; + auto indexed_edge_type = dba.EdgeType("indexed_edgetype"); + dba.SetIndexCount(indexed_edge_type, 1); + { + // Test MATCH ()-[r:indexed_edgetype]->() RETURN r; + auto *query = QUERY(SINGLE_QUERY( + MATCH(PATTERN(NODE("anon1"), EDGE("r", memgraph::query::EdgeAtom::Direction::OUT, {"indexed_edgetype"}), + NODE("anon2"))), + RETURN("r"))); + auto symbol_table = memgraph::query::MakeSymbolTable(query); + auto planner = MakePlanner<TypeParam>(&dba, this->storage, symbol_table, query); + CheckPlan(planner.plan(), symbol_table, ExpectScanAllByEdgeType(), ExpectProduce()); + } + { + // Test MATCH (a)-[r:indexed_edgetype]->() RETURN r; + auto *query = QUERY(SINGLE_QUERY( + MATCH(PATTERN(NODE("a"), EDGE("r", memgraph::query::EdgeAtom::Direction::OUT, {"indexed_edgetype"}), + NODE("anon2"))), + RETURN("r"))); + auto symbol_table = memgraph::query::MakeSymbolTable(query); + auto planner = MakePlanner<TypeParam>(&dba, this->storage, symbol_table, query); + CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(), ExpectProduce()); + } + { + // Test MATCH ()-[r:indexed_edgetype]->(b) RETURN r; + auto *query = QUERY(SINGLE_QUERY( + MATCH(PATTERN(NODE("anon1"), EDGE("r", memgraph::query::EdgeAtom::Direction::OUT, {"indexed_edgetype"}), + NODE("b"))), + RETURN("r"))); + auto symbol_table = memgraph::query::MakeSymbolTable(query); + auto planner = MakePlanner<TypeParam>(&dba, this->storage, symbol_table, query); + CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(), ExpectProduce()); + } + { + // Test MATCH (a)-[r:indexed_edgetype]->(b) RETURN r; + auto *query = QUERY(SINGLE_QUERY( + MATCH( + PATTERN(NODE("a"), EDGE("r", memgraph::query::EdgeAtom::Direction::OUT, {"indexed_edgetype"}), NODE("b"))), + RETURN("r"))); + auto symbol_table = memgraph::query::MakeSymbolTable(query); + auto planner = MakePlanner<TypeParam>(&dba, this->storage, symbol_table, query); + CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(), ExpectProduce()); + } + { + // Test MATCH ()-[r:not_indexed_edgetype]->() RETURN r; + auto *query = QUERY(SINGLE_QUERY( + MATCH(PATTERN(NODE("anon1"), EDGE("r", memgraph::query::EdgeAtom::Direction::OUT, {"not_indexed_edgetype"}), + NODE("anon2"))), + RETURN("r"))); + auto symbol_table = memgraph::query::MakeSymbolTable(query); + auto planner = MakePlanner<TypeParam>(&dba, this->storage, symbol_table, query); + CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(), ExpectProduce()); + } +} + TYPED_TEST(TestPlanner, MatchFilterPropIsNotNull) { FakeDbAccessor dba; auto label = dba.Label("label"); auto prop = PROPERTY_PAIR(dba, "prop"); dba.SetIndexCount(label, 1); dba.SetIndexCount(label, prop.second, 1); - { // Test MATCH (n :label) -[r]- (m) WHERE n.prop IS NOT NULL RETURN n auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", "label"), EDGE("r"), NODE("m"))), diff --git a/tests/unit/query_plan_checker.hpp b/tests/unit/query_plan_checker.hpp index 92089eb82..6eef3841a 100644 --- a/tests/unit/query_plan_checker.hpp +++ b/tests/unit/query_plan_checker.hpp @@ -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 @@ -65,6 +65,7 @@ class PlanChecker : public virtual HierarchicalLogicalOperatorVisitor { PRE_VISIT(ScanAllByLabelPropertyValue); PRE_VISIT(ScanAllByLabelPropertyRange); PRE_VISIT(ScanAllByLabelProperty); + PRE_VISIT(ScanAllByEdgeType); PRE_VISIT(ScanAllById); PRE_VISIT(Expand); PRE_VISIT(ExpandVariable); @@ -170,6 +171,7 @@ using ExpectCreateExpand = OpChecker<CreateExpand>; using ExpectDelete = OpChecker<Delete>; using ExpectScanAll = OpChecker<ScanAll>; using ExpectScanAllByLabel = OpChecker<ScanAllByLabel>; +using ExpectScanAllByEdgeType = OpChecker<ScanAllByEdgeType>; using ExpectScanAllById = OpChecker<ScanAllById>; using ExpectExpand = OpChecker<Expand>; using ExpectConstructNamedPath = OpChecker<ConstructNamedPath>; @@ -560,6 +562,12 @@ class FakeDbAccessor { return 0; } + int64_t EdgesCount(memgraph::storage::EdgeTypeId edge_type) const { + auto found = edge_type_index_.find(edge_type); + if (found != edge_type_index_.end()) return found->second; + return 0; + } + bool LabelIndexExists(memgraph::storage::LabelId label) const { return label_index_.find(label) != label_index_.end(); } @@ -573,6 +581,10 @@ class FakeDbAccessor { return false; } + bool EdgeTypeIndexExists(memgraph::storage::EdgeTypeId edge_type) const { + return edge_type_index_.find(edge_type) != edge_type_index_.end(); + } + std::optional<memgraph::storage::LabelPropertyIndexStats> GetIndexStats( const memgraph::storage::LabelId label, const memgraph::storage::PropertyId property) const { return memgraph::storage::LabelPropertyIndexStats{.statistic = 0, .avg_group_size = 1}; // unique id @@ -594,6 +606,8 @@ class FakeDbAccessor { label_property_index_.emplace_back(label, property, count); } + void SetIndexCount(memgraph::storage::EdgeTypeId edge_type, int64_t count) { edge_type_index_[edge_type] = count; } + memgraph::storage::LabelId NameToLabel(const std::string &name) { auto found = labels_.find(name); if (found != labels_.end()) return found->second; @@ -608,6 +622,8 @@ class FakeDbAccessor { return edge_types_.emplace(name, memgraph::storage::EdgeTypeId::FromUint(edge_types_.size())).first->second; } + memgraph::storage::EdgeTypeId EdgeType(const std::string &name) { return NameToEdgeType(name); } + memgraph::storage::PropertyId NameToProperty(const std::string &name) { auto found = properties_.find(name); if (found != properties_.end()) return found->second; @@ -632,6 +648,7 @@ class FakeDbAccessor { std::unordered_map<memgraph::storage::LabelId, int64_t> label_index_; std::vector<std::tuple<memgraph::storage::LabelId, memgraph::storage::PropertyId, int64_t>> label_property_index_; + std::unordered_map<memgraph::storage::EdgeTypeId, int64_t> edge_type_index_; }; } // namespace memgraph::query::plan diff --git a/tests/unit/storage_v2_decoder_encoder.cpp b/tests/unit/storage_v2_decoder_encoder.cpp index 9b627cb77..15db49b1c 100644 --- a/tests/unit/storage_v2_decoder_encoder.cpp +++ b/tests/unit/storage_v2_decoder_encoder.cpp @@ -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 @@ -337,6 +337,7 @@ TEST_F(DecoderEncoderTest, PropertyValueInvalidMarker) { case memgraph::storage::durability::Marker::SECTION_CONSTRAINTS: case memgraph::storage::durability::Marker::SECTION_DELTA: case memgraph::storage::durability::Marker::SECTION_EPOCH_HISTORY: + case memgraph::storage::durability::Marker::SECTION_EDGE_INDICES: case memgraph::storage::durability::Marker::SECTION_OFFSETS: case memgraph::storage::durability::Marker::DELTA_VERTEX_CREATE: case memgraph::storage::durability::Marker::DELTA_VERTEX_DELETE: @@ -355,6 +356,8 @@ TEST_F(DecoderEncoderTest, PropertyValueInvalidMarker) { case memgraph::storage::durability::Marker::DELTA_LABEL_PROPERTY_INDEX_DROP: case memgraph::storage::durability::Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_SET: case memgraph::storage::durability::Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR: + case memgraph::storage::durability::Marker::DELTA_EDGE_TYPE_INDEX_CREATE: + case memgraph::storage::durability::Marker::DELTA_EDGE_TYPE_INDEX_DROP: case memgraph::storage::durability::Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE: case memgraph::storage::durability::Marker::DELTA_EXISTENCE_CONSTRAINT_DROP: case memgraph::storage::durability::Marker::DELTA_UNIQUE_CONSTRAINT_CREATE: diff --git a/tests/unit/storage_v2_durability_inmemory.cpp b/tests/unit/storage_v2_durability_inmemory.cpp index 54671077f..7794f2ab9 100644 --- a/tests/unit/storage_v2_durability_inmemory.cpp +++ b/tests/unit/storage_v2_durability_inmemory.cpp @@ -69,6 +69,7 @@ class DurabilityTest : public ::testing::TestWithParam<bool> { ONLY_EXTENDED, ONLY_EXTENDED_WITH_BASE_INDICES_AND_CONSTRAINTS, BASE_WITH_EXTENDED, + BASE_WITH_EDGE_TYPE_INDEXED, }; public: @@ -270,6 +271,15 @@ class DurabilityTest : public ::testing::TestWithParam<bool> { if (single_transaction) ASSERT_FALSE(acc->Commit().HasError()); } + void CreateEdgeIndex(memgraph::storage::Storage *store, memgraph::storage::EdgeTypeId edge_type) { + { + // Create edge-type index. + auto unique_acc = store->UniqueAccess(ReplicationRole::MAIN); + ASSERT_FALSE(unique_acc->CreateIndex(edge_type).HasError()); + ASSERT_FALSE(unique_acc->Commit().HasError()); + } + } + void VerifyDataset(memgraph::storage::Storage *store, DatasetType type, bool properties_on_edges, bool verify_info = true) { auto base_label_indexed = store->NameToLabel("base_indexed"); @@ -310,13 +320,19 @@ class DurabilityTest : public ::testing::TestWithParam<bool> { UnorderedElementsAre(std::make_pair(base_label_indexed, property_id), std::make_pair(extended_label_indexed, property_count))); break; + case DatasetType::BASE_WITH_EDGE_TYPE_INDEXED: + ASSERT_THAT(info.label, UnorderedElementsAre(base_label_unindexed)); + ASSERT_THAT(info.label_property, UnorderedElementsAre(std::make_pair(base_label_indexed, property_id))); + ASSERT_THAT(info.edge_type, UnorderedElementsAre(et1)); + break; } } // Verify index statistics { switch (type) { - case DatasetType::ONLY_BASE: { + case DatasetType::ONLY_BASE: + case DatasetType::BASE_WITH_EDGE_TYPE_INDEXED: { const auto l_stats = acc->GetIndexStats(base_label_unindexed); ASSERT_TRUE(l_stats); ASSERT_EQ(l_stats->count, 1); @@ -379,6 +395,7 @@ class DurabilityTest : public ::testing::TestWithParam<bool> { auto info = acc->ListAllConstraints(); switch (type) { case DatasetType::ONLY_BASE: + case DatasetType::BASE_WITH_EDGE_TYPE_INDEXED: ASSERT_THAT(info.existence, UnorderedElementsAre(std::make_pair(base_label_unindexed, property_id))); ASSERT_THAT(info.unique, UnorderedElementsAre( std::make_pair(base_label_unindexed, std::set{property_id, property_extra}))); @@ -402,6 +419,7 @@ class DurabilityTest : public ::testing::TestWithParam<bool> { bool have_base_dataset = false; bool have_extended_dataset = false; + bool have_edge_type_indexed_dataset = false; switch (type) { case DatasetType::ONLY_BASE: case DatasetType::ONLY_BASE_WITH_EXTENDED_INDICES_AND_CONSTRAINTS: @@ -415,6 +433,9 @@ class DurabilityTest : public ::testing::TestWithParam<bool> { have_base_dataset = true; have_extended_dataset = true; break; + case DatasetType::BASE_WITH_EDGE_TYPE_INDEXED: + have_base_dataset = true; + have_edge_type_indexed_dataset = true; } // Verify base dataset. @@ -675,6 +696,19 @@ class DurabilityTest : public ::testing::TestWithParam<bool> { } } + if (have_edge_type_indexed_dataset) { + MG_ASSERT(properties_on_edges, "Edge-type indexing needs --properties-on-edges!"); + // Verify edge-type indices. + { + std::vector<memgraph::storage::EdgeAccessor> edges; + edges.reserve(kNumBaseEdges / 2); + for (auto edge : acc->Edges(et1, memgraph::storage::View::OLD)) { + edges.push_back(edge); + } + ASSERT_EQ(edges.size(), kNumBaseEdges / 2); + } + } + if (verify_info) { auto info = store->GetBaseInfo(); if (have_base_dataset) { @@ -2972,3 +3006,42 @@ TEST_P(DurabilityTest, ConstraintsRecoveryFunctionSetting) { &variant_existence_constraint_creation_func); MG_ASSERT(pval_existence, "Chose wrong type of function for recovery of existence constraint data"); } + +// NOLINTNEXTLINE(hicpp-special-member-functions) +TEST_P(DurabilityTest, EdgeTypeIndexRecovered) { + if (GetParam() == false) { + return; + } + // Create snapshot. + { + memgraph::storage::Config config{.salient.items = {.properties_on_edges = GetParam()}, + .durability = {.storage_directory = storage_directory, .snapshot_on_exit = true}}; + memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)}; + memgraph::dbms::Database db{config, repl_state}; + CreateBaseDataset(db.storage(), GetParam()); + VerifyDataset(db.storage(), DatasetType::ONLY_BASE, GetParam()); + CreateEdgeIndex(db.storage(), db.storage()->NameToEdgeType("base_et1")); + VerifyDataset(db.storage(), DatasetType::BASE_WITH_EDGE_TYPE_INDEXED, GetParam()); + } + + ASSERT_EQ(GetSnapshotsList().size(), 1); + ASSERT_EQ(GetBackupSnapshotsList().size(), 0); + ASSERT_EQ(GetWalsList().size(), 0); + ASSERT_EQ(GetBackupWalsList().size(), 0); + + // Recover snapshot. + memgraph::storage::Config config{.salient.items = {.properties_on_edges = GetParam()}, + .durability = {.storage_directory = storage_directory, .recover_on_startup = true}}; + memgraph::replication::ReplicationState repl_state{memgraph::storage::ReplicationStateRootPath(config)}; + memgraph::dbms::Database db{config, repl_state}; + VerifyDataset(db.storage(), DatasetType::BASE_WITH_EDGE_TYPE_INDEXED, GetParam()); + + // Try to use the storage. + { + auto acc = db.Access(); + auto vertex = acc->CreateVertex(); + auto edge = acc->CreateEdge(&vertex, &vertex, db.storage()->NameToEdgeType("et")); + ASSERT_TRUE(edge.HasValue()); + ASSERT_FALSE(acc->Commit().HasError()); + } +} diff --git a/tests/unit/storage_v2_indices.cpp b/tests/unit/storage_v2_indices.cpp index 8ee053087..23c82313d 100644 --- a/tests/unit/storage_v2_indices.cpp +++ b/tests/unit/storage_v2_indices.cpp @@ -44,6 +44,8 @@ class IndexTest : public testing::Test { this->prop_val = acc->NameToProperty("val"); this->label1 = acc->NameToLabel("label1"); this->label2 = acc->NameToLabel("label2"); + this->edge_type_id1 = acc->NameToEdgeType("edge_type_1"); + this->edge_type_id2 = acc->NameToEdgeType("edge_type_2"); vertex_id = 0; } @@ -61,6 +63,8 @@ class IndexTest : public testing::Test { PropertyId prop_val; LabelId label1; LabelId label2; + EdgeTypeId edge_type_id1; + EdgeTypeId edge_type_id2; VertexAccessor CreateVertex(Storage::Accessor *accessor) { VertexAccessor vertex = accessor->CreateVertex(); @@ -68,11 +72,23 @@ class IndexTest : public testing::Test { return vertex; } + VertexAccessor CreateVertexWithoutProperties(Storage::Accessor *accessor) { + VertexAccessor vertex = accessor->CreateVertex(); + return vertex; + } + + EdgeAccessor CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, Storage::Accessor *accessor) { + auto edge = accessor->CreateEdge(from, to, edge_type); + MG_ASSERT(!edge.HasError()); + MG_ASSERT(!edge->SetProperty(this->prop_id, PropertyValue(vertex_id++)).HasError()); + return edge.GetValue(); + } + template <class TIterable> std::vector<int64_t> GetIds(TIterable iterable, View view = View::OLD) { std::vector<int64_t> ret; - for (auto vertex : iterable) { - ret.push_back(vertex.GetProperty(this->prop_id, view)->ValueInt()); + for (auto item : iterable) { + ret.push_back(item.GetProperty(this->prop_id, view)->ValueInt()); } return ret; } @@ -1292,3 +1308,368 @@ TYPED_TEST(IndexTest, LabelPropertyIndexClearOldDataFromDisk) { ASSERT_EQ(disk_test_utils::GetRealNumberOfEntriesInRocksDB(tx_db), 1); } } + +// NOLINTNEXTLINE(hicpp-special-member-functions) +TYPED_TEST(IndexTest, EdgeTypeIndexCreate) { + if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) { + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_FALSE(acc->EdgeTypeIndexExists(this->edge_type_id1)); + EXPECT_EQ(acc->ListAllIndices().edge_type.size(), 0); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + for (int i = 0; i < 10; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, i % 2 ? this->edge_type_id1 : this->edge_type_id2, acc.get()); + } + ASSERT_NO_ERROR(acc->Commit()); + } + + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9)); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + for (int i = 10; i < 20; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, i % 2 ? this->edge_type_id1 : this->edge_type_id2, acc.get()); + } + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19)); + + acc->AdvanceCommand(); + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19)); + + acc->Abort(); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + for (int i = 10; i < 20; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, i % 2 ? this->edge_type_id1 : this->edge_type_id2, acc.get()); + } + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29)); + + acc->AdvanceCommand(); + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29)); + + ASSERT_NO_ERROR(acc->Commit()); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29)); + + acc->AdvanceCommand(); + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29)); + + ASSERT_NO_ERROR(acc->Commit()); + } + } +} + +// NOLINTNEXTLINE(hicpp-special-member-functions) +TYPED_TEST(IndexTest, EdgeTypeIndexDrop) { + if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) { + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_FALSE(acc->EdgeTypeIndexExists(this->edge_type_id1)); + EXPECT_EQ(acc->ListAllIndices().edge_type.size(), 0); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + for (int i = 0; i < 10; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, i % 2 ? this->edge_type_id1 : this->edge_type_id2, acc.get()); + } + ASSERT_NO_ERROR(acc->Commit()); + } + + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9)); + } + + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->DropIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_FALSE(acc->EdgeTypeIndexExists(this->edge_type_id1)); + EXPECT_EQ(acc->ListAllIndices().label.size(), 0); + } + + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_TRUE(unique_acc->DropIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_FALSE(acc->EdgeTypeIndexExists(this->edge_type_id1)); + EXPECT_EQ(acc->ListAllIndices().label.size(), 0); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + for (int i = 10; i < 20; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, i % 2 ? this->edge_type_id1 : this->edge_type_id2, acc.get()); + } + ASSERT_NO_ERROR(acc->Commit()); + } + + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_TRUE(acc->EdgeTypeIndexExists(this->edge_type_id1)); + EXPECT_THAT(acc->ListAllIndices().edge_type, UnorderedElementsAre(this->edge_type_id1)); + } + + { + auto acc = this->storage->Access(ReplicationRole::MAIN); + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19)); + + acc->AdvanceCommand(); + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19)); + } + } +} + +// NOLINTNEXTLINE(hicpp-special-member-functions) +TYPED_TEST(IndexTest, EdgeTypeIndexBasic) { + // The following steps are performed and index correctness is validated after + // each step: + // 1. Create 10 edges numbered from 0 to 9. + // 2. Add EdgeType1 to odd numbered, and EdgeType2 to even numbered edges. + // 3. Delete even numbered edges. + if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) { + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id2).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + + auto acc = this->storage->Access(ReplicationRole::MAIN); + EXPECT_THAT(acc->ListAllIndices().edge_type, UnorderedElementsAre(this->edge_type_id1, this->edge_type_id2)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), IsEmpty()); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::OLD), View::OLD), IsEmpty()); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), IsEmpty()); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::NEW), View::NEW), IsEmpty()); + + for (int i = 0; i < 10; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, i % 2 ? this->edge_type_id1 : this->edge_type_id2, acc.get()); + } + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), IsEmpty()); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::OLD), View::OLD), IsEmpty()); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::NEW), View::NEW), + UnorderedElementsAre(0, 2, 4, 6, 8)); + + acc->AdvanceCommand(); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::OLD), View::OLD), + UnorderedElementsAre(0, 2, 4, 6, 8)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::NEW), View::NEW), + UnorderedElementsAre(0, 2, 4, 6, 8)); + + for (auto vertex : acc->Vertices(View::OLD)) { + auto edges = vertex.OutEdges(View::OLD)->edges; + for (auto &edge : edges) { + int64_t id = edge.GetProperty(this->prop_id, View::OLD)->ValueInt(); + if (id % 2 == 0) { + ASSERT_NO_ERROR(acc->DetachDelete({}, {&edge}, false)); + } + } + } + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::OLD), View::OLD), + UnorderedElementsAre(0, 2, 4, 6, 8)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::NEW), View::NEW), IsEmpty()); + + acc->AdvanceCommand(); + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::OLD), View::OLD), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::OLD), View::OLD), IsEmpty()); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(1, 3, 5, 7, 9)); + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id2, View::NEW), View::NEW), IsEmpty()); + } +} + +// NOLINTNEXTLINE(hicpp-special-member-functions) +TYPED_TEST(IndexTest, EdgeTypeIndexTransactionalIsolation) { + if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) { + // Check that transactions only see entries they are supposed to see. + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id2).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + + auto acc_before = this->storage->Access(ReplicationRole::MAIN); + auto acc = this->storage->Access(ReplicationRole::MAIN); + auto acc_after = this->storage->Access(ReplicationRole::MAIN); + + for (int i = 0; i < 5; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, this->edge_type_id1, acc.get()); + } + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(0, 1, 2, 3, 4)); + + EXPECT_THAT(this->GetIds(acc_before->Edges(this->edge_type_id1, View::NEW), View::NEW), IsEmpty()); + + EXPECT_THAT(this->GetIds(acc_after->Edges(this->edge_type_id1, View::NEW), View::NEW), IsEmpty()); + + ASSERT_NO_ERROR(acc->Commit()); + + auto acc_after_commit = this->storage->Access(ReplicationRole::MAIN); + + EXPECT_THAT(this->GetIds(acc_before->Edges(this->edge_type_id1, View::NEW), View::NEW), IsEmpty()); + + EXPECT_THAT(this->GetIds(acc_after->Edges(this->edge_type_id1, View::NEW), View::NEW), IsEmpty()); + + EXPECT_THAT(this->GetIds(acc_after_commit->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(0, 1, 2, 3, 4)); + } +} + +// NOLINTNEXTLINE(hicpp-special-member-functions) +TYPED_TEST(IndexTest, EdgeTypeIndexCountEstimate) { + if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) { + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id2).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + + auto acc = this->storage->Access(ReplicationRole::MAIN); + for (int i = 0; i < 20; ++i) { + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + this->CreateEdge(&vertex_from, &vertex_to, i % 3 ? this->edge_type_id1 : this->edge_type_id2, acc.get()); + } + + EXPECT_EQ(acc->ApproximateEdgeCount(this->edge_type_id1), 13); + EXPECT_EQ(acc->ApproximateEdgeCount(this->edge_type_id2), 7); + } +} + +// NOLINTNEXTLINE(hicpp-special-member-functions) +TYPED_TEST(IndexTest, EdgeTypeIndexRepeatingEdgeTypesBetweenSameVertices) { + if constexpr ((std::is_same_v<TypeParam, memgraph::storage::InMemoryStorage>)) { + { + auto unique_acc = this->storage->UniqueAccess(ReplicationRole::MAIN); + EXPECT_FALSE(unique_acc->CreateIndex(this->edge_type_id1).HasError()); + ASSERT_NO_ERROR(unique_acc->Commit()); + } + + auto acc = this->storage->Access(ReplicationRole::MAIN); + auto vertex_from = this->CreateVertexWithoutProperties(acc.get()); + auto vertex_to = this->CreateVertexWithoutProperties(acc.get()); + + for (int i = 0; i < 5; ++i) { + this->CreateEdge(&vertex_from, &vertex_to, this->edge_type_id1, acc.get()); + } + + EXPECT_EQ(acc->ApproximateEdgeCount(this->edge_type_id1), 5); + + EXPECT_THAT(this->GetIds(acc->Edges(this->edge_type_id1, View::NEW), View::NEW), + UnorderedElementsAre(0, 1, 2, 3, 4)); + } +} diff --git a/tests/unit/storage_v2_wal_file.cpp b/tests/unit/storage_v2_wal_file.cpp index 07a35d754..dcb7d3326 100644 --- a/tests/unit/storage_v2_wal_file.cpp +++ b/tests/unit/storage_v2_wal_file.cpp @@ -37,6 +37,10 @@ memgraph::storage::durability::WalDeltaData::Type StorageMetadataOperationToWalD return memgraph::storage::durability::WalDeltaData::Type::LABEL_INDEX_CREATE; case memgraph::storage::durability::StorageMetadataOperation::LABEL_INDEX_DROP: return memgraph::storage::durability::WalDeltaData::Type::LABEL_INDEX_DROP; + case memgraph::storage::durability::StorageMetadataOperation::EDGE_TYPE_INDEX_CREATE: + return memgraph::storage::durability::WalDeltaData::Type::EDGE_INDEX_CREATE; + case memgraph::storage::durability::StorageMetadataOperation::EDGE_TYPE_INDEX_DROP: + return memgraph::storage::durability::WalDeltaData::Type::EDGE_INDEX_DROP; case memgraph::storage::durability::StorageMetadataOperation::LABEL_INDEX_STATS_SET: return memgraph::storage::durability::WalDeltaData::Type::LABEL_INDEX_STATS_SET; case memgraph::storage::durability::StorageMetadataOperation::LABEL_INDEX_STATS_CLEAR: @@ -280,6 +284,41 @@ class DeltaGenerator final { case memgraph::storage::durability::StorageMetadataOperation::UNIQUE_CONSTRAINT_DROP: data.operation_label_properties.label = label; data.operation_label_properties.properties = properties; + break; + case memgraph::storage::durability::StorageMetadataOperation::EDGE_TYPE_INDEX_CREATE: + case memgraph::storage::durability::StorageMetadataOperation::EDGE_TYPE_INDEX_DROP: + MG_ASSERT(false, "Invalid function call!"); + } + data_.emplace_back(timestamp_, data); + } + } + + void AppendEdgeTypeOperation(memgraph::storage::durability::StorageMetadataOperation operation, + const std::string &edge_type) { + auto edge_type_id = memgraph::storage::EdgeTypeId::FromUint(mapper_.NameToId(edge_type)); + wal_file_.AppendOperation(operation, edge_type_id, timestamp_); + if (valid_) { + UpdateStats(timestamp_, 1); + memgraph::storage::durability::WalDeltaData data; + data.type = StorageMetadataOperationToWalDeltaDataType(operation); + switch (operation) { + case memgraph::storage::durability::StorageMetadataOperation::EDGE_TYPE_INDEX_CREATE: + case memgraph::storage::durability::StorageMetadataOperation::EDGE_TYPE_INDEX_DROP: + data.operation_edge_type.edge_type = edge_type; + break; + case memgraph::storage::durability::StorageMetadataOperation::LABEL_INDEX_CREATE: + case memgraph::storage::durability::StorageMetadataOperation::LABEL_INDEX_DROP: + case memgraph::storage::durability::StorageMetadataOperation::LABEL_INDEX_STATS_CLEAR: + case memgraph::storage::durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_CLEAR: + case memgraph::storage::durability::StorageMetadataOperation::LABEL_INDEX_STATS_SET: + case memgraph::storage::durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_CREATE: + case memgraph::storage::durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_DROP: + case memgraph::storage::durability::StorageMetadataOperation::EXISTENCE_CONSTRAINT_CREATE: + case memgraph::storage::durability::StorageMetadataOperation::EXISTENCE_CONSTRAINT_DROP:; + case memgraph::storage::durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_SET: + case memgraph::storage::durability::StorageMetadataOperation::UNIQUE_CONSTRAINT_CREATE: + case memgraph::storage::durability::StorageMetadataOperation::UNIQUE_CONSTRAINT_DROP: + MG_ASSERT(false, "Invalid function call!"); } data_.emplace_back(timestamp_, data); } From 2a5388cea9e5994c121a37f4a1fe96933d884f33 Mon Sep 17 00:00:00 2001 From: Antonio Filipovic <61245998+antoniofilipovic@users.noreply.github.com> Date: Fri, 8 Mar 2024 16:16:30 +0100 Subject: [PATCH 17/18] Add tests to verify log store works properly (#1794) --- .../high_availability/distributed_coords.py | 506 ++++++++++++++++++ 1 file changed, 506 insertions(+) diff --git a/tests/e2e/high_availability/distributed_coords.py b/tests/e2e/high_availability/distributed_coords.py index 3b0964111..7dc3ef238 100644 --- a/tests/e2e/high_availability/distributed_coords.py +++ b/tests/e2e/high_availability/distributed_coords.py @@ -175,6 +175,7 @@ def get_instances_description_no_setup(): "--raft-server-port=10111", ], "log_file": "coordinator1.log", + "data_directory": f"{TEMP_DIR}/coordinator_1", "setup_queries": [], }, "coordinator_2": { @@ -187,6 +188,7 @@ def get_instances_description_no_setup(): "--raft-server-port=10112", ], "log_file": "coordinator2.log", + "data_directory": f"{TEMP_DIR}/coordinator_2", "setup_queries": [], }, "coordinator_3": { @@ -199,6 +201,7 @@ def get_instances_description_no_setup(): "--raft-server-port=10113", ], "log_file": "coordinator3.log", + "data_directory": f"{TEMP_DIR}/coordinator_3", "setup_queries": [], }, } @@ -530,6 +533,8 @@ def test_old_main_comes_back_on_new_leader_as_main(): # 4. Start the old main instance # 5. Run SHOW INSTANCES on the new leader and check that the old main instance is main once again + safe_execute(shutil.rmtree, TEMP_DIR) + inner_memgraph_instances = get_instances_description_no_setup() interactive_mg_runner.start_all(inner_memgraph_instances) @@ -623,6 +628,7 @@ def test_old_main_comes_back_on_new_leader_as_main(): def test_registering_4_coords(): # Goal of this test is to assure registering of multiple coordinators in row works + safe_execute(shutil.rmtree, TEMP_DIR) INSTANCES_DESCRIPTION = { "instance_1": { "args": [ @@ -743,5 +749,505 @@ def test_registering_4_coords(): mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) +def test_registering_coord_log_store(): + # Goal of this test is to assure registering a bunch of instances and de-registering works properly + # w.r.t nuRaft log + # 1. Start basic instances # 3 logs + # 2. Check all is there + # 3. Create 3 additional instances and add them to cluster # 3 logs -> 1st snapshot + # 4. Check everything is there + # 5. Set main # 1 log + # 6. Check correct state + # 7. Drop 2 new instances # 2 logs + # 8. Check correct state + # 9. Drop 1 new instance # 1 log -> 2nd snapshot + # 10. Check correct state + safe_execute(shutil.rmtree, TEMP_DIR) + + INSTANCES_DESCRIPTION = { + "instance_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7687", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10011", + ], + "log_file": "instance_1.log", + "data_directory": f"{TEMP_DIR}/instance_1", + "setup_queries": [], + }, + "instance_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7688", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10012", + ], + "log_file": "instance_2.log", + "data_directory": f"{TEMP_DIR}/instance_2", + "setup_queries": [], + }, + "instance_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7689", + "--log-level", + "TRACE", + "--coordinator-server-port", + "10013", + ], + "log_file": "instance_3.log", + "data_directory": f"{TEMP_DIR}/instance_3", + "setup_queries": [], + }, + "coordinator_1": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7690", + "--log-level=TRACE", + "--raft-server-id=1", + "--raft-server-port=10111", + ], + "log_file": "coordinator1.log", + "setup_queries": [], + }, + "coordinator_2": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7691", + "--log-level=TRACE", + "--raft-server-id=2", + "--raft-server-port=10112", + ], + "log_file": "coordinator2.log", + "setup_queries": [], + }, + "coordinator_3": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7692", + "--log-level=TRACE", + "--raft-server-id=3", + "--raft-server-port=10113", + ], + "log_file": "coordinator3.log", + "setup_queries": [], + }, + "coordinator_4": { + "args": [ + "--experimental-enabled=high-availability", + "--bolt-port", + "7693", + "--log-level=TRACE", + "--raft-server-id=4", + "--raft-server-port=10114", + ], + "log_file": "coordinator4.log", + "setup_queries": [ + "ADD COORDINATOR 1 ON '127.0.0.1:10111';", + "ADD COORDINATOR 2 ON '127.0.0.1:10112';", + "ADD COORDINATOR 3 ON '127.0.0.1:10113';", + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002'", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003'", + ], + }, + } + assert "SET INSTANCE instance_3 TO MAIN" not in INSTANCES_DESCRIPTION["coordinator_4"]["setup_queries"] + + # 1 + interactive_mg_runner.start_all(INSTANCES_DESCRIPTION) + + # 2 + coord_cursor = connect(host="localhost", port=7693).cursor() + + def retrieve_data_show_repl_cluster(): + return sorted(list(execute_and_fetch_all(coord_cursor, "SHOW INSTANCES;"))) + + coordinators = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ("coordinator_4", "127.0.0.1:10114", "", "unknown", "coordinator"), + ] + + basic_instances = [ + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "replica"), + ] + + expected_data_on_coord = [] + expected_data_on_coord.extend(coordinators) + expected_data_on_coord.extend(basic_instances) + + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) + + # 3 + instances_ports_added = [10011, 10012, 10013] + bolt_port_id = 7700 + coord_port_id = 10014 + + additional_instances = [] + for i in range(4, 7): + instance_name = f"instance_{i}" + args_desc = [ + "--experimental-enabled=high-availability", + "--log-level=TRACE", + ] + + bolt_port = f"--bolt-port={bolt_port_id}" + + coord_server_port = f"--coordinator-server-port={coord_port_id}" + + args_desc.append(bolt_port) + args_desc.append(coord_server_port) + + instance_description = { + "args": args_desc, + "log_file": f"instance_{i}.log", + "data_directory": f"{TEMP_DIR}/instance_{i}", + "setup_queries": [], + } + + full_instance_desc = {instance_name: instance_description} + interactive_mg_runner.start(full_instance_desc, instance_name) + repl_port_id = coord_port_id - 10 + assert repl_port_id < 10011, "Wrong test setup, repl port must be smaller than smallest coord port id" + + execute_and_fetch_all( + coord_cursor, + f"REGISTER INSTANCE {instance_name} ON '127.0.0.1:{coord_port_id}' WITH '127.0.0.1:{repl_port_id}'", + ) + + additional_instances.append((f"{instance_name}", "", f"127.0.0.1:{coord_port_id}", "up", "replica")) + instances_ports_added.append(coord_port_id) + coord_port_id += 1 + bolt_port_id += 1 + + # 4 + expected_data_on_coord.extend(additional_instances) + + mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster) + + # 5 + execute_and_fetch_all(coord_cursor, "SET INSTANCE instance_3 TO MAIN") + + # 6 + basic_instances.pop() + basic_instances.append(("instance_3", "", "127.0.0.1:10013", "up", "main")) + + new_expected_data_on_coordinator = [] + + new_expected_data_on_coordinator.extend(coordinators) + new_expected_data_on_coordinator.extend(basic_instances) + new_expected_data_on_coordinator.extend(additional_instances) + + mg_sleep_and_assert(new_expected_data_on_coordinator, retrieve_data_show_repl_cluster) + + # 7 + for i in range(6, 4, -1): + execute_and_fetch_all(coord_cursor, f"UNREGISTER INSTANCE instance_{i};") + additional_instances.pop() + + new_expected_data_on_coordinator = [] + new_expected_data_on_coordinator.extend(coordinators) + new_expected_data_on_coordinator.extend(basic_instances) + new_expected_data_on_coordinator.extend(additional_instances) + + # 8 + mg_sleep_and_assert(new_expected_data_on_coordinator, retrieve_data_show_repl_cluster) + + # 9 + + new_expected_data_on_coordinator = [] + new_expected_data_on_coordinator.extend(coordinators) + new_expected_data_on_coordinator.extend(basic_instances) + + execute_and_fetch_all(coord_cursor, f"UNREGISTER INSTANCE instance_4;") + + # 10 + mg_sleep_and_assert(new_expected_data_on_coordinator, retrieve_data_show_repl_cluster) + + +def test_multiple_failovers_in_row_no_leadership_change(): + # Goal of this test is to assure multiple failovers in row work without leadership change + # 1. Start basic instances + # 2. Check all is there + # 3. Kill MAIN (instance_3) + # 4. Expect failover (instance_1) + # 5. Kill instance_1 + # 6. Expect failover instance_2 + # 7. Start instance_3 + # 8. Expect instance_3 and instance_2 (MAIN) up + # 9. Kill instance_2 + # 10. Expect instance_3 MAIN + # 11. Write some data on instance_3 + # 12. Start instance_2 and instance_1 + # 13. Expect instance_1 and instance2 to be up and cluster to have correct state + # 13. Expect data to be replicated + + # 1 + inner_memgraph_instances = get_instances_description_no_setup() + interactive_mg_runner.start_all(inner_memgraph_instances, keep_directories=False) + + coord_cursor_3 = connect(host="localhost", port=7692).cursor() + + setup_queries = [ + "ADD COORDINATOR 1 ON '127.0.0.1:10111'", + "ADD COORDINATOR 2 ON '127.0.0.1:10112'", + "REGISTER INSTANCE instance_1 ON '127.0.0.1:10011' WITH '127.0.0.1:10001'", + "REGISTER INSTANCE instance_2 ON '127.0.0.1:10012' WITH '127.0.0.1:10002'", + "REGISTER INSTANCE instance_3 ON '127.0.0.1:10013' WITH '127.0.0.1:10003'", + "SET INSTANCE instance_3 TO MAIN", + ] + + for query in setup_queries: + execute_and_fetch_all(coord_cursor_3, query) + + # 2 + + def get_func_show_instances(cursor): + def show_instances_follower_coord(): + return sorted(list(execute_and_fetch_all(cursor, "SHOW INSTANCES;"))) + + return show_instances_follower_coord + + coordinator_data = [ + ("coordinator_1", "127.0.0.1:10111", "", "unknown", "coordinator"), + ("coordinator_2", "127.0.0.1:10112", "", "unknown", "coordinator"), + ("coordinator_3", "127.0.0.1:10113", "", "unknown", "coordinator"), + ] + + leader_data = [] + leader_data.extend(coordinator_data) + leader_data.extend( + [ + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + ) + + follower_data = [] + follower_data.extend(coordinator_data) + follower_data.extend( + [ + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), + ] + ) + + coord_cursor_1 = connect(host="localhost", port=7690).cursor() + coord_cursor_2 = connect(host="localhost", port=7691).cursor() + + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_1)) + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_2)) + mg_sleep_and_assert_collection(leader_data, get_func_show_instances(coord_cursor_3)) + + # 3 + + interactive_mg_runner.kill(inner_memgraph_instances, "instance_3") + + # 4 + + leader_data = [] + leader_data.extend(coordinator_data) + leader_data.extend( + [ + ("instance_1", "", "127.0.0.1:10011", "up", "main"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ] + ) + + follower_data = [] + follower_data.extend(coordinator_data) + follower_data.extend( + [ + ("instance_1", "", "", "unknown", "main"), + ("instance_2", "", "", "unknown", "replica"), + ( + "instance_3", + "", + "", + "unknown", + "main", + ), # TODO(antoniofilipovic) change to unknown after PR with transitions + ] + ) + + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_1)) + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_2)) + mg_sleep_and_assert_collection(leader_data, get_func_show_instances(coord_cursor_3)) + + # 5 + interactive_mg_runner.kill(inner_memgraph_instances, "instance_1") + + # 6 + leader_data = [] + leader_data.extend(coordinator_data) + leader_data.extend( + [ + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "down", "unknown"), + ] + ) + + follower_data = [] + follower_data.extend(coordinator_data) + follower_data.extend( + [ + ("instance_1", "", "", "unknown", "main"), + ("instance_2", "", "", "unknown", "main"), # TODO(antoniofilipovic) change to unknown + ("instance_3", "", "", "unknown", "main"), # TODO(antoniofilipovic) change to unknown + ] + ) + + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_1)) + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_2)) + mg_sleep_and_assert_collection(leader_data, get_func_show_instances(coord_cursor_3)) + + # 7 + + interactive_mg_runner.start(inner_memgraph_instances, "instance_3") + + # 8 + + leader_data = [] + leader_data.extend(coordinator_data) + leader_data.extend( + [ + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "up", "main"), + ("instance_3", "", "127.0.0.1:10013", "up", "replica"), + ] + ) + + follower_data = [] + follower_data.extend(coordinator_data) + follower_data.extend( + [ + ("instance_1", "", "", "unknown", "main"), # TODO(antoniofilipovic) change to unknown + ("instance_2", "", "", "unknown", "main"), + ("instance_3", "", "", "unknown", "replica"), + ] + ) + + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_1)) + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_2)) + mg_sleep_and_assert_collection(leader_data, get_func_show_instances(coord_cursor_3)) + + # 9 + interactive_mg_runner.kill(inner_memgraph_instances, "instance_2") + + # 10 + leader_data = [] + leader_data.extend(coordinator_data) + leader_data.extend( + [ + ("instance_1", "", "127.0.0.1:10011", "down", "unknown"), + ("instance_2", "", "127.0.0.1:10012", "down", "unknown"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + ) + + follower_data = [] + follower_data.extend(coordinator_data) + follower_data.extend( + [ + ("instance_1", "", "", "unknown", "main"), # TODO(antoniofilipovic) change to unknown + ("instance_2", "", "", "unknown", "main"), # TODO(antoniofilipovic) change to unknown + ("instance_3", "", "", "unknown", "main"), + ] + ) + + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_1)) + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_2)) + mg_sleep_and_assert_collection(leader_data, get_func_show_instances(coord_cursor_3)) + + # 11 + + instance_3_cursor = connect(port=7689, host="localhost").cursor() + + with pytest.raises(Exception) as e: + execute_and_fetch_all(instance_3_cursor, "CREATE ();") + assert "At least one SYNC replica has not confirmed committing last transaction." in str(e.value) + + # 12 + interactive_mg_runner.start(inner_memgraph_instances, "instance_1") + interactive_mg_runner.start(inner_memgraph_instances, "instance_2") + + # 13 + leader_data = [] + leader_data.extend(coordinator_data) + leader_data.extend( + [ + ("instance_1", "", "127.0.0.1:10011", "up", "replica"), + ("instance_2", "", "127.0.0.1:10012", "up", "replica"), + ("instance_3", "", "127.0.0.1:10013", "up", "main"), + ] + ) + + follower_data = [] + follower_data.extend(coordinator_data) + follower_data.extend( + [ + ("instance_1", "", "", "unknown", "replica"), + ("instance_2", "", "", "unknown", "replica"), + ("instance_3", "", "", "unknown", "main"), + ] + ) + + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_1)) + mg_sleep_and_assert_collection(follower_data, get_func_show_instances(coord_cursor_2)) + mg_sleep_and_assert_collection(leader_data, get_func_show_instances(coord_cursor_3)) + + # 14. + + def show_replicas(): + return sorted(list(execute_and_fetch_all(instance_3_cursor, "SHOW REPLICAS;"))) + + replicas = [ + ( + "instance_1", + "127.0.0.1:10001", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 2, "behind": 0, "status": "ready"}}, + ), + ( + "instance_2", + "127.0.0.1:10002", + "sync", + {"ts": 0, "behind": None, "status": "ready"}, + {"memgraph": {"ts": 2, "behind": 0, "status": "ready"}}, + ), + ] + mg_sleep_and_assert_collection(replicas, show_replicas) + + def get_vertex_count_func(cursor): + def get_vertex_count(): + return execute_and_fetch_all(cursor, "MATCH (n) RETURN count(n)")[0][0] + + return get_vertex_count + + mg_sleep_and_assert(1, get_vertex_count_func(connect(port=7687, host="localhost").cursor())) + + mg_sleep_and_assert(1, get_vertex_count_func(connect(port=7688, host="localhost").cursor())) + + if __name__ == "__main__": sys.exit(pytest.main([__file__, "-rA"])) From 1c71d605ff1585176b9330486d9d9802dc7c9239 Mon Sep 17 00:00:00 2001 From: Aidar Samerkhanov <aidar.samerkhanov@memgraph.io> Date: Sat, 9 Mar 2024 07:20:40 +0400 Subject: [PATCH 18/18] Fix PatternVisitor compilation in toolchain-v5 (#1803) --- src/query/plan/preprocess.cpp | 4 ++++ src/query/plan/preprocess.hpp | 4 +--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/query/plan/preprocess.cpp b/src/query/plan/preprocess.cpp index 1554ce8ce..ca605a46a 100644 --- a/src/query/plan/preprocess.cpp +++ b/src/query/plan/preprocess.cpp @@ -660,6 +660,10 @@ void PatternVisitor::Visit(Exists &op) { std::vector<FilterMatching> PatternVisitor::getFilterMatchings() { return filter_matchings_; } +std::vector<PatternComprehensionMatching> PatternVisitor::getPatternComprehensionMatchings() { + return pattern_comprehension_matchings_; +} + static void ParseForeach(query::Foreach &foreach, SingleQueryPart &query_part, AstStorage &storage, SymbolTable &symbol_table) { for (auto *clause : foreach.clauses_) { diff --git a/src/query/plan/preprocess.hpp b/src/query/plan/preprocess.hpp index b1ad253c4..5d4e2e8d2 100644 --- a/src/query/plan/preprocess.hpp +++ b/src/query/plan/preprocess.hpp @@ -239,9 +239,7 @@ class PatternVisitor : public ExpressionVisitor<void> { void Visit(PatternComprehension &op) override; std::vector<FilterMatching> getFilterMatchings(); - std::vector<PatternComprehensionMatching> getPatternComprehensionMatchings() { - return pattern_comprehension_matchings_; - } + std::vector<PatternComprehensionMatching> getPatternComprehensionMatchings(); SymbolTable &symbol_table_; AstStorage &storage_;