From 7b78665cd896a063bdc81af1b168d6b69d6d55d0 Mon Sep 17 00:00:00 2001 From: Jure Bajic Date: Wed, 27 Apr 2022 10:13:16 +0200 Subject: [PATCH 1/4] Implement Bolt over WebSocket with asio * Replace server implementation with asio * Add support for bolt over WebSocket --- CMakeLists.txt | 3 +- src/communication/context.cpp | 10 +- src/communication/v2/listener.hpp | 135 ++++++ src/communication/v2/pool.hpp | 68 +++ src/communication/v2/server.hpp | 128 ++++++ src/communication/v2/session.hpp | 513 +++++++++++++++++++++++ src/communication/websocket/listener.hpp | 2 - src/communication/websocket/server.hpp | 2 - src/communication/websocket/session.hpp | 2 - src/memgraph.cpp | 34 +- 10 files changed, 872 insertions(+), 25 deletions(-) create mode 100644 src/communication/v2/listener.hpp create mode 100644 src/communication/v2/pool.hpp create mode 100644 src/communication/v2/server.hpp create mode 100644 src/communication/v2/session.hpp diff --git a/CMakeLists.txt b/CMakeLists.txt index dc5fc204f..916389424 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -184,7 +184,8 @@ set(CMAKE_CXX_STANDARD_REQUIRED ON) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wall \ -Werror=switch -Werror=switch-bool -Werror=return-type \ -Werror=return-stack-address \ - -Wno-c99-designator") + -Wno-c99-designator \ + -DBOOST_ASIO_USE_TS_EXECUTOR_AS_DEFAULT") # Don't omit frame pointer in RelWithDebInfo, for additional callchain debug. set(CMAKE_CXX_FLAGS_RELWITHDEBINFO diff --git a/src/communication/context.cpp b/src/communication/context.cpp index b3e58824b..53cb4586b 100644 --- a/src/communication/context.cpp +++ b/src/communication/context.cpp @@ -78,14 +78,18 @@ bool ClientContext::use_ssl() { return use_ssl_; } ServerContext::ServerContext(const std::string &key_file, const std::string &cert_file, const std::string &ca_file, bool verify_peer) { - ctx_.emplace(boost::asio::ssl::context::tls_server); + namespace ssl = boost::asio::ssl; + ctx_.emplace(ssl::context::tls_server); + // NOLINTNEXTLINE(hicpp-signed-bitwise) + ctx_->set_options(ssl::context::default_workarounds | ssl::context::no_sslv2 | ssl::context::no_sslv3 | + ssl::context::single_dh_use); ctx_->set_default_verify_paths(); // TODO: add support for encrypted private keys // TODO: add certificate revocation list (CRL) boost::system::error_code ec; ctx_->use_certificate_chain_file(cert_file, ec); MG_ASSERT(!ec, "Couldn't load server certificate from file: {}", cert_file); - ctx_->use_private_key_file(key_file, boost::asio::ssl::context::pem, ec); + ctx_->use_private_key_file(key_file, ssl::context::pem, ec); MG_ASSERT(!ec, "Couldn't load server private key from file: {}", key_file); ctx_->set_options(SSL_OP_NO_SSLv3, ec); @@ -100,7 +104,7 @@ ServerContext::ServerContext(const std::string &key_file, const std::string &cer if (verify_peer) { // Enable verification of the client certificate. // NOLINTNEXTLINE(hicpp-signed-bitwise) - ctx_->set_verify_mode(boost::asio::ssl::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert, ec); + ctx_->set_verify_mode(ssl::verify_peer | ssl::verify_fail_if_no_peer_cert, ec); MG_ASSERT(!ec, "Setting SSL verification mode failed!"); } } diff --git a/src/communication/v2/listener.hpp b/src/communication/v2/listener.hpp new file mode 100644 index 000000000..bd26bea12 --- /dev/null +++ b/src/communication/v2/listener.hpp @@ -0,0 +1,135 @@ +// Copyright 2022 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 +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "communication/context.hpp" +#include "communication/v2/pool.hpp" +#include "communication/v2/session.hpp" +#include "utils/spin_lock.hpp" +#include "utils/synchronized.hpp" + +namespace memgraph::communication::v2 { + +template +class Listener final : public std::enable_shared_from_this> { + using tcp = boost::asio::ip::tcp; + using SessionHandler = Session; + using std::enable_shared_from_this>::shared_from_this; + + public: + Listener(const Listener &) = delete; + Listener(Listener &&) = delete; + Listener &operator=(const Listener &) = delete; + Listener &operator=(Listener &&) = delete; + ~Listener() {} + + template + static std::shared_ptr Create(Args &&...args) { + return std::shared_ptr{new Listener(std::forward(args)...)}; + } + + void Start() { DoAccept(); } + + bool IsRunning() const noexcept { return alive_.load(std::memory_order_relaxed); } + + private: + Listener(boost::asio::io_context &io_context, TSessionData *data, ServerContext *server_context, + tcp::endpoint &endpoint, const std::string_view service_name, const uint64_t inactivity_timeout_sec) + : io_context_(io_context), + data_(data), + server_context_(server_context), + acceptor_(io_context_), + endpoint_{endpoint}, + service_name_{service_name}, + inactivity_timeout_{inactivity_timeout_sec} { + boost::system::error_code ec; + // Open the acceptor + acceptor_.open(endpoint.protocol(), ec); + if (ec) { + OnError(ec, "open"); + return; + } + + // Allow address reuse + acceptor_.set_option(boost::asio::socket_base::reuse_address(true), ec); + if (ec) { + OnError(ec, "set_option"); + return; + } + + // Bind to the server address + acceptor_.bind(endpoint, ec); + if (ec) { + spdlog::error( + utils::MessageWithLink("Cannot bind to socket on endpoint {}.", endpoint, "https://memgr.ph/socket")); + OnError(ec, "bind"); + return; + } + + acceptor_.listen(boost::asio::socket_base::max_listen_connections, ec); + if (ec) { + OnError(ec, "listen"); + return; + } + } + + void DoAccept() { + acceptor_.async_accept(io_context_, + [shared_this = shared_from_this()](auto ec, boost::asio::ip::tcp::socket &&socket) { + shared_this->OnAccept(ec, std::move(socket)); + }); + } + + void OnAccept(boost::system::error_code ec, tcp::socket socket) { + if (ec) { + return OnError(ec, "accept"); + } + + auto session = SessionHandler::Create(std::move(socket), data_, *server_context_, endpoint_, inactivity_timeout_, + service_name_); + session->Start(); + DoAccept(); + } + + void OnError(const boost::system::error_code &ec, const std::string_view what) { + spdlog::error("Listener failed on {}: {}", what, ec.message()); + alive_.store(false, std::memory_order_relaxed); + } + + boost::asio::io_context &io_context_; + TSessionData *data_; + ServerContext *server_context_; + tcp::acceptor acceptor_; + + tcp::endpoint endpoint_; + std::string_view service_name_; + std::chrono::seconds inactivity_timeout_; + + std::atomic alive_; +}; +} // namespace memgraph::communication::v2 diff --git a/src/communication/v2/pool.hpp b/src/communication/v2/pool.hpp new file mode 100644 index 000000000..f29675aef --- /dev/null +++ b/src/communication/v2/pool.hpp @@ -0,0 +1,68 @@ +// Copyright 2022 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 +#include +#include + +#include +#include + +#include "utils/logging.hpp" + +namespace memgraph::communication::v2 { + +class IOContextThreadPool final { + private: + using IOContext = boost::asio::io_context; + using IOContextGuard = boost::asio::executor_work_guard; + + public: + explicit IOContextThreadPool(size_t pool_size) : guard_{io_context_.get_executor()}, pool_size_{pool_size} { + MG_ASSERT(pool_size != 0, "Pool size must be greater then 0!"); + } + + IOContextThreadPool(const IOContextThreadPool &) = delete; + IOContextThreadPool &operator=(const IOContextThreadPool &) = delete; + IOContextThreadPool(IOContextThreadPool &&) = delete; + IOContextThreadPool &operator=(IOContextThreadPool &&) = delete; + ~IOContextThreadPool() = default; + + void Run() { + background_threads_.reserve(pool_size_); + for (size_t i = 0; i < pool_size_; ++i) { + background_threads_.emplace_back([this]() { io_context_.run(); }); + } + running_ = true; + } + + void Shutdown() { + io_context_.stop(); + running_ = false; + } + + void AwaitShutdown() { background_threads_.clear(); } + + bool IsRunning() const noexcept { return running_; } + + IOContext &GetIOContext() noexcept { return io_context_; } + + private: + /// The pool of io_context. + IOContext io_context_; + IOContextGuard guard_; + size_t pool_size_; + std::vector background_threads_; + bool running_{false}; +}; +} // namespace memgraph::communication::v2 diff --git a/src/communication/v2/server.hpp b/src/communication/v2/server.hpp new file mode 100644 index 000000000..0165035b4 --- /dev/null +++ b/src/communication/v2/server.hpp @@ -0,0 +1,128 @@ +// Copyright 2022 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 +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include "communication/context.hpp" +#include "communication/init.hpp" +#include "communication/v2/listener.hpp" +#include "communication/v2/pool.hpp" +#include "utils/logging.hpp" +#include "utils/message.hpp" +#include "utils/thread.hpp" + +namespace memgraph::communication::v2 { + +using Socket = boost::asio::ip::tcp::socket; +using ServerEndpoint = boost::asio::ip::tcp::endpoint; +/** + * Communication server. + * + * Listens for incoming connections on the server port and assigns them to the + * connection listener. The listener and session are implemented using asio + * async model. Currently the implemented model is thread per core model + * opposed to io_context per core. The reasoning for opting for the former model + * is the robustness to the multiple resource demanding queries that can be split + * across multiple threads, and then a single thread would not block io_context, + * unlike in the latter model where it is possible that thread that accepts + * request is being blocked by demanding query. + * All logic is contained within handlers that are being dispatched + * on a single strand per session. The only exception is write which is + * synchronous since the nature of the clients conenction is synchronous as + * well. + * + * Current Server architecture: + * incoming connection -> server -> listener -> session + + * + * @tparam TSession the server can handle different Sessions, each session + * represents a different protocol so the same network infrastructure + * can be used for handling different protocols + * @tparam TSessionData the class with objects that will be forwarded to the + * session + */ +template +class Server final { + using ServerHandler = Server; + + public: + /** + * Constructs and binds server to endpoint, operates on session data and + * invokes workers_count workers + */ + Server(ServerEndpoint &endpoint, TSessionData *session_data, ServerContext *server_context, + const int inactivity_timeout_sec, const std::string_view service_name, + size_t workers_count = std::thread::hardware_concurrency()) + : endpoint_{endpoint}, + service_name_{service_name}, + context_thread_pool_{workers_count}, + listener_{Listener::Create(context_thread_pool_.GetIOContext(), session_data, + server_context, endpoint_, service_name_, + inactivity_timeout_sec)} {} + + ~Server() { MG_ASSERT(!IsRunning(), "Server wasn't shutdown properly"); } + + Server(const Server &) = delete; + Server(Server &&) = delete; + Server &operator=(const Server &) = delete; + Server &operator=(Server &&) = delete; + + const auto &Endpoint() const { + MG_ASSERT(IsRunning(), "You can't get the server endpoint when it's not running!"); + return endpoint_; + } + + bool Start() { + if (IsRunning()) { + spdlog::error("The server is already running"); + return false; + } + listener_->Start(); + + spdlog::info("{} server is fully armed and operational", service_name_); + spdlog::info("{} listening on {}", service_name_, endpoint_.address()); + context_thread_pool_.Run(); + + return true; + } + + void Shutdown() { + context_thread_pool_.Shutdown(); + spdlog::info("{} shutting down...", service_name_); + } + + void AwaitShutdown() { context_thread_pool_.AwaitShutdown(); } + + bool IsRunning() const noexcept { return context_thread_pool_.IsRunning() && listener_->IsRunning(); } + + private: + ServerEndpoint endpoint_; + std::string service_name_; + + IOContextThreadPool context_thread_pool_; + std::shared_ptr> listener_; +}; + +} // namespace memgraph::communication::v2 diff --git a/src/communication/v2/session.hpp b/src/communication/v2/session.hpp new file mode 100644 index 000000000..0add8244d --- /dev/null +++ b/src/communication/v2/session.hpp @@ -0,0 +1,513 @@ +// Copyright 2022 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "communication/context.hpp" +#include "communication/exceptions.hpp" +#include "utils/logging.hpp" +#include "utils/variant_helpers.hpp" + +namespace memgraph::communication::v2 { + +/** + * This is used to provide input to user Sessions. All Sessions used with the + * network stack should use this class as their input stream. + */ +using InputStream = communication::Buffer::ReadEnd; +using tcp = boost::asio::ip::tcp; + +/** + * This is used to provide output from user Sessions. All Sessions used with the + * network stack should use this class for their output stream. + */ +class OutputStream final { + public: + explicit OutputStream(std::function write_function) + : write_function_(write_function) {} + + OutputStream(const OutputStream &) = delete; + OutputStream(OutputStream &&) = delete; + OutputStream &operator=(const OutputStream &) = delete; + OutputStream &operator=(OutputStream &&) = delete; + ~OutputStream() = default; + + bool Write(const uint8_t *data, size_t len, bool have_more = false) { return write_function_(data, len, have_more); } + + bool Write(const std::string &str, bool have_more = false) { + return Write(reinterpret_cast(str.data()), str.size(), have_more); + } + + private: + std::function write_function_; +}; + +/** + * This class is used internally in the communication stack to handle all user + * Websocket Sessions. It handles socket ownership, inactivity timeout and protocol + * wrapping. + */ +template +class WebsocketSession : public std::enable_shared_from_this> { + using WebSocket = boost::beast::websocket::stream; + using std::enable_shared_from_this>::shared_from_this; + + public: + template + static std::shared_ptr Create(Args &&...args) { + return std::shared_ptr(new WebsocketSession(std::forward(args)...)); + } + + // Start the asynchronous accept operation + template + void DoAccept(boost::beast::http::request> req) { + execution_active_ = true; + // Set suggested timeout settings for the websocket + ws_.set_option(boost::beast::websocket::stream_base::timeout::suggested(boost::beast::role_type::server)); + boost::asio::socket_base::keep_alive option(true); + + // Set a decorator to change the Server of the handshake + ws_.set_option(boost::beast::websocket::stream_base::decorator([](boost::beast::websocket::response_type &res) { + res.set(boost::beast::http::field::server, std::string("Memgraph Bolt WS")); + res.set(boost::beast::http::field::sec_websocket_protocol, "binary"); + })); + ws_.binary(true); + + // Accept the websocket handshake + ws_.async_accept( + req, boost::asio::bind_executor(strand_, std::bind_front(&WebsocketSession::OnAccept, shared_from_this()))); + } + + bool Write(const uint8_t *data, size_t len) { + if (!IsConnected()) { + return false; + } + + boost::system::error_code ec; + ws_.write(boost::asio::buffer(data, len), ec); + if (ec) { + OnError(ec, "write"); + return false; + } + return true; + } + + private: + // Take ownership of the socket + explicit WebsocketSession(tcp::socket &&socket, TSessionData *data, tcp::endpoint endpoint, + std::string_view service_name) + : ws_(std::move(socket)), + strand_{boost::asio::make_strand(ws_.get_executor())}, + output_stream_([this](const uint8_t *data, size_t len, bool /*have_more*/) { return Write(data, len); }), + session_(data, endpoint, input_buffer_.read_end(), &output_stream_), + endpoint_{endpoint}, + remote_endpoint_{ws_.next_layer().socket().remote_endpoint()}, + service_name_{service_name} {} + + void OnAccept(boost::beast::error_code ec) { + if (ec) { + return OnError(ec, "accept"); + } + + // Read a message + DoRead(); + } + + void DoRead() { + if (!IsConnected()) { + return; + } + // Read a message into our buffer + auto buffer = input_buffer_.write_end()->Allocate(); + ws_.async_read_some( + boost::asio::buffer(buffer.data, buffer.len), + boost::asio::bind_executor(strand_, std::bind_front(&WebsocketSession::OnRead, shared_from_this()))); + } + + void OnRead(const boost::system::error_code &ec, [[maybe_unused]] const size_t bytes_transferred) { + // This indicates that the WebsocketSession was closed + if (ec == boost::beast::websocket::error::closed) { + return; + } + if (ec) { + OnError(ec, "read"); + } + input_buffer_.write_end()->Written(bytes_transferred); + + try { + session_.Execute(); + DoRead(); + } catch (const SessionClosedException &e) { + spdlog::info("{} client {}:{} closed the connection.", service_name_, remote_endpoint_.address(), + remote_endpoint_.port()); + DoClose(); + } catch (const std::exception &e) { + spdlog::error( + "Exception was thrown while processing event in {} session " + "associated with {}:{}", + service_name_, remote_endpoint_.address(), remote_endpoint_.port()); + spdlog::debug("Exception message: {}", e.what()); + DoClose(); + } + } + + void OnError(const boost::system::error_code &ec, const std::string_view action) { + spdlog::error("Websocket Bolt session error: {} on {}", ec.message(), action); + + DoClose(); + } + + void DoClose() { + ws_.async_close( + boost::beast::websocket::close_code::normal, + boost::asio::bind_executor( + strand_, [shared_this = shared_from_this()](boost::beast::error_code ec) { shared_this->OnClose(ec); })); + } + + void OnClose(const boost::system::error_code &ec) { + if (!IsConnected()) { + return; + } + if (ec) { + return OnError(ec, "close"); + } + } + + bool IsConnected() const { return ws_.is_open() && execution_active_; } + + WebSocket ws_; + boost::asio::strand strand_; + + communication::Buffer input_buffer_; + OutputStream output_stream_; + TSession session_; + tcp::endpoint endpoint_; + tcp::endpoint remote_endpoint_; + std::string_view service_name_; + bool execution_active_{false}; +}; + +/** + * This class is used internally in the communication stack to handle all user + * Sessions. It handles socket ownership, inactivity timeout and protocol + * wrapping. + */ +template +class Session final : public std::enable_shared_from_this> { + using TCPSocket = tcp::socket; + using SSLSocket = boost::asio::ssl::stream; + using std::enable_shared_from_this>::shared_from_this; + + public: + template + static std::shared_ptr Create(Args &&...args) { + return std::shared_ptr(new Session(std::forward(args)...)); + } + + Session(const Session &) = delete; + Session(Session &&) = delete; + Session &operator=(const Session &) = delete; + Session &operator=(Session &&) = delete; + ~Session() { + if (IsConnected()) { + spdlog::error("Session: Destructor called while execution is active"); + } + } + + bool Start() { + if (execution_active_) { + return false; + } + execution_active_ = true; + timeout_timer_.async_wait(boost::asio::bind_executor(strand_, std::bind(&Session::OnTimeout, shared_from_this()))); + + if (std::holds_alternative(socket_)) { + boost::asio::dispatch(strand_, [shared_this = shared_from_this()] { shared_this->DoHandshake(); }); + } else { + boost::asio::dispatch(strand_, [shared_this = shared_from_this()] { shared_this->DoRead(); }); + } + return true; + } + + bool Write(const uint8_t *data, size_t len, bool have_more = false) { + if (!IsConnected()) { + return false; + } + return std::visit( + utils::Overloaded{[shared_this = shared_from_this(), data, len, have_more](TCPSocket &socket) mutable { + boost::system::error_code ec; + while (len > 0) { + const auto sent = socket.send(boost::asio::buffer(data, len), + MSG_NOSIGNAL | (have_more ? MSG_MORE : 0), ec); + if (ec) { + shared_this->OnError(ec); + return false; + } + data += sent; + len -= sent; + } + return true; + }, + [shared_this = shared_from_this(), data, len](SSLSocket &socket) mutable { + boost::system::error_code ec; + while (len > 0) { + const auto sent = socket.write_some(boost::asio::buffer(data, len), ec); + if (ec) { + shared_this->OnError(ec); + return false; + } + data += sent; + len -= sent; + } + return true; + }}, + socket_); + } + + bool IsConnected() const { + return std::visit([this](const auto &socket) { return execution_active_ && socket.lowest_layer().is_open(); }, + socket_); + } + + private: + explicit Session(tcp::socket &&socket, TSessionData *data, ServerContext &server_context, tcp::endpoint endpoint, + const std::chrono::seconds inactivity_timeout_sec, std::string_view service_name) + : socket_(CreateSocket(std::move(socket), server_context)), + strand_{boost::asio::make_strand(GetExecutor())}, + output_stream_([this](const uint8_t *data, size_t len, bool have_more) { return Write(data, len, have_more); }), + session_(data, endpoint, input_buffer_.read_end(), &output_stream_), + data_{data}, + endpoint_{endpoint}, + remote_endpoint_{GetRemoteEndpoint()}, + service_name_{service_name}, + timeout_seconds_(inactivity_timeout_sec), + timeout_timer_(GetExecutor()) { + ExecuteForSocket([](auto &&socket) { + socket.lowest_layer().set_option(tcp::no_delay(true)); // enable PSH + socket.lowest_layer().set_option(boost::asio::socket_base::keep_alive(true)); // enable SO_KEEPALIVE + socket.lowest_layer().non_blocking(false); + }); + timeout_timer_.expires_at(boost::asio::steady_timer::time_point::max()); + spdlog::info("Accepted a connection from {}:", service_name_, remote_endpoint_.address(), remote_endpoint_.port()); + } + + void DoRead() { + if (!IsConnected()) { + return; + } + timeout_timer_.expires_after(timeout_seconds_); + ExecuteForSocket([this](auto &&socket) { + auto buffer = input_buffer_.write_end()->Allocate(); + socket.async_read_some( + boost::asio::buffer(buffer.data, buffer.len), + boost::asio::bind_executor(strand_, std::bind_front(&Session::OnRead, shared_from_this()))); + }); + } + + bool IsWebsocketUpgrade(boost::beast::http::request_parser &parser) { + boost::system::error_code error_code_parsing; + parser.put(boost::asio::buffer(input_buffer_.read_end()->data(), input_buffer_.read_end()->size()), + error_code_parsing); + if (error_code_parsing) { + return false; + } + + return boost::beast::websocket::is_upgrade(parser.get()); + } + + void OnRead(const boost::system::error_code &ec, const size_t bytes_transferred) { + if (ec) { + return OnError(ec); + } + input_buffer_.write_end()->Written(bytes_transferred); + + // Can be a websocket connection only on the first read, since it is not + // expected from clients to upgrade from tcp to websocket + if (!has_received_msg_) { + has_received_msg_ = true; + boost::beast::http::request_parser parser; + + if (IsWebsocketUpgrade(parser)) { + spdlog::info("Switching {} to websocket connection", remote_endpoint_); + if (std::holds_alternative(socket_)) { + auto sock = std::get(std::move(socket_)); + WebsocketSession::Create(std::move(sock), data_, endpoint_, service_name_) + ->DoAccept(parser.release()); + execution_active_ = false; + return; + } + spdlog::error("Error while upgrading connection to websocket"); + DoShutdown(); + } + } + + try { + session_.Execute(); + DoRead(); + } catch (const SessionClosedException &e) { + spdlog::info("{} client {}:{} closed the connection.", service_name_, remote_endpoint_.address(), + remote_endpoint_.port()); + DoShutdown(); + } catch (const std::exception &e) { + spdlog::error( + "Exception was thrown while processing event in {} session " + "associated with {}:{}", + service_name_, remote_endpoint_.address(), remote_endpoint_.port()); + spdlog::debug("Exception message: {}", e.what()); + DoShutdown(); + } + } + + void OnError(const boost::system::error_code &ec) { + if (ec == boost::asio::error::operation_aborted) { + return; + } + execution_active_ = false; + + if (ec == boost::asio::error::eof) { + spdlog::info("Session closed by peer"); + } else { + spdlog::error("Session error: {}", ec.message()); + } + + DoShutdown(); + } + + void DoShutdown() { + if (!IsConnected()) { + return; + } + execution_active_ = false; + timeout_timer_.cancel(); + ExecuteForSocket([](auto &socket) { + boost::system::error_code ec; + auto &lowest_layer = socket.lowest_layer(); + lowest_layer.shutdown(boost::asio::ip::tcp::socket::shutdown_both, ec); + if (ec) { + spdlog::error("Session shutdown failed: {}", ec.what()); + } + lowest_layer.close(); + }); + } + + void DoHandshake() { + if (!IsConnected()) { + return; + } + if (auto *socket = std::get_if(&socket_); socket) { + socket->async_handshake( + boost::asio::ssl::stream_base::server, + boost::asio::bind_executor(strand_, std::bind_front(&Session::OnHandshake, shared_from_this()))); + } + } + + void OnHandshake(const boost::system::error_code &ec) { + if (ec) { + return OnError(ec); + } + DoRead(); + } + + void OnClose(const boost::system::error_code &ec) { + if (ec) { + return OnError(ec); + } + } + + void OnTimeout() { + if (!IsConnected()) { + return; + } + // Check whether the deadline has passed. We compare the deadline against + // the current time since a new asynchronous operation may have moved the + // deadline before this actor had a chance to run. + if (timeout_timer_.expiry() <= boost::asio::steady_timer::clock_type::now()) { + // The deadline has passed. Stop the session. The other actors will + // terminate as soon as possible. + spdlog::info("Shutting down session after {} of inactivity", timeout_seconds_); + DoShutdown(); + } else { + // Put the actor back to sleep. + timeout_timer_.async_wait( + boost::asio::bind_executor(strand_, std::bind(&Session::OnTimeout, shared_from_this()))); + } + } + + std::variant CreateSocket(tcp::socket &&socket, ServerContext &context) { + if (context.use_ssl()) { + ssl_context_.emplace(context.context_clone()); + return SSLSocket{std::move(socket), *ssl_context_}; + } + + return TCPSocket{std::move(socket)}; + } + + auto GetExecutor() { + return std::visit(utils::Overloaded{[](auto &&socket) { return socket.get_executor(); }}, socket_); + } + + auto GetRemoteEndpoint() const { + return std::visit(utils::Overloaded{[](const auto &socket) { return socket.lowest_layer().remote_endpoint(); }}, + socket_); + } + + template + decltype(auto) ExecuteForSocket(F &&fun) { + return std::visit(utils::Overloaded{std::forward(fun)}, socket_); + } + + std::variant socket_; + std::optional> ssl_context_; + boost::asio::strand strand_; + + communication::Buffer input_buffer_; + OutputStream output_stream_; + TSession session_; + TSessionData *data_; + tcp::endpoint endpoint_; + tcp::endpoint remote_endpoint_; + std::string_view service_name_; + std::chrono::seconds timeout_seconds_; + boost::asio::steady_timer timeout_timer_; + bool execution_active_{false}; + bool has_received_msg_{false}; +}; +} // namespace memgraph::communication::v2 diff --git a/src/communication/websocket/listener.hpp b/src/communication/websocket/listener.hpp index 9a0ba0dc0..0ad19f960 100644 --- a/src/communication/websocket/listener.hpp +++ b/src/communication/websocket/listener.hpp @@ -11,8 +11,6 @@ #pragma once -#define BOOST_ASIO_USE_TS_EXECUTOR_AS_DEFAULT - #include #include diff --git a/src/communication/websocket/server.hpp b/src/communication/websocket/server.hpp index 18c377a7d..0853d3ebc 100644 --- a/src/communication/websocket/server.hpp +++ b/src/communication/websocket/server.hpp @@ -11,8 +11,6 @@ #pragma once -#define BOOST_ASIO_USE_TS_EXECUTOR_AS_DEFAULT - #include #include diff --git a/src/communication/websocket/session.hpp b/src/communication/websocket/session.hpp index 232ef5e15..0e5c92aa3 100644 --- a/src/communication/websocket/session.hpp +++ b/src/communication/websocket/session.hpp @@ -11,8 +11,6 @@ #pragma once -#define BOOST_ASIO_USE_TS_EXECUTOR_AS_DEFAULT - #include #include #include diff --git a/src/memgraph.cpp b/src/memgraph.cpp index bad6069ff..0554eca8b 100644 --- a/src/memgraph.cpp +++ b/src/memgraph.cpp @@ -81,8 +81,8 @@ #include "communication/bolt/v1/exceptions.hpp" #include "communication/bolt/v1/session.hpp" #include "communication/init.hpp" -#include "communication/server.hpp" -#include "communication/session.hpp" +#include "communication/v2/server.hpp" +#include "communication/v2/session.hpp" #include "glue/communication.hpp" #include "auth/auth.hpp" @@ -842,13 +842,14 @@ class AuthChecker final : public memgraph::query::AuthChecker { memgraph::utils::Synchronized *auth_; }; -class BoltSession final : public memgraph::communication::bolt::Session { +class BoltSession final : public memgraph::communication::bolt::Session { public: - BoltSession(SessionData *data, const memgraph::io::network::Endpoint &endpoint, - memgraph::communication::InputStream *input_stream, memgraph::communication::OutputStream *output_stream) - : memgraph::communication::bolt::Session(input_stream, output_stream), + BoltSession(SessionData *data, const memgraph::communication::v2::ServerEndpoint &endpoint, + memgraph::communication::v2::InputStream *input_stream, + memgraph::communication::v2::OutputStream *output_stream) + : memgraph::communication::bolt::Session(input_stream, output_stream), db_(data->db), interpreter_(data->interpreter_context), auth_(data->auth), @@ -858,8 +859,8 @@ class BoltSession final : public memgraph::communication::bolt::Session::TEncoder; + using memgraph::communication::bolt::Session::TEncoder; void BeginTransaction() override { interpreter_.BeginTransaction(); } @@ -877,7 +878,8 @@ class BoltSession final : public memgraph::communication::bolt::SessionRecord(endpoint_.address, user_ ? *username : "", query, memgraph::storage::PropertyValue(params_pv)); + audit_log_->Record(endpoint_.address().to_string(), user_ ? *username : "", query, + memgraph::storage::PropertyValue(params_pv)); } #endif try { @@ -996,10 +998,10 @@ class BoltSession final : public memgraph::communication::bolt::Session; +using ServerT = memgraph::communication::v2::Server; using memgraph::communication::ServerContext; // Needed to correctly handle memgraph destruction from a signal handler. @@ -1241,8 +1243,10 @@ int main(int argc, char **argv) { memgraph::utils::MessageWithLink("Using non-secure Bolt connection (without SSL).", "https://memgr.ph/ssl")); } - ServerT server({FLAGS_bolt_address, static_cast(FLAGS_bolt_port)}, &session_data, &context, - FLAGS_bolt_session_inactivity_timeout, service_name, FLAGS_bolt_num_workers); + auto server_endpoint = memgraph::communication::v2::ServerEndpoint{ + boost::asio::ip::address::from_string(FLAGS_bolt_address), static_cast(FLAGS_bolt_port)}; + ServerT server(server_endpoint, &session_data, &context, FLAGS_bolt_session_inactivity_timeout, service_name, + FLAGS_bolt_num_workers); // Setup telemetry std::optional telemetry; From a8ffcfa04655212ff6f8a6ade2bd1a9241379e90 Mon Sep 17 00:00:00 2001 From: Jure Bajic Date: Wed, 27 Apr 2022 13:31:37 +0200 Subject: [PATCH 2/4] Update license year MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Update license year Co-authored-by: János Benjamin Antal Co-authored-by: János Benjamin Antal --- licenses/BSL.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/licenses/BSL.txt b/licenses/BSL.txt index b06442cce..94ca81a32 100644 --- a/licenses/BSL.txt +++ b/licenses/BSL.txt @@ -36,7 +36,7 @@ ADDITIONAL USE GRANT: You may use the Licensed Work in accordance with the 3. using the Licensed Work to create a work or solution which competes (or might reasonably be expected to compete) with the Licensed Work. -CHANGE DATE: 2026-18-02 +CHANGE DATE: 2026-27-04 CHANGE LICENSE: Apache License, Version 2.0 For information about alternative licensing arrangements, please visit: https://memgraph.com/legal. From b782271be87a04173ebe7c724358ba107b15dd9a Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 28 Apr 2022 20:28:44 +0200 Subject: [PATCH 3/4] Fix shared module loading * Moving function add_query_module from CMakeLists from tests/e2e/magic_functions to tests/e2e * Adding failing test copying behavior when loading c module (.so) into memgraph. * Fixing issue where NO_ERROR status returned MgpTransAddFixedResult was converted to false * Removing unnecessary transformation * removing incorrect parameterization of test * re-adding parametrized transformation --- src/query/procedure/module.cpp | 4 +-- tests/e2e/CMakeLists.txt | 8 ++++++ tests/e2e/magic_functions/CMakeLists.txt | 7 ----- tests/e2e/streams/kafka_streams_tests.py | 28 +++++++++++++------ .../streams/transformations/CMakeLists.txt | 1 + .../transformations/c_transformations.cpp | 22 +++++++++++++++ 6 files changed, 52 insertions(+), 18 deletions(-) create mode 100644 tests/e2e/streams/transformations/c_transformations.cpp diff --git a/src/query/procedure/module.cpp b/src/query/procedure/module.cpp index 357dfba31..427010803 100644 --- a/src/query/procedure/module.cpp +++ b/src/query/procedure/module.cpp @@ -832,8 +832,8 @@ bool SharedLibraryModule::Load(const std::filesystem::path &file_path) { return with_error(error); } for (auto &trans : module_def->transformations) { - const bool was_result_added = MgpTransAddFixedResult(&trans.second); - if (!was_result_added) { + const bool success = MGP_ERROR_NO_ERROR == MgpTransAddFixedResult(&trans.second); + if (!success) { const auto error = fmt::format("Unable to add result to transformation in module {}; add result failed", file_path); return with_error(error); diff --git a/tests/e2e/CMakeLists.txt b/tests/e2e/CMakeLists.txt index 4a53a0836..c8c0913f5 100644 --- a/tests/e2e/CMakeLists.txt +++ b/tests/e2e/CMakeLists.txt @@ -1,3 +1,11 @@ +# Set up C++ functions for e2e tests +function(add_query_module target_name src) + add_library(${target_name} SHARED ${src}) + SET_TARGET_PROPERTIES(${target_name} PROPERTIES PREFIX "") + target_include_directories(${target_name} PRIVATE ${CMAKE_SOURCE_DIR}/include) +endfunction() + + function(copy_e2e_python_files TARGET_PREFIX FILE_NAME) add_custom_target(memgraph__e2e__${TARGET_PREFIX}__${FILE_NAME} ALL COMMAND ${CMAKE_COMMAND} -E copy diff --git a/tests/e2e/magic_functions/CMakeLists.txt b/tests/e2e/magic_functions/CMakeLists.txt index fb4986724..3ab627e22 100644 --- a/tests/e2e/magic_functions/CMakeLists.txt +++ b/tests/e2e/magic_functions/CMakeLists.txt @@ -1,10 +1,3 @@ -# Set up C++ functions for e2e tests -function(add_query_module target_name src) - add_library(${target_name} SHARED ${src}) - SET_TARGET_PROPERTIES(${target_name} PROPERTIES PREFIX "") - target_include_directories(${target_name} PRIVATE ${CMAKE_SOURCE_DIR}/include) -endfunction() - # Set up Python functions for e2e tests function(copy_magic_functions_e2e_python_files FILE_NAME) copy_e2e_python_files(functions ${FILE_NAME}) diff --git a/tests/e2e/streams/kafka_streams_tests.py b/tests/e2e/streams/kafka_streams_tests.py index 1455fe95d..0d2e14c78 100755 --- a/tests/e2e/streams/kafka_streams_tests.py +++ b/tests/e2e/streams/kafka_streams_tests.py @@ -18,12 +18,14 @@ import time from multiprocessing import Process, Value import common -TRANSFORMATIONS_TO_CHECK = [ +TRANSFORMATIONS_TO_CHECK_C = [ + "empty_transformation"] + +TRANSFORMATIONS_TO_CHECK_PY = [ "kafka_transform.simple", "kafka_transform.with_parameters"] - -@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK) +@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK_PY) def test_simple(kafka_producer, kafka_topics, connection, transformation): assert len(kafka_topics) > 0 cursor = connection.cursor() @@ -44,7 +46,7 @@ def test_simple(kafka_producer, kafka_topics, connection, transformation): cursor, topic, common.SIMPLE_MSG) -@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK) +@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK_PY) def test_separate_consumers( kafka_producer, kafka_topics, @@ -125,7 +127,7 @@ def test_start_from_last_committed_offset( cursor, kafka_topics[0], message) -@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK) +@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK_PY) def test_check_stream( kafka_producer, kafka_topics, @@ -307,7 +309,7 @@ def test_restart_after_error(kafka_producer, kafka_topics, connection): cursor, "MATCH (n:VERTEX { id : 42 }) RETURN n") -@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK) +@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK_PY) def test_bootstrap_server( kafka_producer, kafka_topics, @@ -334,7 +336,7 @@ def test_bootstrap_server( cursor, topic, common.SIMPLE_MSG) -@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK) +@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK_PY) def test_bootstrap_server_empty( kafka_producer, kafka_topics, @@ -352,7 +354,7 @@ def test_bootstrap_server_empty( ) -@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK) +@pytest.mark.parametrize("transformation", TRANSFORMATIONS_TO_CHECK_PY) def test_set_offset(kafka_producer, kafka_topics, connection, transformation): assert len(kafka_topics) > 0 cursor = connection.cursor() @@ -450,6 +452,14 @@ def test_info_procedure(kafka_topics, connection): (local, configs, consumer_group, reducted_credentials, kafka_topics)] common.validate_info(stream_info, expected_stream_info) - +@pytest.mark.parametrize("transformation",TRANSFORMATIONS_TO_CHECK_C) +def test_load_c_transformations(connection, transformation): + cursor = connection.cursor() + query = "CALL mg.transformations() YIELD * WITH name WHERE name STARTS WITH 'c_transformations." + transformation + "' RETURN name" + result = common.execute_and_fetch_all( + cursor, query) + assert len(result) == 1 + assert result[0][0] == "c_transformations." + transformation + if __name__ == "__main__": sys.exit(pytest.main([__file__, "-rA"])) diff --git a/tests/e2e/streams/transformations/CMakeLists.txt b/tests/e2e/streams/transformations/CMakeLists.txt index 8934a3ce7..5ecafd43a 100644 --- a/tests/e2e/streams/transformations/CMakeLists.txt +++ b/tests/e2e/streams/transformations/CMakeLists.txt @@ -1,2 +1,3 @@ copy_streams_e2e_python_files(kafka_transform.py) copy_streams_e2e_python_files(pulsar_transform.py) +add_query_module(c_transformations c_transformations.cpp) diff --git a/tests/e2e/streams/transformations/c_transformations.cpp b/tests/e2e/streams/transformations/c_transformations.cpp new file mode 100644 index 000000000..984a04c61 --- /dev/null +++ b/tests/e2e/streams/transformations/c_transformations.cpp @@ -0,0 +1,22 @@ +// Copyright 2022 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 "mg_procedure.h" + +extern "C" int mgp_init_module(mgp_module *module, mgp_memory *memory) { + static const auto no_op_cb = [](mgp_messages *msg, mgp_graph *graph, mgp_result *result, mgp_memory *memory) {}; + + if (MGP_ERROR_NO_ERROR != mgp_module_add_transformation(module, "empty_transformation", no_op_cb)) { + return 1; + } + + return 0; +} \ No newline at end of file From 10ea9c773ed76a4d6d3220be2537c7eba8bae263 Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 5 May 2022 17:48:17 +0200 Subject: [PATCH 4/4] =?UTF-8?q?Making=20mgp=5Ferror=20enum=20class=20when?= =?UTF-8?q?=20compiling=20c++.=20Remains=20enum=20when=20com=E2=80=A6=20(#?= =?UTF-8?q?389)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Making mgp_error enum class when compiling c++. Remains enum when compiling c. --- include/mg_procedure.h | 414 +++++++++--------- src/query/procedure/mg_procedure_helpers.cpp | 2 +- src/query/procedure/mg_procedure_helpers.hpp | 6 +- src/query/procedure/mg_procedure_impl.cpp | 66 +-- src/query/procedure/module.cpp | 75 ++-- src/query/procedure/py_module.cpp | 115 ++--- src/query/stream/streams.cpp | 48 +- .../e2e/magic_functions/functions/c_read.cpp | 30 +- .../e2e/magic_functions/functions/c_write.cpp | 12 +- .../transformations/c_transformations.cpp | 2 +- tests/unit/mgp_trans_c_api.cpp | 6 +- tests/unit/query_function_mgp_module.cpp | 15 +- tests/unit/query_procedure_mgp_module.cpp | 54 ++- tests/unit/query_procedure_mgp_type.cpp | 12 +- tests/unit/query_procedure_py_module.cpp | 6 +- tests/unit/query_procedures_mgp_graph.cpp | 23 +- tests/unit/test_utils.hpp | 2 +- 17 files changed, 461 insertions(+), 427 deletions(-) diff --git a/include/mg_procedure.h b/include/mg_procedure.h index 8bf29afeb..5d2527e81 100644 --- a/include/mg_procedure.h +++ b/include/mg_procedure.h @@ -16,6 +16,9 @@ #ifdef __cplusplus extern "C" { +#define MGP_ENUM_CLASS enum class +#else +#define MGP_ENUM_CLASS enum #endif #if __cplusplus >= 201703L @@ -33,19 +36,13 @@ extern "C" { /// All functions return an error code that can be used to figure out whether the API call was successful or not. In /// case of failure, the specific error code can be used to identify the reason of the failure. -enum MGP_NODISCARD mgp_error { - MGP_ERROR_NO_ERROR = 0, - MGP_ERROR_UNKNOWN_ERROR, - MGP_ERROR_UNABLE_TO_ALLOCATE, - MGP_ERROR_INSUFFICIENT_BUFFER, - MGP_ERROR_OUT_OF_RANGE, - MGP_ERROR_LOGIC_ERROR, - MGP_ERROR_DELETED_OBJECT, - MGP_ERROR_INVALID_ARGUMENT, - MGP_ERROR_KEY_ALREADY_EXISTS, - MGP_ERROR_IMMUTABLE_OBJECT, - MGP_ERROR_VALUE_CONVERSION, - MGP_ERROR_SERIALIZATION_ERROR, +MGP_ENUM_CLASS MGP_NODISCARD mgp_error{ + MGP_ERROR_NO_ERROR, MGP_ERROR_UNKNOWN_ERROR, + MGP_ERROR_UNABLE_TO_ALLOCATE, MGP_ERROR_INSUFFICIENT_BUFFER, + MGP_ERROR_OUT_OF_RANGE, MGP_ERROR_LOGIC_ERROR, + MGP_ERROR_DELETED_OBJECT, MGP_ERROR_INVALID_ARGUMENT, + MGP_ERROR_KEY_ALREADY_EXISTS, MGP_ERROR_IMMUTABLE_OBJECT, + MGP_ERROR_VALUE_CONVERSION, MGP_ERROR_SERIALIZATION_ERROR, }; ///@} @@ -70,7 +67,7 @@ struct mgp_memory; /// Unlike malloc, this function is not thread-safe. /// `size_in_bytes` must be greater than 0. /// The resulting pointer must be freed with mgp_free. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. enum mgp_error mgp_alloc(struct mgp_memory *memory, size_t size_in_bytes, void **result); /// Allocate an aligned block of memory with given size in bytes. @@ -78,7 +75,7 @@ enum mgp_error mgp_alloc(struct mgp_memory *memory, size_t size_in_bytes, void * /// `size_in_bytes` must be greater than 0. /// `alignment` must be a power of 2 value. /// The resulting pointer must be freed with mgp_free. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. enum mgp_error mgp_aligned_alloc(struct mgp_memory *memory, size_t size_in_bytes, size_t alignment, void **result); /// Deallocate an allocation from mgp_alloc or mgp_aligned_alloc. @@ -92,14 +89,14 @@ void mgp_free(struct mgp_memory *memory, void *ptr); /// This function can be used to allocate global memory that persists /// beyond a single invocation of mgp_main. /// The resulting pointer must be freed with mgp_global_free. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. enum mgp_error mgp_global_alloc(size_t size_in_bytes, void **result); /// Allocate an aligned global block of memory with given size in bytes. /// This function can be used to allocate global memory that persists /// beyond a single invocation of mgp_main. /// The resulting pointer must be freed with mgp_global_free. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to serve the requested allocation. enum mgp_error mgp_global_aligned_alloc(size_t size_in_bytes, size_t alignment, void **result); /// Deallocate an allocation from mgp_global_alloc or mgp_global_aligned_alloc. @@ -172,28 +169,28 @@ void mgp_value_destroy(struct mgp_value *val); /// Construct a value representing `null` in openCypher. /// You need to free the instance through mgp_value_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_null(struct mgp_memory *memory, struct mgp_value **result); /// Construct a boolean value. /// Non-zero values represent `true`, while zero represents `false`. /// You need to free the instance through mgp_value_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_bool(int val, struct mgp_memory *memory, struct mgp_value **result); /// Construct an integer value. /// You need to free the instance through mgp_value_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_int(int64_t val, struct mgp_memory *memory, struct mgp_value **result); /// Construct a double floating point value. /// You need to free the instance through mgp_value_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_double(double val, struct mgp_memory *memory, struct mgp_value **result); /// Construct a character string value from a NULL terminated string. /// You need to free the instance through mgp_value_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_string(const char *val, struct mgp_memory *memory, struct mgp_value **result); /// Create a mgp_value storing a mgp_list. @@ -201,7 +198,7 @@ enum mgp_error mgp_value_make_string(const char *val, struct mgp_memory *memory, /// the list is given to the created mgp_value and destroying the mgp_value will /// destroy the mgp_list. Therefore, if a mgp_value is successfully created /// you must not call mgp_list_destroy on the given list. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_list(struct mgp_list *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_map. @@ -209,7 +206,7 @@ enum mgp_error mgp_value_make_list(struct mgp_list *val, struct mgp_value **resu /// the map is given to the created mgp_value and destroying the mgp_value will /// destroy the mgp_map. Therefore, if a mgp_value is successfully created /// you must not call mgp_map_destroy on the given map. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_map(struct mgp_map *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_vertex. @@ -217,7 +214,7 @@ enum mgp_error mgp_value_make_map(struct mgp_map *val, struct mgp_value **result /// the vertex is given to the created mgp_value and destroying the mgp_value /// will destroy the mgp_vertex. Therefore, if a mgp_value is successfully /// created you must not call mgp_vertex_destroy on the given vertex. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_vertex(struct mgp_vertex *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_edge. @@ -225,7 +222,7 @@ enum mgp_error mgp_value_make_vertex(struct mgp_vertex *val, struct mgp_value ** /// the edge is given to the created mgp_value and destroying the mgp_value will /// destroy the mgp_edge. Therefore, if a mgp_value is successfully created you /// must not call mgp_edge_destroy on the given edge. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_edge(struct mgp_edge *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_path. @@ -233,7 +230,7 @@ enum mgp_error mgp_value_make_edge(struct mgp_edge *val, struct mgp_value **resu /// the path is given to the created mgp_value and destroying the mgp_value will /// destroy the mgp_path. Therefore, if a mgp_value is successfully created you /// must not call mgp_path_destroy on the given path. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_path(struct mgp_path *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_date. @@ -241,7 +238,7 @@ enum mgp_error mgp_value_make_path(struct mgp_path *val, struct mgp_value **resu /// the date is transferred to the created mgp_value and destroying the mgp_value will /// destroy the mgp_date. Therefore, if a mgp_value is successfully created you /// must not call mgp_date_destroy on the given date. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_date(struct mgp_date *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_local_time. @@ -249,7 +246,7 @@ enum mgp_error mgp_value_make_date(struct mgp_date *val, struct mgp_value **resu /// the local time is transferred to the created mgp_value and destroying the mgp_value will /// destroy the mgp_local_time. Therefore, if a mgp_value is successfully created you /// must not call mgp_local_time_destroy on the given local time. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_local_time(struct mgp_local_time *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_local_date_time. @@ -257,7 +254,7 @@ enum mgp_error mgp_value_make_local_time(struct mgp_local_time *val, struct mgp_ /// the local date-time is transferred to the created mgp_value and destroying the mgp_value will /// destroy the mgp_local_date_time. Therefore, if a mgp_value is successfully created you /// must not call mgp_local_date_time_destroy on the given local date-time. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_local_date_time(struct mgp_local_date_time *val, struct mgp_value **result); /// Create a mgp_value storing a mgp_duration. @@ -265,7 +262,7 @@ enum mgp_error mgp_value_make_local_date_time(struct mgp_local_date_time *val, s /// the duration is transferred to the created mgp_value and destroying the mgp_value will /// destroy the mgp_duration. Therefore, if a mgp_value is successfully created you /// must not call mgp_duration_destroy on the given duration. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_value. enum mgp_error mgp_value_make_duration(struct mgp_duration *val, struct mgp_value **result); /// Get the type of the value contained in mgp_value. @@ -399,7 +396,7 @@ enum mgp_error mgp_value_get_duration(struct mgp_value *val, struct mgp_duration /// The created list will have allocated enough memory for `capacity` elements /// of mgp_value, but it will not contain any elements. Therefore, /// mgp_list_size will return 0. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_list. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_list. enum mgp_error mgp_list_make_empty(size_t capacity, struct mgp_memory *memory, struct mgp_list **result); /// Free the memory used by the given mgp_list and contained elements. @@ -409,8 +406,8 @@ void mgp_list_destroy(struct mgp_list *list); /// The list copies the given value and therefore does not take ownership of the /// original value. You still need to call mgp_value_destroy to free the /// original value. -/// Return MGP_ERROR_INSUFFICIENT_BUFFER if there's no capacity. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. +/// Return mgp_error::MGP_ERROR_INSUFFICIENT_BUFFER if there's no capacity. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. enum mgp_error mgp_list_append(struct mgp_list *list, struct mgp_value *val); /// Append a copy of mgp_value to mgp_list increasing capacity if needed. @@ -419,7 +416,7 @@ enum mgp_error mgp_list_append(struct mgp_list *list, struct mgp_value *val); /// original value. /// In case of a capacity change, the previously contained elements will move in /// memory and any references to them will be invalid. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. enum mgp_error mgp_list_append_extend(struct mgp_list *list, struct mgp_value *val); /// Get the number of elements stored in mgp_list. @@ -432,12 +429,12 @@ enum mgp_error mgp_list_size(struct mgp_list *list, size_t *result); enum mgp_error mgp_list_capacity(struct mgp_list *list, size_t *result); /// Get the element in mgp_list at given position. -/// MGP_ERROR_OUT_OF_RANGE is returned if the index is not within mgp_list_size. +/// mgp_error::MGP_ERROR_OUT_OF_RANGE is returned if the index is not within mgp_list_size. enum mgp_error mgp_list_at(struct mgp_list *list, size_t index, struct mgp_value **result); /// Create an empty map of character strings to mgp_value instances. /// You need to free the created instance with mgp_map_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_map. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_map. enum mgp_error mgp_map_make_empty(struct mgp_memory *memory, struct mgp_map **result); /// Free the memory used by the given mgp_map and contained items. @@ -448,8 +445,8 @@ void mgp_map_destroy(struct mgp_map *map); /// In case of insertion, both the string and the value are copied into the map. /// Therefore, the map does not take ownership of the original key nor value, so /// you still need to free their memory explicitly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate for insertion. -/// Return MGP_ERROR_KEY_ALREADY_EXISTS if a previous mapping already exists. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate for insertion. +/// Return mgp_error::MGP_ERROR_KEY_ALREADY_EXISTS if a previous mapping already exists. enum mgp_error mgp_map_insert(struct mgp_map *map, const char *key, struct mgp_value *value); /// Get the number of items stored in mgp_map. @@ -475,7 +472,7 @@ struct mgp_map_items_iterator; /// Start iterating over items contained in the given map. /// The resulting mgp_map_items_iterator needs to be deallocated with /// mgp_map_items_iterator_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_map_items_iterator. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_map_items_iterator. enum mgp_error mgp_map_iter_items(struct mgp_map *map, struct mgp_memory *memory, struct mgp_map_items_iterator **result); @@ -500,12 +497,12 @@ enum mgp_error mgp_map_items_iterator_next(struct mgp_map_items_iterator *it, st /// Create a path with the copy of the given starting vertex. /// You need to free the created instance with mgp_path_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_path. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_path. enum mgp_error mgp_path_make_with_start(struct mgp_vertex *vertex, struct mgp_memory *memory, struct mgp_path **result); /// Copy a mgp_path. /// Returned pointer must be freed with mgp_path_destroy. -/// MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_path. +/// mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE is returned if unable to allocate a mgp_path. enum mgp_error mgp_path_copy(struct mgp_path *path, struct mgp_memory *memory, struct mgp_path **result); /// Free the memory used by the given mgp_path and contained vertices and edges. @@ -517,8 +514,8 @@ void mgp_path_destroy(struct mgp_path *path); /// explicitly. /// The last vertex on the path will become the other endpoint of the given /// edge, as continued from the current last vertex. -/// Return MGP_ERROR_LOGIC_ERROR if the current last vertex in the path is not part of the given edge. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for path extension. +/// Return mgp_error::MGP_ERROR_LOGIC_ERROR if the current last vertex in the path is not part of the given edge. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for path extension. enum mgp_error mgp_path_expand(struct mgp_path *path, struct mgp_edge *edge); /// Get the number of edges in a mgp_path. @@ -527,12 +524,12 @@ enum mgp_error mgp_path_size(struct mgp_path *path, size_t *result); /// Get the vertex from a path at given index. /// The valid index range is [0, mgp_path_size]. -/// MGP_ERROR_OUT_OF_RANGE is returned if index is out of range. +/// mgp_error::MGP_ERROR_OUT_OF_RANGE is returned if index is out of range. enum mgp_error mgp_path_vertex_at(struct mgp_path *path, size_t index, struct mgp_vertex **result); /// Get the edge from a path at given index. /// The valid index range is [0, mgp_path_size - 1]. -/// MGP_ERROR_OUT_OF_RANGE is returned if index is out of range. +/// mgp_error::MGP_ERROR_OUT_OF_RANGE is returned if index is out of range. enum mgp_error mgp_path_edge_at(struct mgp_path *path, size_t index, struct mgp_edge **result); /// Result is non-zero if given paths are equal, otherwise 0. @@ -553,18 +550,18 @@ struct mgp_result_record; struct mgp_func_result; /// Set the error as the result of the procedure. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE ff there's no memory for copying the error message. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE ff there's no memory for copying the error message. enum mgp_error mgp_result_set_error_msg(struct mgp_result *res, const char *error_msg); /// Create a new record for results. /// The previously obtained mgp_result_record pointer is no longer valid, and you must not use it. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_result_record. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_result_record. enum mgp_error mgp_result_new_record(struct mgp_result *res, struct mgp_result_record **result); /// Assign a value to a field in the given record. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory to copy the mgp_value to mgp_result_record. -/// Return MGP_ERROR_OUT_OF_RANGE if there is no field named `field_name`. -/// Return MGP_ERROR_LOGIC_ERROR `val` does not satisfy the type of the field name `field_name`. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory to copy the mgp_value to +/// mgp_result_record. Return mgp_error::MGP_ERROR_OUT_OF_RANGE if there is no field named `field_name`. Return +/// mgp_error::MGP_ERROR_LOGIC_ERROR `val` does not satisfy the type of the field name `field_name`. enum mgp_error mgp_result_record_insert(struct mgp_result_record *record, const char *field_name, struct mgp_value *val); ///@} @@ -608,7 +605,7 @@ enum mgp_error mgp_properties_iterator_get(struct mgp_properties_iterator *it, s /// The previous mgp_property obtained through mgp_properties_iterator_get /// will be invalidated, and you must not use its value. /// Result is NULL if the end of the iteration has been reached. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_property. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_property. enum mgp_error mgp_properties_iterator_next(struct mgp_properties_iterator *it, struct mgp_property **result); /// Iterator over edges of a vertex. @@ -634,32 +631,32 @@ enum mgp_error mgp_vertex_underlying_graph_is_mutable(struct mgp_vertex *v, int /// Set the value of a property on a vertex. /// When the value is `null`, then the property is removed from the vertex. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for storing the property. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `v` is immutable. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `v` has been modified by another transaction. -/// Return MGP_ERROR_VALUE_CONVERSION if `property_value` is vertex, edge or path. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for storing the property. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `v` is immutable. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `v` has been modified by another transaction. +/// Return mgp_error::MGP_ERROR_VALUE_CONVERSION if `property_value` is vertex, edge or path. enum mgp_error mgp_vertex_set_property(struct mgp_vertex *v, const char *property_name, struct mgp_value *property_value); /// Add the label to the vertex. /// If the vertex already has the label, this function does nothing. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for storing the label. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `v` is immutable. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `v` has been modified by another transaction. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for storing the label. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `v` is immutable. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `v` has been modified by another transaction. enum mgp_error mgp_vertex_add_label(struct mgp_vertex *v, struct mgp_label label); /// Remove the label from the vertex. /// If the vertex doesn't have the label, this function does nothing. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `v` is immutable. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `v` has been modified by another transaction. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `v` is immutable. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `v` has been modified by another transaction. enum mgp_error mgp_vertex_remove_label(struct mgp_vertex *v, struct mgp_label label); /// Copy a mgp_vertex. /// Resulting pointer must be freed with mgp_vertex_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertex. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertex. enum mgp_error mgp_vertex_copy(struct mgp_vertex *v, struct mgp_memory *memory, struct mgp_vertex **result); /// Free the memory used by a mgp_vertex. @@ -669,26 +666,26 @@ void mgp_vertex_destroy(struct mgp_vertex *v); enum mgp_error mgp_vertex_equal(struct mgp_vertex *v1, struct mgp_vertex *v2, int *result); /// Get the number of labels a given vertex has. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_labels_count(struct mgp_vertex *v, size_t *result); /// Get mgp_label in mgp_vertex at given index. -/// Return MGP_ERROR_OUT_OF_RANGE if the index is out of range. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_OUT_OF_RANGE if the index is out of range. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_label_at(struct mgp_vertex *v, size_t index, struct mgp_label *result); /// Result is non-zero if the given vertex has the given label. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_has_label(struct mgp_vertex *v, struct mgp_label label, int *result); /// Result is non-zero if the given vertex has a label with given name. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_has_label_named(struct mgp_vertex *v, const char *label_name, int *result); /// Get a copy of a vertex property mapped to a given name. /// Resulting value must be freed with mgp_value_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_get_property(struct mgp_vertex *v, const char *property_name, struct mgp_memory *memory, struct mgp_value **result); @@ -696,8 +693,8 @@ enum mgp_error mgp_vertex_get_property(struct mgp_vertex *v, const char *propert /// The properties of the vertex are copied when the iterator is created, therefore later changes won't affect them. /// The resulting mgp_properties_iterator needs to be deallocated with /// mgp_properties_iterator_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_properties_iterator. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_properties_iterator. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_iter_properties(struct mgp_vertex *v, struct mgp_memory *memory, struct mgp_properties_iterator **result); @@ -705,8 +702,8 @@ enum mgp_error mgp_vertex_iter_properties(struct mgp_vertex *v, struct mgp_memor /// The connection information of the vertex is copied when the iterator is created, therefore later creation or /// deletion of edges won't affect the iterated edges, however the property changes on the edges will be visible. /// The resulting mgp_edges_iterator needs to be deallocated with mgp_edges_iterator_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edges_iterator. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edges_iterator. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_iter_in_edges(struct mgp_vertex *v, struct mgp_memory *memory, struct mgp_edges_iterator **result); @@ -714,8 +711,8 @@ enum mgp_error mgp_vertex_iter_in_edges(struct mgp_vertex *v, struct mgp_memory /// The connection information of the vertex is copied when the iterator is created, therefore later creation or /// deletion of edges won't affect the iterated edges, however the property changes on the edges will be visible. /// The resulting mgp_edges_iterator needs to be deallocated with mgp_edges_iterator_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edges_iterator. -/// Return MGP_ERROR_DELETED_OBJECT if `v` has been deleted. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edges_iterator. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `v` has been deleted. enum mgp_error mgp_vertex_iter_out_edges(struct mgp_vertex *v, struct mgp_memory *memory, struct mgp_edges_iterator **result); @@ -734,7 +731,7 @@ enum mgp_error mgp_edges_iterator_get(struct mgp_edges_iterator *it, struct mgp_ /// The previous mgp_edge obtained through mgp_edges_iterator_get /// will be invalidated, and you must not use its value. /// Result is NULL if the end of the iteration has been reached. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edge. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edge. enum mgp_error mgp_edges_iterator_next(struct mgp_edges_iterator *it, struct mgp_edge **result); /// ID of an edge; valid during a single query execution. @@ -753,7 +750,7 @@ enum mgp_error mgp_edge_underlying_graph_is_mutable(struct mgp_edge *e, int *res /// Copy a mgp_edge. /// Resulting pointer must be freed with mgp_edge_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edge. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edge. enum mgp_error mgp_edge_copy(struct mgp_edge *e, struct mgp_memory *memory, struct mgp_edge **result); /// Free the memory used by a mgp_edge. @@ -777,27 +774,27 @@ enum mgp_error mgp_edge_get_to(struct mgp_edge *e, struct mgp_vertex **result); /// Get a copy of a edge property mapped to a given name. /// Resulting value must be freed with mgp_value_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. -/// Return MGP_ERROR_DELETED_OBJECT if `e` has been deleted. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_value. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `e` has been deleted. enum mgp_error mgp_edge_get_property(struct mgp_edge *e, const char *property_name, struct mgp_memory *memory, struct mgp_value **result); /// Set the value of a property on an edge. /// When the value is `null`, then the property is removed from the edge. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for storing the property. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `e` is immutable. -/// Return MGP_ERROR_DELETED_OBJECT if `e` has been deleted. -/// Return MGP_ERROR_LOGIC_ERROR if properties on edges are disabled. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `e` has been modified by another transaction. -/// Return MGP_ERROR_VALUE_CONVERSION if `property_value` is vertex, edge or path. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for storing the property. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `e` is immutable. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `e` has been deleted. +/// Return mgp_error::MGP_ERROR_LOGIC_ERROR if properties on edges are disabled. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `e` has been modified by another transaction. +/// Return mgp_error::MGP_ERROR_VALUE_CONVERSION if `property_value` is vertex, edge or path. enum mgp_error mgp_edge_set_property(struct mgp_edge *e, const char *property_name, struct mgp_value *property_value); /// Start iterating over properties stored in the given edge. /// The properties of the edge are copied when the iterator is created, therefore later changes won't affect them. /// Resulting mgp_properties_iterator needs to be deallocated with /// mgp_properties_iterator_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_properties_iterator. -/// Return MGP_ERROR_DELETED_OBJECT if `e` has been deleted. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_properties_iterator. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `e` has been deleted. enum mgp_error mgp_edge_iter_properties(struct mgp_edge *e, struct mgp_memory *memory, struct mgp_properties_iterator **result); @@ -806,7 +803,7 @@ struct mgp_graph; /// Get the vertex corresponding to given ID, or NULL if no such vertex exists. /// Resulting vertex must be freed using mgp_vertex_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the vertex. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the vertex. enum mgp_error mgp_graph_get_vertex_by_id(struct mgp_graph *g, struct mgp_vertex_id id, struct mgp_memory *memory, struct mgp_vertex **result); @@ -818,34 +815,34 @@ enum mgp_error mgp_graph_is_mutable(struct mgp_graph *graph, int *result); /// Add a new vertex to the graph. /// Resulting vertex must be freed using mgp_vertex_destroy. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertex. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertex. enum mgp_error mgp_graph_create_vertex(struct mgp_graph *graph, struct mgp_memory *memory, struct mgp_vertex **result); /// Delete a vertex from the graph. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. -/// Return MGP_ERROR_LOGIC_ERROR if `vertex` has edges. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `vertex` has been modified by another transaction. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. +/// Return mgp_error::MGP_ERROR_LOGIC_ERROR if `vertex` has edges. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `vertex` has been modified by another transaction. enum mgp_error mgp_graph_delete_vertex(struct mgp_graph *graph, struct mgp_vertex *vertex); /// Delete a vertex and all of its edges from the graph. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `vertex` has been modified by another transaction. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `vertex` has been modified by another transaction. enum mgp_error mgp_graph_detach_delete_vertex(struct mgp_graph *graph, struct mgp_vertex *vertex); /// Add a new directed edge between the two vertices with the specified label. /// Resulting edge must be freed using mgp_edge_destroy. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edge. -/// Return MGP_ERROR_DELETED_OBJECT if `from` or `to` has been deleted. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `from` or `to` has been modified by another transaction. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_edge. +/// Return mgp_error::MGP_ERROR_DELETED_OBJECT if `from` or `to` has been deleted. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `from` or `to` has been modified by another transaction. enum mgp_error mgp_graph_create_edge(struct mgp_graph *graph, struct mgp_vertex *from, struct mgp_vertex *to, struct mgp_edge_type type, struct mgp_memory *memory, struct mgp_edge **result); /// Delete an edge from the graph. -/// Return MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. -/// Return MGP_ERROR_SERIALIZATION_ERROR if `edge`, its source or destination vertex has been modified by another -/// transaction. +/// Return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT if `graph` is immutable. +/// Return mgp_error::MGP_ERROR_SERIALIZATION_ERROR if `edge`, its source or destination vertex has been modified by +/// another transaction. enum mgp_error mgp_graph_delete_edge(struct mgp_graph *graph, struct mgp_edge *edge); /// Iterator over vertices. @@ -856,7 +853,7 @@ void mgp_vertices_iterator_destroy(struct mgp_vertices_iterator *it); /// Start iterating over vertices of the given graph. /// Resulting mgp_vertices_iterator needs to be deallocated with mgp_vertices_iterator_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertices_iterator. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertices_iterator. enum mgp_error mgp_graph_iter_vertices(struct mgp_graph *g, struct mgp_memory *memory, struct mgp_vertices_iterator **result); @@ -883,20 +880,20 @@ struct mgp_date_parameters { /// Create a date from a string following the ISO 8601 format. /// Resulting date must be freed with mgp_date_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_date_from_string(const char *string, struct mgp_memory *memory, struct mgp_date **date); /// Create a date from mgp_date_parameter. /// Resulting date must be freed with mgp_date_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_date_from_parameters(struct mgp_date_parameters *parameters, struct mgp_memory *memory, struct mgp_date **date); /// Copy a mgp_date. /// Resulting pointer must be freed with mgp_date_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_date_copy(struct mgp_date *date, struct mgp_memory *memory, struct mgp_date **result); /// Free the memory used by a mgp_date. @@ -919,26 +916,26 @@ enum mgp_error mgp_date_timestamp(struct mgp_date *date, int64_t *timestamp); /// Get the date representing current date. /// Resulting date must be freed with mgp_date_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_date_now(struct mgp_memory *memory, struct mgp_date **date); /// Add a duration to the date. /// Resulting date must be freed with mgp_date_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid date. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_date_add_duration(struct mgp_date *date, struct mgp_duration *dur, struct mgp_memory *memory, struct mgp_date **result); /// Subtract a duration from the date. /// Resulting date must be freed with mgp_date_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid date. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_date_sub_duration(struct mgp_date *date, struct mgp_duration *dur, struct mgp_memory *memory, struct mgp_date **result); /// Get a duration between two dates. /// Resulting duration must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_date_diff(struct mgp_date *first, struct mgp_date *second, struct mgp_memory *memory, struct mgp_duration **result); @@ -952,21 +949,21 @@ struct mgp_local_time_parameters { /// Create a local time from a string following the ISO 8601 format. /// Resulting local time must be freed with mgp_local_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_local_time_from_string(const char *string, struct mgp_memory *memory, struct mgp_local_time **local_time); /// Create a local time from mgp_local_time_parameters. /// Resulting local time must be freed with mgp_local_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_local_time_from_parameters(struct mgp_local_time_parameters *parameters, struct mgp_memory *memory, struct mgp_local_time **local_time); /// Copy a mgp_local_time. /// Resulting pointer must be freed with mgp_local_time_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_local_time_copy(struct mgp_local_time *local_time, struct mgp_memory *memory, struct mgp_local_time **result); @@ -996,26 +993,26 @@ enum mgp_error mgp_local_time_timestamp(struct mgp_local_time *local_time, int64 /// Get the local time representing current time. /// Resulting pointer must be freed with mgp_local_time_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_local_time_now(struct mgp_memory *memory, struct mgp_local_time **local_time); /// Add a duration to the local time. /// Resulting pointer must be freed with mgp_local_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local time. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local time. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_local_time_add_duration(struct mgp_local_time *local_time, struct mgp_duration *dur, struct mgp_memory *memory, struct mgp_local_time **result); /// Subtract a duration from the local time. /// Resulting pointer must be freed with mgp_local_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local time. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local time. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_local_time_sub_duration(struct mgp_local_time *local_time, struct mgp_duration *dur, struct mgp_memory *memory, struct mgp_local_time **result); /// Get a duration between two local times. /// Resulting pointer must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_date. enum mgp_error mgp_local_time_diff(struct mgp_local_time *first, struct mgp_local_time *second, struct mgp_memory *memory, struct mgp_duration **result); @@ -1026,22 +1023,22 @@ struct mgp_local_date_time_parameters { /// Create a local date-time from a string following the ISO 8601 format. /// Resulting local date-time must be freed with mgp_local_date_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. enum mgp_error mgp_local_date_time_from_string(const char *string, struct mgp_memory *memory, struct mgp_local_date_time **local_date_time); /// Create a local date-time from mgp_local_date_time_parameters. /// Resulting local date-time must be freed with mgp_local_date_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. enum mgp_error mgp_local_date_time_from_parameters(struct mgp_local_date_time_parameters *parameters, struct mgp_memory *memory, struct mgp_local_date_time **local_date_time); /// Copy a mgp_local_date_time. /// Resulting pointer must be freed with mgp_local_date_time_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. enum mgp_error mgp_local_date_time_copy(struct mgp_local_date_time *local_date_time, struct mgp_memory *memory, struct mgp_local_date_time **result); @@ -1081,26 +1078,26 @@ enum mgp_error mgp_local_date_time_timestamp(struct mgp_local_date_time *local_d /// Get the local date-time representing current date and time. /// Resulting local date-time must be freed with mgp_local_date_time_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. enum mgp_error mgp_local_date_time_now(struct mgp_memory *memory, struct mgp_local_date_time **local_date_time); /// Add a duration to the local date-time. /// Resulting local date-time must be freed with mgp_local_date_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local date-time. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local date-time. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. enum mgp_error mgp_local_date_time_add_duration(struct mgp_local_date_time *local_date_time, struct mgp_duration *dur, struct mgp_memory *memory, struct mgp_local_date_time **result); /// Subtract a duration from the local date-time. /// Resulting local date-time must be freed with mgp_local_date_time_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local date-time. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid local date-time. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. enum mgp_error mgp_local_date_time_sub_duration(struct mgp_local_date_time *local_date_time, struct mgp_duration *dur, struct mgp_memory *memory, struct mgp_local_date_time **result); /// Get a duration between two local date-times. /// Resulting duration must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_local_date_time. enum mgp_error mgp_local_date_time_diff(struct mgp_local_date_time *first, struct mgp_local_date_time *second, struct mgp_memory *memory, struct mgp_duration **result); @@ -1115,26 +1112,26 @@ struct mgp_duration_parameters { /// Create a duration from a string following the ISO 8601 format. /// Resulting duration must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the string cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. enum mgp_error mgp_duration_from_string(const char *string, struct mgp_memory *memory, struct mgp_duration **duration); /// Create a duration from mgp_duration_parameters. /// Resulting duration must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the parameters cannot be parsed correctly. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. enum mgp_error mgp_duration_from_parameters(struct mgp_duration_parameters *parameters, struct mgp_memory *memory, struct mgp_duration **duration); /// Create a duration from microseconds. /// Resulting duration must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. enum mgp_error mgp_duration_from_microseconds(int64_t microseconds, struct mgp_memory *memory, struct mgp_duration **duration); /// Copy a mgp_duration. /// Resulting pointer must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. enum mgp_error mgp_duration_copy(struct mgp_duration *duration, struct mgp_memory *memory, struct mgp_duration **result); @@ -1149,20 +1146,20 @@ enum mgp_error mgp_duration_get_microseconds(struct mgp_duration *duration, int6 /// Apply unary minus operator to the duration. /// Resulting pointer must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. enum mgp_error mgp_duration_neg(struct mgp_duration *dur, struct mgp_memory *memory, struct mgp_duration **result); /// Add two durations. /// Resulting pointer must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid duration. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid duration. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. enum mgp_error mgp_duration_add(struct mgp_duration *first, struct mgp_duration *second, struct mgp_memory *memory, struct mgp_duration **result); /// Subtract two durations. /// Resulting pointer must be freed with mgp_duration_destroy. -/// Return MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid duration. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the operation results in an invalid duration. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_duration. enum mgp_error mgp_duration_sub(struct mgp_duration *first, struct mgp_duration *second, struct mgp_memory *memory, struct mgp_duration **result); ///@} @@ -1171,7 +1168,7 @@ enum mgp_error mgp_duration_sub(struct mgp_duration *first, struct mgp_duration /// The previous mgp_vertex obtained through mgp_vertices_iterator_get /// will be invalidated, and you must not use its value. /// Result is NULL if the end of the iteration has been reached. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertex. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate a mgp_vertex. enum mgp_error mgp_vertices_iterator_next(struct mgp_vertices_iterator *it, struct mgp_vertex **result); ///@} @@ -1190,29 +1187,29 @@ struct mgp_type; /// Get the type representing any value that isn't `null`. /// /// The ANY type is the parent type of all types. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_any(struct mgp_type **result); /// Get the type representing boolean values. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_bool(struct mgp_type **result); /// Get the type representing character string values. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_string(struct mgp_type **result); /// Get the type representing integer values. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_int(struct mgp_type **result); /// Get the type representing floating-point values. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_float(struct mgp_type **result); /// Get the type representing any number value. /// /// This is the parent type for numeric types, i.e. INTEGER and FLOAT. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_number(struct mgp_type **result); /// Get the type representing map values. @@ -1224,51 +1221,51 @@ enum mgp_error mgp_type_number(struct mgp_type **result); /// /// @sa mgp_type_node /// @sa mgp_type_relationship -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_map(struct mgp_type **result); /// Get the type representing graph node values. /// /// Since a node contains a map of properties, the node itself is also of MAP /// type. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_node(struct mgp_type **result); /// Get the type representing graph relationship values. /// /// Since a relationship contains a map of properties, the relationship itself /// is also of MAP type. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_relationship(struct mgp_type **result); /// Get the type representing a graph path (walk) from one node to another. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_path(struct mgp_type **result); /// Build a type representing a list of values of given `element_type`. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_list(struct mgp_type *element_type, struct mgp_type **result); /// Get the type representing a date. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_date(struct mgp_type **result); /// Get the type representing a local time. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_local_time(struct mgp_type **result); /// Get the type representing a local date-time. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_local_date_time(struct mgp_type **result); /// Get the type representing a duration. -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_duration(struct mgp_type **result); /// Build a type representing either a `null` value or a value of given `type`. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate the new type. enum mgp_error mgp_type_nullable(struct mgp_type *type, struct mgp_type **result); ///@} @@ -1309,9 +1306,9 @@ typedef void (*mgp_proc_cb)(struct mgp_list *, struct mgp_graph *, struct mgp_re /// Note that Unicode characters are not allowed. Additionally, names are /// case-sensitive. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for mgp_proc. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid procedure name. -/// RETURN MGP_ERROR_LOGIC_ERROR if a procedure with the same name was already registered. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for mgp_proc. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid procedure name. +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if a procedure with the same name was already registered. enum mgp_error mgp_module_add_read_procedure(struct mgp_module *module, const char *name, mgp_proc_cb cb, struct mgp_proc **result); @@ -1320,9 +1317,9 @@ enum mgp_error mgp_module_add_read_procedure(struct mgp_module *module, const ch /// The `name` must be a valid identifier, following the same rules as the /// procedure`name` in mgp_module_add_read_procedure. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for mgp_proc. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid procedure name. -/// RETURN MGP_ERROR_LOGIC_ERROR if a procedure with the same name was already registered. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for mgp_proc. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid procedure name. +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if a procedure with the same name was already registered. enum mgp_error mgp_module_add_write_procedure(struct mgp_module *module, const char *name, mgp_proc_cb cb, struct mgp_proc **result); @@ -1337,9 +1334,9 @@ enum mgp_error mgp_module_add_write_procedure(struct mgp_module *module, const c /// /// Passed in `type` describes what kind of values can be used as the argument. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. -/// RETURN MGP_ERROR_LOGIC_ERROR if the procedure already has any optional argument. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if the procedure already has any optional argument. enum mgp_error mgp_proc_add_arg(struct mgp_proc *proc, const char *name, struct mgp_type *type); /// Add an optional argument with a default value to a procedure. @@ -1360,10 +1357,10 @@ enum mgp_error mgp_proc_add_arg(struct mgp_proc *proc, const char *name, struct /// a graph element (node, relationship, path) and it must satisfy the given /// `type`. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. -/// RETURN MGP_ERROR_VALUE_CONVERSION if `default_value` is a graph element (vertex, edge or path). -/// RETURN MGP_ERROR_LOGIC_ERROR if `default_value` does not satisfy `type`. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. +/// RETURN mgp_error::MGP_ERROR_VALUE_CONVERSION if `default_value` is a graph element (vertex, edge or path). +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if `default_value` does not satisfy `type`. enum mgp_error mgp_proc_add_opt_arg(struct mgp_proc *proc, const char *name, struct mgp_type *type, struct mgp_value *default_value); @@ -1375,9 +1372,9 @@ enum mgp_error mgp_proc_add_opt_arg(struct mgp_proc *proc, const char *name, str /// Passed in `type` describes what kind of values can be returned through the /// result field. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid result name. -/// RETURN MGP_ERROR_LOGIC_ERROR if a result field with the same name was already added. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid result name. +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if a result field with the same name was already added. enum mgp_error mgp_proc_add_result(struct mgp_proc *proc, const char *name, struct mgp_type *type); /// Add a result field to a procedure and mark it as deprecated. @@ -1385,9 +1382,9 @@ enum mgp_error mgp_proc_add_result(struct mgp_proc *proc, const char *name, stru /// This is the same as mgp_proc_add_result, but the result field will be marked /// as deprecated. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid result name. -/// RETURN MGP_ERROR_LOGIC_ERROR if a result field with the same name was already added. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid result name. +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if a result field with the same name was already added. enum mgp_error mgp_proc_add_deprecated_result(struct mgp_proc *proc, const char *name, struct mgp_type *type); ///@} @@ -1439,45 +1436,45 @@ enum mgp_error mgp_message_source_type(struct mgp_message *message, enum mgp_sou /// Supported stream sources: /// - Kafka /// - Pulsar -/// Return MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. enum mgp_error mgp_message_payload(struct mgp_message *message, const char **result); /// Get the payload size /// Supported stream sources: /// - Kafka /// - Pulsar -/// Return MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. enum mgp_error mgp_message_payload_size(struct mgp_message *message, size_t *result); /// Get the name of topic /// Supported stream sources: /// - Kafka /// - Pulsar -/// Return MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. enum mgp_error mgp_message_topic_name(struct mgp_message *message, const char **result); /// Get the key of mgp_message as a byte array /// Supported stream sources: /// - Kafka -/// Return MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. enum mgp_error mgp_message_key(struct mgp_message *message, const char **result); /// Get the key size of mgp_message /// Supported stream sources: /// - Kafka -/// Return MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. enum mgp_error mgp_message_key_size(struct mgp_message *message, size_t *result); /// Get the timestamp of mgp_message as a byte array /// Supported stream sources: /// - Kafka -/// Return MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. enum mgp_error mgp_message_timestamp(struct mgp_message *message, int64_t *result); /// Get the message offset from a message. /// Supported stream sources: /// - Kafka -/// Return MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if the message is from an unsupported stream source. enum mgp_error mgp_message_offset(struct mgp_message *message, int64_t *result); /// Get the number of messages contained in the mgp_messages list @@ -1501,9 +1498,9 @@ typedef void (*mgp_trans_cb)(struct mgp_messages *, struct mgp_graph *, struct m /// Note that Unicode characters are not allowed. Additionally, names are /// case-sensitive. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for transformation. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid transformation name. -/// RETURN MGP_ERROR_LOGIC_ERROR if a transformation with the same name was already registered. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for transformation. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid transformation name. +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if a transformation with the same name was already registered. enum mgp_error mgp_module_add_transformation(struct mgp_module *module, const char *name, mgp_trans_cb cb); /// @} @@ -1525,9 +1522,9 @@ enum mgp_error mgp_module_add_transformation(struct mgp_module *module, const ch /// /// Passed in `type` describes what kind of values can be used as the argument. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. -/// Return MGP_ERROR_LOGIC_ERROR if the function already has any optional argument. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. +/// Return mgp_error::MGP_ERROR_LOGIC_ERROR if the function already has any optional argument. enum mgp_error mgp_func_add_arg(struct mgp_func *func, const char *name, struct mgp_type *type); /// Add an optional argument with a default value to a function. @@ -1547,10 +1544,10 @@ enum mgp_error mgp_func_add_arg(struct mgp_func *func, const char *name, struct /// a graph element (node, relationship, path) and it must satisfy the given /// `type`. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. -/// Return MGP_ERROR_VALUE_CONVERSION if `default_value` is a graph element (vertex, edge or path). -/// Return MGP_ERROR_LOGIC_ERROR if `default_value` does not satisfy `type`. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for an argument. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid argument name. +/// Return mgp_error::MGP_ERROR_VALUE_CONVERSION if `default_value` is a graph element (vertex, edge or path). +/// Return mgp_error::MGP_ERROR_LOGIC_ERROR if `default_value` does not satisfy `type`. enum mgp_error mgp_func_add_opt_arg(struct mgp_func *func, const char *name, struct mgp_type *type, struct mgp_value *default_value); @@ -1568,19 +1565,20 @@ typedef void (*mgp_func_cb)(struct mgp_list *, struct mgp_func_context *, struct /// uppercase Latin letters. The name must begin with a non-digit character. /// Note that Unicode characters are not allowed. /// -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for mgp_func. -/// Return MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid function name. -/// RETURN MGP_ERROR_LOGIC_ERROR if a function with the same name was already registered. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory for mgp_func. +/// Return mgp_error::MGP_ERROR_INVALID_ARGUMENT if `name` is not a valid function name. +/// RETURN mgp_error::MGP_ERROR_LOGIC_ERROR if a function with the same name was already registered. enum mgp_error mgp_module_add_function(struct mgp_module *module, const char *name, mgp_func_cb cb, struct mgp_func **result); /// Set an error message as an output to the Magic function -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if there's no memory for copying the error message. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if there's no memory for copying the error message. enum mgp_error mgp_func_result_set_error_msg(struct mgp_func_result *result, const char *error_msg, struct mgp_memory *memory); /// Set an output value for the Magic function -/// Return MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory to copy the mgp_value to mgp_func_result. +/// Return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE if unable to allocate memory to copy the mgp_value to +/// mgp_func_result. enum mgp_error mgp_func_result_set_value(struct mgp_func_result *result, struct mgp_value *value, struct mgp_memory *memory); /// @} diff --git a/src/query/procedure/mg_procedure_helpers.cpp b/src/query/procedure/mg_procedure_helpers.cpp index 25ddf192e..6b206e7dc 100644 --- a/src/query/procedure/mg_procedure_helpers.cpp +++ b/src/query/procedure/mg_procedure_helpers.cpp @@ -24,7 +24,7 @@ MgpUniquePtr GetStringValueOrSetError(const char *string, mgp_memory } bool InsertResultOrSetError(mgp_result *result, mgp_result_record *record, const char *result_name, mgp_value *value) { - if (const auto err = mgp_result_record_insert(record, result_name, value); err != MGP_ERROR_NO_ERROR) { + if (const auto err = mgp_result_record_insert(record, result_name, value); err != mgp_error::MGP_ERROR_NO_ERROR) { const auto error_msg = fmt::format("Unable to set the result for {}, error = {}", result_name, err); static_cast(mgp_result_set_error_msg(result, error_msg.c_str())); return false; diff --git a/src/query/procedure/mg_procedure_helpers.hpp b/src/query/procedure/mg_procedure_helpers.hpp index 8a8b2200c..cb8bd55db 100644 --- a/src/query/procedure/mg_procedure_helpers.hpp +++ b/src/query/procedure/mg_procedure_helpers.hpp @@ -25,7 +25,7 @@ TResult Call(TFunc func, TArgs... args) { static_assert(std::is_trivially_copyable_v); static_assert((std::is_trivially_copyable_v> && ...)); TResult result{}; - MG_ASSERT(func(args..., &result) == MGP_ERROR_NO_ERROR); + MG_ASSERT(func(args..., &result) == mgp_error::MGP_ERROR_NO_ERROR); return result; } @@ -50,10 +50,10 @@ mgp_error CreateMgpObject(MgpUniquePtr &obj, TFunc func, TArgs &&...args) template [[nodiscard]] bool TryOrSetError(Fun &&func, mgp_result *result) { - if (const auto err = func(); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = func(); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { static_cast(mgp_result_set_error_msg(result, "Not enough memory!")); return false; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { const auto error_msg = fmt::format("Unexpected error ({})!", err); static_cast(mgp_result_set_error_msg(result, error_msg.c_str())); return false; diff --git a/src/query/procedure/mg_procedure_impl.cpp b/src/query/procedure/mg_procedure_impl.cpp index f38f4c6c6..0e707baf7 100644 --- a/src/query/procedure/mg_procedure_impl.cpp +++ b/src/query/procedure/mg_procedure_impl.cpp @@ -143,48 +143,48 @@ template WrapExceptionsHelper(std::forward(func), std::forward(args)...); } catch (const DeletedObjectException &neoe) { spdlog::error("Deleted object error during mg API call: {}", neoe.what()); - return MGP_ERROR_DELETED_OBJECT; + return mgp_error::MGP_ERROR_DELETED_OBJECT; } catch (const KeyAlreadyExistsException &kaee) { spdlog::error("Key already exists error during mg API call: {}", kaee.what()); - return MGP_ERROR_KEY_ALREADY_EXISTS; + return mgp_error::MGP_ERROR_KEY_ALREADY_EXISTS; } catch (const InsufficientBufferException &ibe) { spdlog::error("Insufficient buffer error during mg API call: {}", ibe.what()); - return MGP_ERROR_INSUFFICIENT_BUFFER; + return mgp_error::MGP_ERROR_INSUFFICIENT_BUFFER; } catch (const ImmutableObjectException &ioe) { spdlog::error("Immutable object error during mg API call: {}", ioe.what()); - return MGP_ERROR_IMMUTABLE_OBJECT; + return mgp_error::MGP_ERROR_IMMUTABLE_OBJECT; } catch (const ValueConversionException &vce) { spdlog::error("Value converion error during mg API call: {}", vce.what()); - return MGP_ERROR_VALUE_CONVERSION; + return mgp_error::MGP_ERROR_VALUE_CONVERSION; } catch (const SerializationException &se) { spdlog::error("Serialization error during mg API call: {}", se.what()); - return MGP_ERROR_SERIALIZATION_ERROR; + return mgp_error::MGP_ERROR_SERIALIZATION_ERROR; } catch (const std::bad_alloc &bae) { spdlog::error("Memory allocation error during mg API call: {}", bae.what()); - return MGP_ERROR_UNABLE_TO_ALLOCATE; + return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE; } catch (const memgraph::utils::OutOfMemoryException &oome) { spdlog::error("Memory limit exceeded during mg API call: {}", oome.what()); - return MGP_ERROR_UNABLE_TO_ALLOCATE; + return mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE; } catch (const std::out_of_range &oore) { spdlog::error("Out of range error during mg API call: {}", oore.what()); - return MGP_ERROR_OUT_OF_RANGE; + return mgp_error::MGP_ERROR_OUT_OF_RANGE; } catch (const std::invalid_argument &iae) { spdlog::error("Invalid argument error during mg API call: {}", iae.what()); - return MGP_ERROR_INVALID_ARGUMENT; + return mgp_error::MGP_ERROR_INVALID_ARGUMENT; } catch (const std::logic_error &lee) { spdlog::error("Logic error during mg API call: {}", lee.what()); - return MGP_ERROR_LOGIC_ERROR; + return mgp_error::MGP_ERROR_LOGIC_ERROR; } catch (const std::exception &e) { spdlog::error("Unexpected error during mg API call: {}", e.what()); - return MGP_ERROR_UNKNOWN_ERROR; + return mgp_error::MGP_ERROR_UNKNOWN_ERROR; } catch (const memgraph::utils::temporal::InvalidArgumentException &e) { spdlog::error("Invalid argument was sent to an mg API call for temporal types: {}", e.what()); - return MGP_ERROR_INVALID_ARGUMENT; + return mgp_error::MGP_ERROR_INVALID_ARGUMENT; } catch (...) { spdlog::error("Unexpected error during mg API call"); - return MGP_ERROR_UNKNOWN_ERROR; + return mgp_error::MGP_ERROR_UNKNOWN_ERROR; } - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } // Graph mutations @@ -846,7 +846,7 @@ mgp_value_type MgpValueGetType(const mgp_value &val) noexcept { return val.type; mgp_error mgp_value_get_type(mgp_value *val, mgp_value_type *result) { static_assert(noexcept(MgpValueGetType(*val))); *result = MgpValueGetType(*val); - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } // NOLINTNEXTLINE(cppcoreguidelines-macro-usage) @@ -854,7 +854,7 @@ mgp_error mgp_value_get_type(mgp_value *val, mgp_value_type *result) { mgp_error mgp_value_is_##type_lowercase(mgp_value *val, int *result) { \ static_assert(noexcept(MgpValueGetType(*val))); \ *result = MgpValueGetType(*val) == MGP_VALUE_TYPE_##type_uppercase; \ - return MGP_ERROR_NO_ERROR; \ + return mgp_error::MGP_ERROR_NO_ERROR; \ } DEFINE_MGP_VALUE_IS(null, NULL) @@ -874,27 +874,27 @@ DEFINE_MGP_VALUE_IS(duration, DURATION) mgp_error mgp_value_get_bool(mgp_value *val, int *result) { *result = val->bool_v ? 1 : 0; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_value_get_int(mgp_value *val, int64_t *result) { *result = val->int_v; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_value_get_double(mgp_value *val, double *result) { *result = val->double_v; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_value_get_string(mgp_value *val, const char **result) { static_assert(noexcept(val->string_v.c_str())); *result = val->string_v.c_str(); - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } // NOLINTNEXTLINE(cppcoreguidelines-macro-usage) #define DEFINE_MGP_VALUE_GET(type) \ mgp_error mgp_value_get_##type(mgp_value *val, mgp_##type **result) { \ *result = val->type##_v; \ - return MGP_ERROR_NO_ERROR; \ + return mgp_error::MGP_ERROR_NO_ERROR; \ } DEFINE_MGP_VALUE_GET(list) @@ -940,13 +940,13 @@ mgp_error mgp_list_append_extend(mgp_list *list, mgp_value *val) { mgp_error mgp_list_size(mgp_list *list, size_t *result) { static_assert(noexcept(list->elems.size())); *result = list->elems.size(); - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_list_capacity(mgp_list *list, size_t *result) { static_assert(noexcept(list->elems.capacity())); *result = list->elems.capacity(); - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_list_at(mgp_list *list, size_t i, mgp_value **result) { @@ -978,7 +978,7 @@ mgp_error mgp_map_insert(mgp_map *map, const char *key, mgp_value *value) { mgp_error mgp_map_size(mgp_map *map, size_t *result) { static_assert(noexcept(map->items.size())); *result = map->items.size(); - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_map_at(mgp_map *map, const char *key, mgp_value **result) { @@ -1089,7 +1089,7 @@ size_t MgpPathSize(const mgp_path &path) noexcept { return path.edges.size(); } mgp_error mgp_path_size(mgp_path *path, size_t *result) { *result = MgpPathSize(*path); - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_path_vertex_at(mgp_path *path, size_t i, mgp_vertex **result) { @@ -1690,7 +1690,7 @@ mgp_error mgp_vertex_equal(mgp_vertex *v1, mgp_vertex *v2, int *result) { // NOLINTNEXTLINE(clang-diagnostic-unevaluated-expression) static_assert(noexcept(*result = *v1 == *v2 ? 1 : 0)); *result = *v1 == *v2 ? 1 : 0; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_vertex_labels_count(mgp_vertex *v, size_t *result) { @@ -1950,7 +1950,7 @@ mgp_error mgp_edge_equal(mgp_edge *e1, mgp_edge *e2, int *result) { // NOLINTNEXTLINE(clang-diagnostic-unevaluated-expression) static_assert(noexcept(*result = *e1 == *e2 ? 1 : 0)); *result = *e1 == *e2 ? 1 : 0; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_edge_get_type(mgp_edge *e, mgp_edge_type *result) { @@ -1967,12 +1967,12 @@ mgp_error mgp_edge_get_type(mgp_edge *e, mgp_edge_type *result) { mgp_error mgp_edge_get_from(mgp_edge *e, mgp_vertex **result) { *result = &e->from; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_edge_get_to(mgp_edge *e, mgp_vertex **result) { *result = &e->to; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_edge_get_property(mgp_edge *e, const char *name, mgp_memory *memory, mgp_value **result) { @@ -2082,7 +2082,7 @@ mgp_error mgp_graph_get_vertex_by_id(mgp_graph *graph, mgp_vertex_id id, mgp_mem mgp_error mgp_graph_is_mutable(mgp_graph *graph, int *result) { *result = MgpGraphIsMutable(*graph) ? 1 : 0; - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; }; mgp_error mgp_graph_create_vertex(struct mgp_graph *graph, mgp_memory *memory, mgp_vertex **result) { @@ -2507,7 +2507,7 @@ mgp_error mgp_proc_add_result(mgp_proc *proc, const char *name, mgp_type *type) mgp_error MgpTransAddFixedResult(mgp_trans *trans) noexcept { if (const auto err = AddResultToProp(trans, "query", Call(mgp_type_string), false); - err != MGP_ERROR_NO_ERROR) { + err != mgp_error::MGP_ERROR_NO_ERROR) { return err; } return AddResultToProp(trans, "parameters", Call(mgp_type_nullable, Call(mgp_type_map)), @@ -2754,7 +2754,7 @@ mgp_error mgp_message_offset(struct mgp_message *message, int64_t *result) { mgp_error mgp_messages_size(mgp_messages *messages, size_t *result) { static_assert(noexcept(messages->messages.size())); *result = messages->messages.size(); - return MGP_ERROR_NO_ERROR; + return mgp_error::MGP_ERROR_NO_ERROR; } mgp_error mgp_messages_at(mgp_messages *messages, size_t index, mgp_message **result) { diff --git a/src/query/procedure/module.cpp b/src/query/procedure/module.cpp index 427010803..af3837048 100644 --- a/src/query/procedure/module.cpp +++ b/src/query/procedure/module.cpp @@ -121,18 +121,18 @@ void RegisterMgLoad(ModuleRegistry *module_registry, utils::RWLock *lock, Builti bool succ = false; WithUpgradedLock(lock, [&]() { const char *arg_as_string{nullptr}; - if (const auto err = mgp_value_get_string(arg, &arg_as_string); err != MGP_ERROR_NO_ERROR) { + if (const auto err = mgp_value_get_string(arg, &arg_as_string); err != mgp_error::MGP_ERROR_NO_ERROR) { succ = false; } else { succ = module_registry->LoadOrReloadModuleFromName(arg_as_string); } }); if (!succ) { - MG_ASSERT(mgp_result_set_error_msg(result, "Failed to (re)load the module.") == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_result_set_error_msg(result, "Failed to (re)load the module.") == mgp_error::MGP_ERROR_NO_ERROR); } }; mgp_proc load("load", load_cb, utils::NewDeleteResource()); - MG_ASSERT(mgp_proc_add_arg(&load, "module_name", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&load, "module_name", Call(mgp_type_string)) == mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("load", std::move(load)); } @@ -235,11 +235,16 @@ void RegisterMgProcedures( } }; mgp_proc procedures("procedures", procedures_cb, utils::NewDeleteResource()); - MG_ASSERT(mgp_proc_add_result(&procedures, "name", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&procedures, "signature", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&procedures, "is_write", Call(mgp_type_bool)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&procedures, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&procedures, "is_editable", Call(mgp_type_bool)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "name", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "signature", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "is_write", Call(mgp_type_bool)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "is_editable", Call(mgp_type_bool)) == + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("procedures", std::move(procedures)); } @@ -298,9 +303,12 @@ void RegisterMgTransformations(const std::map(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&procedures, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&procedures, "is_editable", Call(mgp_type_bool)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "name", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&procedures, "is_editable", Call(mgp_type_bool)) == + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("transformations", std::move(procedures)); } @@ -374,10 +382,14 @@ void RegisterMgFunctions( } }; mgp_proc functions("functions", functions_cb, utils::NewDeleteResource()); - MG_ASSERT(mgp_proc_add_result(&functions, "name", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&functions, "signature", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&functions, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&functions, "is_editable", Call(mgp_type_bool)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&functions, "name", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&functions, "signature", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&functions, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&functions, "is_editable", Call(mgp_type_bool)) == + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("functions", std::move(functions)); } namespace { @@ -469,9 +481,10 @@ void RegisterMgGetModuleFiles(ModuleRegistry *module_registry, BuiltinModule *mo mgp_proc get_module_files("get_module_files", get_module_files_cb, utils::NewDeleteResource(), {.required_privilege = AuthQuery::Privilege::MODULE_READ}); - MG_ASSERT(mgp_proc_add_result(&get_module_files, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&get_module_files, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT(mgp_proc_add_result(&get_module_files, "is_editable", Call(mgp_type_bool)) == - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("get_module_files", std::move(get_module_files)); } @@ -530,8 +543,10 @@ void RegisterMgGetModuleFile(ModuleRegistry *module_registry, BuiltinModule *mod }; mgp_proc get_module_file("get_module_file", std::move(get_module_file_cb), utils::NewDeleteResource(), {.required_privilege = AuthQuery::Privilege::MODULE_READ}); - MG_ASSERT(mgp_proc_add_arg(&get_module_file, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&get_module_file, "content", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&get_module_file, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&get_module_file, "content", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("get_module_file", std::move(get_module_file)); } @@ -609,9 +624,12 @@ void RegisterMgCreateModuleFile(ModuleRegistry *module_registry, utils::RWLock * }; mgp_proc create_module_file("create_module_file", std::move(create_module_file_cb), utils::NewDeleteResource(), {.required_privilege = AuthQuery::Privilege::MODULE_WRITE}); - MG_ASSERT(mgp_proc_add_arg(&create_module_file, "filename", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_arg(&create_module_file, "content", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_result(&create_module_file, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&create_module_file, "filename", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&create_module_file, "content", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_result(&create_module_file, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("create_module_file", std::move(create_module_file)); } @@ -664,8 +682,10 @@ void RegisterMgUpdateModuleFile(ModuleRegistry *module_registry, utils::RWLock * }; mgp_proc update_module_file("update_module_file", std::move(update_module_file_cb), utils::NewDeleteResource(), {.required_privilege = AuthQuery::Privilege::MODULE_WRITE}); - MG_ASSERT(mgp_proc_add_arg(&update_module_file, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_arg(&update_module_file, "content", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&update_module_file, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&update_module_file, "content", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("update_module_file", std::move(update_module_file)); } @@ -721,7 +741,8 @@ void RegisterMgDeleteModuleFile(ModuleRegistry *module_registry, utils::RWLock * }; mgp_proc delete_module_file("delete_module_file", std::move(delete_module_file_cb), utils::NewDeleteResource(), {.required_privilege = AuthQuery::Privilege::MODULE_WRITE}); - MG_ASSERT(mgp_proc_add_arg(&delete_module_file, "path", Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&delete_module_file, "path", Call(mgp_type_string)) == + mgp_error::MGP_ERROR_NO_ERROR); module->AddProcedure("delete_module_file", std::move(delete_module_file)); } @@ -832,7 +853,7 @@ bool SharedLibraryModule::Load(const std::filesystem::path &file_path) { return with_error(error); } for (auto &trans : module_def->transformations) { - const bool success = MGP_ERROR_NO_ERROR == MgpTransAddFixedResult(&trans.second); + const bool success = mgp_error::MGP_ERROR_NO_ERROR == MgpTransAddFixedResult(&trans.second); if (!success) { const auto error = fmt::format("Unable to add result to transformation in module {}; add result failed", file_path); @@ -941,7 +962,7 @@ bool PythonModule::Load(const std::filesystem::path &file_path) { auto module_cb = [&](auto *module_def, auto * /*memory*/) { auto result = ImportPyModule(file_path.stem().c_str(), module_def); for (auto &trans : module_def->transformations) { - succ = MgpTransAddFixedResult(&trans.second) == MGP_ERROR_NO_ERROR; + succ = MgpTransAddFixedResult(&trans.second) == mgp_error::MGP_ERROR_NO_ERROR; if (!succ) { return result; } diff --git a/src/query/procedure/py_module.cpp b/src/query/procedure/py_module.cpp index 6aa74d0ab..f106ca284 100644 --- a/src/query/procedure/py_module.cpp +++ b/src/query/procedure/py_module.cpp @@ -55,49 +55,49 @@ PyObject *gMgpSerializationError{nullptr}; // NOLINT(cppcoreguidelines-avo // Returns true if an exception is raised bool RaiseExceptionFromErrorCode(const mgp_error error) { switch (error) { - case MGP_ERROR_NO_ERROR: + case mgp_error::MGP_ERROR_NO_ERROR: return false; - case MGP_ERROR_UNKNOWN_ERROR: { + case mgp_error::MGP_ERROR_UNKNOWN_ERROR: { PyErr_SetString(gMgpUnknownError, "Unknown error happened."); return true; } - case MGP_ERROR_UNABLE_TO_ALLOCATE: { + case mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE: { PyErr_SetString(gMgpUnableToAllocateError, "Unable to allocate memory."); return true; } - case MGP_ERROR_INSUFFICIENT_BUFFER: { + case mgp_error::MGP_ERROR_INSUFFICIENT_BUFFER: { PyErr_SetString(gMgpInsufficientBufferError, "Insufficient buffer."); return true; } - case MGP_ERROR_OUT_OF_RANGE: { + case mgp_error::MGP_ERROR_OUT_OF_RANGE: { PyErr_SetString(gMgpOutOfRangeError, "Out of range."); return true; } - case MGP_ERROR_LOGIC_ERROR: { + case mgp_error::MGP_ERROR_LOGIC_ERROR: { PyErr_SetString(gMgpLogicErrorError, "Logic error."); return true; } - case MGP_ERROR_DELETED_OBJECT: { + case mgp_error::MGP_ERROR_DELETED_OBJECT: { PyErr_SetString(gMgpDeletedObjectError, "Accessing deleted object."); return true; } - case MGP_ERROR_INVALID_ARGUMENT: { + case mgp_error::MGP_ERROR_INVALID_ARGUMENT: { PyErr_SetString(gMgpInvalidArgumentError, "Invalid argument."); return true; } - case MGP_ERROR_KEY_ALREADY_EXISTS: { + case mgp_error::MGP_ERROR_KEY_ALREADY_EXISTS: { PyErr_SetString(gMgpKeyAlreadyExistsError, "Key already exists."); return true; } - case MGP_ERROR_IMMUTABLE_OBJECT: { + case mgp_error::MGP_ERROR_IMMUTABLE_OBJECT: { PyErr_SetString(gMgpImmutableObjectError, "Cannot modify immutable object."); return true; } - case MGP_ERROR_VALUE_CONVERSION: { + case mgp_error::MGP_ERROR_VALUE_CONVERSION: { PyErr_SetString(gMgpValueConversionError, "Value conversion failed."); return true; } - case MGP_ERROR_SERIALIZATION_ERROR: { + case mgp_error::MGP_ERROR_SERIALIZATION_ERROR: { PyErr_SetString(gMgpSerializationError, "Operation cannot be serialized."); return true; } @@ -902,7 +902,7 @@ std::optional AddRecordFromPython(mgp_result *result, py::Obj if (field_val == nullptr) { return py::FetchError(); } - if (mgp_result_record_insert(record, field_name, field_val) != MGP_ERROR_NO_ERROR) { + if (mgp_result_record_insert(record, field_name, field_val) != mgp_error::MGP_ERROR_NO_ERROR) { std::stringstream ss; ss << "Unable to insert field '" << py::Object::FromBorrow(key) << "' with value: '" << py::Object::FromBorrow(val) << "'; did you set the correct field type?"; @@ -2281,9 +2281,10 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { auto py_seq_to_list = [memory](PyObject *seq, Py_ssize_t len, const auto &py_seq_get_item) { static_assert(std::numeric_limits::max() <= std::numeric_limits::max()); MgpUniquePtr list{nullptr, &mgp_list_destroy}; - if (const auto err = CreateMgpObject(list, mgp_list_make_empty, len, memory); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = CreateMgpObject(list, mgp_list_make_empty, len, memory); + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during making mgp_list"}; } for (Py_ssize_t i = 0; i < len; ++i) { @@ -2292,17 +2293,17 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { v = PyObjectToMgpValue(e, memory); const auto err = mgp_list_append(list.get(), v); mgp_value_destroy(v); - if (err != MGP_ERROR_NO_ERROR) { - if (err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (err != mgp_error::MGP_ERROR_NO_ERROR) { + if (err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; } throw std::runtime_error{"Unexpected error during appending to mgp_list"}; } } mgp_value *v{nullptr}; - if (const auto err = mgp_value_make_list(list.get(), &v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_list(list.get(), &v); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during making mgp_value"}; } static_cast(list.release()); @@ -2334,7 +2335,7 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { }; mgp_value *mgp_v{nullptr}; - mgp_error last_error{MGP_ERROR_NO_ERROR}; + mgp_error last_error{mgp_error::MGP_ERROR_NO_ERROR}; if (o == Py_None) { last_error = mgp_value_make_null(memory, &mgp_v); @@ -2360,10 +2361,10 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { MgpUniquePtr map{nullptr, mgp_map_destroy}; const auto map_err = CreateMgpObject(map, mgp_map_make_empty, memory); - if (map_err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (map_err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; } - if (map_err != MGP_ERROR_NO_ERROR) { + if (map_err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during creating mgp_map"}; } @@ -2384,16 +2385,16 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { MgpUniquePtr v{PyObjectToMgpValue(value, memory), mgp_value_destroy}; - if (const auto err = mgp_map_insert(map.get(), k, v.get()); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_map_insert(map.get(), k, v.get()); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during inserting an item to mgp_map"}; } } - if (const auto err = mgp_value_make_map(map.get(), &mgp_v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_map(map.get(), &mgp_v); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during creating mgp_value"}; } static_cast(map.release()); @@ -2402,14 +2403,14 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { // Copy the edge and pass the ownership to the created mgp_value. if (const auto err = CreateMgpObject(e, mgp_edge_copy, reinterpret_cast(o)->edge, memory); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during copying mgp_edge"}; } - if (const auto err = mgp_value_make_edge(e.get(), &mgp_v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_edge(e.get(), &mgp_v); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during copying mgp_edge"}; } static_cast(e.release()); @@ -2418,14 +2419,14 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { // Copy the edge and pass the ownership to the created mgp_value. if (const auto err = CreateMgpObject(p, mgp_path_copy, reinterpret_cast(o)->path, memory); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during copying mgp_path"}; } - if (const auto err = mgp_value_make_path(p.get(), &mgp_v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_path(p.get(), &mgp_v); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during copying mgp_path"}; } static_cast(p.release()); @@ -2434,14 +2435,14 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { // Copy the edge and pass the ownership to the created mgp_value. if (const auto err = CreateMgpObject(v, mgp_vertex_copy, reinterpret_cast(o)->vertex, memory); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during copying mgp_vertex"}; } - if (const auto err = mgp_value_make_vertex(v.get(), &mgp_v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_vertex(v.get(), &mgp_v); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error during copying mgp_vertex"}; } static_cast(v.release()); @@ -2474,14 +2475,14 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { MgpUniquePtr date{nullptr, mgp_date_destroy}; if (const auto err = CreateMgpObject(date, mgp_date_from_parameters, ¶meters, memory); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_date"}; } - if (const auto err = mgp_value_make_date(date.get(), &mgp_v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_date(date.get(), &mgp_v); err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_value"}; } static_cast(date.release()); @@ -2499,14 +2500,15 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { MgpUniquePtr local_time{nullptr, mgp_local_time_destroy}; if (const auto err = CreateMgpObject(local_time, mgp_local_time_from_parameters, ¶meters, memory); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_local_time"}; } - if (const auto err = mgp_value_make_local_time(local_time.get(), &mgp_v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_local_time(local_time.get(), &mgp_v); + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_value"}; } static_cast(local_time.release()); @@ -2531,15 +2533,15 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { MgpUniquePtr local_date_time{nullptr, mgp_local_date_time_destroy}; if (const auto err = CreateMgpObject(local_date_time, mgp_local_date_time_from_parameters, ¶meters, memory); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_local_date_time"}; } if (const auto err = mgp_value_make_local_date_time(local_date_time.get(), &mgp_v); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_value"}; } static_cast(local_date_time.release()); @@ -2558,14 +2560,15 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { MgpUniquePtr duration{nullptr, mgp_duration_destroy}; if (const auto err = CreateMgpObject(duration, mgp_duration_from_microseconds, microseconds, memory); - err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_duration"}; } - if (const auto err = mgp_value_make_duration(duration.get(), &mgp_v); err == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (const auto err = mgp_value_make_duration(duration.get(), &mgp_v); + err == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; - } else if (err != MGP_ERROR_NO_ERROR) { + } else if (err != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_value"}; } static_cast(duration.release()); @@ -2573,10 +2576,10 @@ mgp_value *PyObjectToMgpValue(PyObject *o, mgp_memory *memory) { throw std::invalid_argument("Unsupported PyObject conversion"); } - if (last_error == MGP_ERROR_UNABLE_TO_ALLOCATE) { + if (last_error == mgp_error::MGP_ERROR_UNABLE_TO_ALLOCATE) { throw std::bad_alloc{}; } - if (last_error != MGP_ERROR_NO_ERROR) { + if (last_error != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error{"Unexpected error while creating mgp_value"}; } diff --git a/src/query/stream/streams.cpp b/src/query/stream/streams.cpp index 1c7a10e03..7e2dd5c6b 100644 --- a/src/query/stream/streams.cpp +++ b/src/query/stream/streams.cpp @@ -181,25 +181,27 @@ void Streams::RegisterKafkaProcedures() { const auto offset = procedure::Call(mgp_value_get_int, arg_offset); auto lock_ptr = streams_.Lock(); auto it = GetStream(*lock_ptr, std::string(stream_name)); - std::visit(utils::Overloaded{ - [&](StreamData &kafka_stream) { - auto stream_source_ptr = kafka_stream.stream_source->Lock(); - const auto error = stream_source_ptr->SetStreamOffset(offset); - if (error.HasError()) { - MG_ASSERT(mgp_result_set_error_msg(result, error.GetError().c_str()) == MGP_ERROR_NO_ERROR, - "Unable to set procedure error message of procedure: {}", proc_name); - } - }, - [](auto && /*other*/) { - throw QueryRuntimeException("'{}' can be only used for Kafka stream sources", proc_name); - }}, + std::visit(utils::Overloaded{[&](StreamData &kafka_stream) { + auto stream_source_ptr = kafka_stream.stream_source->Lock(); + const auto error = stream_source_ptr->SetStreamOffset(offset); + if (error.HasError()) { + MG_ASSERT(mgp_result_set_error_msg(result, error.GetError().c_str()) == + mgp_error::MGP_ERROR_NO_ERROR, + "Unable to set procedure error message of procedure: {}", proc_name); + } + }, + [](auto && /*other*/) { + throw QueryRuntimeException("'{}' can be only used for Kafka stream sources", + proc_name); + }}, it->second); }; mgp_proc proc(proc_name, set_stream_offset, utils::NewDeleteResource()); MG_ASSERT(mgp_proc_add_arg(&proc, "stream_name", procedure::Call(mgp_type_string)) == - MGP_ERROR_NO_ERROR); - MG_ASSERT(mgp_proc_add_arg(&proc, "offset", procedure::Call(mgp_type_int)) == MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); + MG_ASSERT(mgp_proc_add_arg(&proc, "offset", procedure::Call(mgp_type_int)) == + mgp_error::MGP_ERROR_NO_ERROR); procedure::gModuleRegistry.RegisterMgProcedure(proc_name, std::move(proc)); } @@ -345,19 +347,19 @@ void Streams::RegisterKafkaProcedures() { mgp_proc proc(proc_name, get_stream_info, utils::NewDeleteResource()); MG_ASSERT(mgp_proc_add_arg(&proc, "stream_name", procedure::Call(mgp_type_string)) == - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT(mgp_proc_add_result(&proc, consumer_group_result_name.data(), - procedure::Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + procedure::Call(mgp_type_string)) == mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT( mgp_proc_add_result(&proc, topics_result_name.data(), procedure::Call(mgp_type_list, procedure::Call(mgp_type_string))) == - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT(mgp_proc_add_result(&proc, bootstrap_servers_result_name.data(), - procedure::Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + procedure::Call(mgp_type_string)) == mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT(mgp_proc_add_result(&proc, configs_result_name.data(), procedure::Call(mgp_type_map)) == - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT(mgp_proc_add_result(&proc, credentials_result_name.data(), procedure::Call(mgp_type_map)) == - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); procedure::gModuleRegistry.RegisterMgProcedure(proc_name, std::move(proc)); } @@ -432,14 +434,14 @@ void Streams::RegisterPulsarProcedures() { mgp_proc proc(proc_name, get_stream_info, utils::NewDeleteResource()); MG_ASSERT(mgp_proc_add_arg(&proc, "stream_name", procedure::Call(mgp_type_string)) == - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT(mgp_proc_add_result(&proc, service_url_result_name.data(), - procedure::Call(mgp_type_string)) == MGP_ERROR_NO_ERROR); + procedure::Call(mgp_type_string)) == mgp_error::MGP_ERROR_NO_ERROR); MG_ASSERT( mgp_proc_add_result(&proc, topics_result_name.data(), procedure::Call(mgp_type_list, procedure::Call(mgp_type_string))) == - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); procedure::gModuleRegistry.RegisterMgProcedure(proc_name, std::move(proc)); } diff --git a/tests/e2e/magic_functions/functions/c_read.cpp b/tests/e2e/magic_functions/functions/c_read.cpp index 87c42bc04..75c023e58 100644 --- a/tests/e2e/magic_functions/functions/c_read.cpp +++ b/tests/e2e/magic_functions/functions/c_read.cpp @@ -21,13 +21,13 @@ static void ReturnFunctionArgument(struct mgp_list *args, mgp_func_context *ctx, struct mgp_memory *memory) { mgp_value *value{nullptr}; auto err_code = mgp_list_at(args, 0, &value); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Failed to fetch list!", memory); return; } err_code = mgp_func_result_set_value(result, value, memory); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Failed to construct return value!", memory); return; } @@ -37,13 +37,13 @@ static void ReturnOptionalArgument(struct mgp_list *args, mgp_func_context *ctx, struct mgp_memory *memory) { mgp_value *value{nullptr}; auto err_code = mgp_list_at(args, 0, &value); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Failed to fetch list!", memory); return; } err_code = mgp_func_result_set_value(result, value, memory); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Failed to construct return value!", memory); return; } @@ -51,7 +51,7 @@ static void ReturnOptionalArgument(struct mgp_list *args, mgp_func_context *ctx, double GetElementFromArg(struct mgp_list *args, int index) { mgp_value *value{nullptr}; - if (mgp_list_at(args, index, &value) != MGP_ERROR_NO_ERROR) { + if (mgp_list_at(args, index, &value) != mgp_error::MGP_ERROR_NO_ERROR) { throw std::runtime_error("Error while argument fetching."); } @@ -87,7 +87,7 @@ static void AddTwoNumbers(struct mgp_list *args, mgp_func_context *ctx, mgp_func memgraph::utils::OnScopeExit delete_summation_value([&value] { mgp_value_destroy(value); }); auto err_code = mgp_func_result_set_value(result, value, memory); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Failed to construct return value!", memory); } } @@ -99,7 +99,7 @@ static void ReturnNull(struct mgp_list *args, mgp_func_context *ctx, mgp_func_re memgraph::utils::OnScopeExit delete_null([&value] { mgp_value_destroy(value); }); auto err_code = mgp_func_result_set_value(result, value, memory); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Failed to fetch list!", memory); } } @@ -111,14 +111,14 @@ extern "C" int mgp_init_module(struct mgp_module *module, struct mgp_memory *mem { mgp_func *func{nullptr}; auto err_code = mgp_module_add_function(module, "return_function_argument", ReturnFunctionArgument, &func); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } mgp_type *type_any{nullptr}; mgp_type_any(&type_any); err_code = mgp_func_add_arg(func, "argument", type_any); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } } @@ -126,7 +126,7 @@ extern "C" int mgp_init_module(struct mgp_module *module, struct mgp_memory *mem { mgp_func *func{nullptr}; auto err_code = mgp_module_add_function(module, "return_optional_argument", ReturnOptionalArgument, &func); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } @@ -137,7 +137,7 @@ extern "C" int mgp_init_module(struct mgp_module *module, struct mgp_memory *mem mgp_type *type_int{nullptr}; mgp_type_int(&type_int); err_code = mgp_func_add_opt_arg(func, "opt_argument", type_int, default_value); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } } @@ -145,18 +145,18 @@ extern "C" int mgp_init_module(struct mgp_module *module, struct mgp_memory *mem { mgp_func *func{nullptr}; auto err_code = mgp_module_add_function(module, "add_two_numbers", AddTwoNumbers, &func); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } mgp_type *type_number{nullptr}; mgp_type_number(&type_number); err_code = mgp_func_add_arg(func, "first", type_number); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } err_code = mgp_func_add_arg(func, "second", type_number); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } } @@ -164,7 +164,7 @@ extern "C" int mgp_init_module(struct mgp_module *module, struct mgp_memory *mem { mgp_func *func{nullptr}; auto err_code = mgp_module_add_function(module, "return_null", ReturnNull, &func); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } } diff --git a/tests/e2e/magic_functions/functions/c_write.cpp b/tests/e2e/magic_functions/functions/c_write.cpp index a95ab2192..76a2ad08e 100644 --- a/tests/e2e/magic_functions/functions/c_write.cpp +++ b/tests/e2e/magic_functions/functions/c_write.cpp @@ -26,13 +26,13 @@ static void TryToWrite(struct mgp_list *args, mgp_func_context *ctx, mgp_func_re // Setting a property should set an error auto err_code = mgp_vertex_set_property(vertex, name, value); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Cannot set property in the function!", memory); return; } err_code = mgp_func_result_set_value(result, value, memory); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { mgp_func_result_set_error_msg(result, "Failed to construct return value!", memory); return; } @@ -44,21 +44,21 @@ extern "C" int mgp_init_module(struct mgp_module *module, struct mgp_memory *mem { mgp_func *func{nullptr}; auto err_code = mgp_module_add_function(module, "try_to_write", TryToWrite, &func); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } mgp_type *type_vertex{nullptr}; mgp_type_node(&type_vertex); err_code = mgp_func_add_arg(func, "argument", type_vertex); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } mgp_type *type_string{nullptr}; mgp_type_string(&type_string); err_code = mgp_func_add_arg(func, "name", type_string); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } @@ -67,7 +67,7 @@ extern "C" int mgp_init_module(struct mgp_module *module, struct mgp_memory *mem mgp_type *nullable_type{nullptr}; mgp_type_nullable(any_type, &nullable_type); err_code = mgp_func_add_arg(func, "value", nullable_type); - if (err_code != MGP_ERROR_NO_ERROR) { + if (err_code != mgp_error::MGP_ERROR_NO_ERROR) { return 1; } } diff --git a/tests/e2e/streams/transformations/c_transformations.cpp b/tests/e2e/streams/transformations/c_transformations.cpp index 984a04c61..3241f3f66 100644 --- a/tests/e2e/streams/transformations/c_transformations.cpp +++ b/tests/e2e/streams/transformations/c_transformations.cpp @@ -14,7 +14,7 @@ extern "C" int mgp_init_module(mgp_module *module, mgp_memory *memory) { static const auto no_op_cb = [](mgp_messages *msg, mgp_graph *graph, mgp_result *result, mgp_memory *memory) {}; - if (MGP_ERROR_NO_ERROR != mgp_module_add_transformation(module, "empty_transformation", no_op_cb)) { + if (mgp_error::MGP_ERROR_NO_ERROR != mgp_module_add_transformation(module, "empty_transformation", no_op_cb)) { return 1; } diff --git a/tests/unit/mgp_trans_c_api.cpp b/tests/unit/mgp_trans_c_api.cpp index 2d62a3c16..a9a81b158 100644 --- a/tests/unit/mgp_trans_c_api.cpp +++ b/tests/unit/mgp_trans_c_api.cpp @@ -23,13 +23,13 @@ TEST(MgpTransTest, TestMgpTransApi) { // for different string cases as these are all handled by // IsValidIdentifier(). // Maybe add a mock instead and expect IsValidIdentifier() to be called once? - EXPECT_EQ(mgp_module_add_transformation(&module, "dash-dash", no_op_cb), MGP_ERROR_INVALID_ARGUMENT); + EXPECT_EQ(mgp_module_add_transformation(&module, "dash-dash", no_op_cb), mgp_error::MGP_ERROR_INVALID_ARGUMENT); EXPECT_TRUE(module.transformations.empty()); - EXPECT_EQ(mgp_module_add_transformation(&module, "transform", no_op_cb), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_transformation(&module, "transform", no_op_cb), mgp_error::MGP_ERROR_NO_ERROR); EXPECT_NE(module.transformations.find("transform"), module.transformations.end()); // Try to register a transformation twice - EXPECT_EQ(mgp_module_add_transformation(&module, "transform", no_op_cb), MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_module_add_transformation(&module, "transform", no_op_cb), mgp_error::MGP_ERROR_LOGIC_ERROR); EXPECT_TRUE(module.transformations.size() == 1); } diff --git a/tests/unit/query_function_mgp_module.cpp b/tests/unit/query_function_mgp_module.cpp index 457d6a236..d945bf3d7 100644 --- a/tests/unit/query_function_mgp_module.cpp +++ b/tests/unit/query_function_mgp_module.cpp @@ -25,25 +25,26 @@ TEST(Module, InvalidFunctionRegistration) { mgp_module module(memgraph::utils::NewDeleteResource()); mgp_func *func{nullptr}; // Other test cases are covered within the procedure API. This is only sanity check - EXPECT_EQ(mgp_module_add_function(&module, "dashes-not-supported", DummyCallback, &func), MGP_ERROR_INVALID_ARGUMENT); + EXPECT_EQ(mgp_module_add_function(&module, "dashes-not-supported", DummyCallback, &func), + mgp_error::MGP_ERROR_INVALID_ARGUMENT); } TEST(Module, RegisterSameFunctionMultipleTimes) { mgp_module module(memgraph::utils::NewDeleteResource()); mgp_func *func{nullptr}; EXPECT_EQ(module.functions.find("same_name"), module.functions.end()); - EXPECT_EQ(mgp_module_add_function(&module, "same_name", DummyCallback, &func), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_function(&module, "same_name", DummyCallback, &func), mgp_error::MGP_ERROR_NO_ERROR); EXPECT_NE(module.functions.find("same_name"), module.functions.end()); - EXPECT_EQ(mgp_module_add_function(&module, "same_name", DummyCallback, &func), MGP_ERROR_LOGIC_ERROR); - EXPECT_EQ(mgp_module_add_function(&module, "same_name", DummyCallback, &func), MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_module_add_function(&module, "same_name", DummyCallback, &func), mgp_error::MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_module_add_function(&module, "same_name", DummyCallback, &func), mgp_error::MGP_ERROR_LOGIC_ERROR); EXPECT_NE(module.functions.find("same_name"), module.functions.end()); } TEST(Module, CaseSensitiveFunctionNames) { mgp_module module(memgraph::utils::NewDeleteResource()); mgp_func *func{nullptr}; - EXPECT_EQ(mgp_module_add_function(&module, "not_same", DummyCallback, &func), MGP_ERROR_NO_ERROR); - EXPECT_EQ(mgp_module_add_function(&module, "NoT_saME", DummyCallback, &func), MGP_ERROR_NO_ERROR); - EXPECT_EQ(mgp_module_add_function(&module, "NOT_SAME", DummyCallback, &func), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_function(&module, "not_same", DummyCallback, &func), mgp_error::MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_function(&module, "NoT_saME", DummyCallback, &func), mgp_error::MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_function(&module, "NOT_SAME", DummyCallback, &func), mgp_error::MGP_ERROR_NO_ERROR); EXPECT_EQ(module.functions.size(), 3U); } diff --git a/tests/unit/query_procedure_mgp_module.cpp b/tests/unit/query_procedure_mgp_module.cpp index 1cbfd1518..a9f99a549 100644 --- a/tests/unit/query_procedure_mgp_module.cpp +++ b/tests/unit/query_procedure_mgp_module.cpp @@ -25,30 +25,34 @@ TEST(Module, InvalidProcedureRegistration) { mgp_module module(memgraph::utils::NewDeleteResource()); mgp_proc *proc{nullptr}; EXPECT_EQ(mgp_module_add_read_procedure(&module, "dashes-not-supported", DummyCallback, &proc), - MGP_ERROR_INVALID_ARGUMENT); + mgp_error::MGP_ERROR_INVALID_ARGUMENT); // as u8string this is u8"unicode\u22c6not\u2014supported" EXPECT_EQ(mgp_module_add_read_procedure(&module, "unicode\xE2\x8B\x86not\xE2\x80\x94supported", DummyCallback, &proc), - MGP_ERROR_INVALID_ARGUMENT); + mgp_error::MGP_ERROR_INVALID_ARGUMENT); // as u8string this is u8"`backticks⋆\u22c6won't-save\u2014you`" EXPECT_EQ( mgp_module_add_read_procedure(&module, "`backticks⋆\xE2\x8B\x86won't-save\xE2\x80\x94you`", DummyCallback, &proc), - MGP_ERROR_INVALID_ARGUMENT); + mgp_error::MGP_ERROR_INVALID_ARGUMENT); EXPECT_EQ(mgp_module_add_read_procedure(&module, "42_name_must_not_start_with_number", DummyCallback, &proc), - MGP_ERROR_INVALID_ARGUMENT); - EXPECT_EQ(mgp_module_add_read_procedure(&module, "div/", DummyCallback, &proc), MGP_ERROR_INVALID_ARGUMENT); - EXPECT_EQ(mgp_module_add_read_procedure(&module, "mul*", DummyCallback, &proc), MGP_ERROR_INVALID_ARGUMENT); + mgp_error::MGP_ERROR_INVALID_ARGUMENT); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "div/", DummyCallback, &proc), + mgp_error::MGP_ERROR_INVALID_ARGUMENT); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "mul*", DummyCallback, &proc), + mgp_error::MGP_ERROR_INVALID_ARGUMENT); EXPECT_EQ(mgp_module_add_read_procedure(&module, "question_mark_is_not_valid?", DummyCallback, &proc), - MGP_ERROR_INVALID_ARGUMENT); + mgp_error::MGP_ERROR_INVALID_ARGUMENT); } TEST(Module, RegisteringTheSameProcedureMultipleTimes) { mgp_module module(memgraph::utils::NewDeleteResource()); mgp_proc *proc{nullptr}; EXPECT_EQ(module.procedures.find("same_name"), module.procedures.end()); - EXPECT_EQ(mgp_module_add_read_procedure(&module, "same_name", DummyCallback, &proc), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "same_name", DummyCallback, &proc), mgp_error::MGP_ERROR_NO_ERROR); EXPECT_NE(module.procedures.find("same_name"), module.procedures.end()); - EXPECT_EQ(mgp_module_add_read_procedure(&module, "same_name", DummyCallback, &proc), MGP_ERROR_LOGIC_ERROR); - EXPECT_EQ(mgp_module_add_read_procedure(&module, "same_name", DummyCallback, &proc), MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "same_name", DummyCallback, &proc), + mgp_error::MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "same_name", DummyCallback, &proc), + mgp_error::MGP_ERROR_LOGIC_ERROR); EXPECT_NE(module.procedures.find("same_name"), module.procedures.end()); } @@ -56,9 +60,9 @@ TEST(Module, CaseSensitiveProcedureNames) { mgp_module module(memgraph::utils::NewDeleteResource()); EXPECT_TRUE(module.procedures.empty()); mgp_proc *proc{nullptr}; - EXPECT_EQ(mgp_module_add_read_procedure(&module, "not_same", DummyCallback, &proc), MGP_ERROR_NO_ERROR); - EXPECT_EQ(mgp_module_add_read_procedure(&module, "NoT_saME", DummyCallback, &proc), MGP_ERROR_NO_ERROR); - EXPECT_EQ(mgp_module_add_read_procedure(&module, "NOT_SAME", DummyCallback, &proc), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "not_same", DummyCallback, &proc), mgp_error::MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "NoT_saME", DummyCallback, &proc), mgp_error::MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_module_add_read_procedure(&module, "NOT_SAME", DummyCallback, &proc), mgp_error::MGP_ERROR_NO_ERROR); EXPECT_EQ(module.procedures.size(), 3U); } @@ -73,37 +77,41 @@ TEST(Module, ProcedureSignature) { mgp_module module(memgraph::utils::NewDeleteResource()); auto *proc = EXPECT_MGP_NO_ERROR(mgp_proc *, mgp_module_add_read_procedure, &module, "proc", &DummyCallback); CheckSignature(proc, "proc() :: ()"); - EXPECT_EQ(mgp_proc_add_arg(proc, "arg1", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_number)), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_proc_add_arg(proc, "arg1", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_number)), + mgp_error::MGP_ERROR_NO_ERROR); CheckSignature(proc, "proc(arg1 :: NUMBER) :: ()"); EXPECT_EQ(mgp_proc_add_opt_arg( proc, "opt1", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_nullable, EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any)), test_utils::CreateValueOwningPtr(EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_null, &memory)).get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); CheckSignature(proc, "proc(arg1 :: NUMBER, opt1 = Null :: ANY?) :: ()"); EXPECT_EQ( mgp_proc_add_result( proc, "res1", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_list, EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_int))), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); CheckSignature(proc, "proc(arg1 :: NUMBER, opt1 = Null :: ANY?) :: (res1 :: LIST OF INTEGER)"); - EXPECT_EQ(mgp_proc_add_arg(proc, "arg2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_number)), MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_proc_add_arg(proc, "arg2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_number)), + mgp_error::MGP_ERROR_LOGIC_ERROR); CheckSignature(proc, "proc(arg1 :: NUMBER, opt1 = Null :: ANY?) :: (res1 :: LIST OF INTEGER)"); - EXPECT_EQ(mgp_proc_add_arg(proc, "arg2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_map)), MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_proc_add_arg(proc, "arg2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_map)), + mgp_error::MGP_ERROR_LOGIC_ERROR); CheckSignature(proc, "proc(arg1 :: NUMBER, opt1 = Null :: ANY?) :: (res1 :: LIST OF INTEGER)"); EXPECT_EQ(mgp_proc_add_deprecated_result(proc, "res2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_string)), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); CheckSignature(proc, "proc(arg1 :: NUMBER, opt1 = Null :: ANY?) :: " "(res1 :: LIST OF INTEGER, DEPRECATED res2 :: STRING)"); - EXPECT_EQ(mgp_proc_add_result(proc, "res2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any)), MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_proc_add_result(proc, "res2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any)), + mgp_error::MGP_ERROR_LOGIC_ERROR); EXPECT_EQ(mgp_proc_add_deprecated_result(proc, "res1", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any)), - MGP_ERROR_LOGIC_ERROR); + mgp_error::MGP_ERROR_LOGIC_ERROR); EXPECT_EQ( mgp_proc_add_opt_arg(proc, "opt2", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_string), test_utils::CreateValueOwningPtr( EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_string, "string=\"value\"", &memory)) .get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); CheckSignature(proc, "proc(arg1 :: NUMBER, opt1 = Null :: ANY?, " "opt2 = \"string=\\\"value\\\"\" :: STRING) :: " @@ -118,7 +126,7 @@ TEST(Module, ProcedureSignatureOnlyOptArg) { proc, "opt1", EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_nullable, EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any)), test_utils::CreateValueOwningPtr(EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_null, &memory)).get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); CheckSignature(proc, "proc(opt1 = Null :: ANY?) :: ()"); } diff --git a/tests/unit/query_procedure_mgp_type.cpp b/tests/unit/query_procedure_mgp_type.cpp index bed4716c2..c9a2800b7 100644 --- a/tests/unit/query_procedure_mgp_type.cpp +++ b/tests/unit/query_procedure_mgp_type.cpp @@ -207,7 +207,7 @@ TEST(CypherType, MapSatisfiesType) { mgp_map_insert( map, "key", test_utils::CreateValueOwningPtr(EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_int, 42, &memory)).get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); auto *mgp_map_v = EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_map, map); const memgraph::query::TypedValue tv_map( std::map{{"key", memgraph::query::TypedValue(42)}}); @@ -287,7 +287,7 @@ TEST(CypherType, PathSatisfiesType) { ASSERT_TRUE(path); alloc.delete_object(mgp_vertex_v); auto mgp_edge_v = alloc.new_object(edge, &graph); - ASSERT_EQ(mgp_path_expand(path, mgp_edge_v), MGP_ERROR_NO_ERROR); + ASSERT_EQ(mgp_path_expand(path, mgp_edge_v), mgp_error::MGP_ERROR_NO_ERROR); alloc.delete_object(mgp_edge_v); auto *mgp_path_v = EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_path, path); const memgraph::query::TypedValue tv_path(memgraph::query::Path(v1, edge, v2)); @@ -343,7 +343,7 @@ TEST(CypherType, ListOfIntSatisfiesType) { mgp_list_append( list, test_utils::CreateValueOwningPtr(EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_int, i, &memory)).get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); tv_list.ValueList().emplace_back(i); auto valid_types = MakeListTypes({EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any), EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_int), @@ -371,14 +371,14 @@ TEST(CypherType, ListOfIntAndBoolSatisfiesType) { mgp_list_append( list, test_utils::CreateValueOwningPtr(EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_int, 42, &memory)).get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); tv_list.ValueList().emplace_back(42); // Add a boolean ASSERT_EQ( mgp_list_append( list, test_utils::CreateValueOwningPtr(EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_bool, 1, &memory)).get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); tv_list.ValueList().emplace_back(true); auto valid_types = MakeListTypes({EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any)}); valid_types.push_back(EXPECT_MGP_NO_ERROR(mgp_type *, mgp_type_any)); @@ -402,7 +402,7 @@ TEST(CypherType, ListOfNullSatisfiesType) { ASSERT_EQ( mgp_list_append( list, test_utils::CreateValueOwningPtr(EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_null, &memory)).get()), - MGP_ERROR_NO_ERROR); + mgp_error::MGP_ERROR_NO_ERROR); tv_list.ValueList().emplace_back(); // List with Null satisfies all nullable list element types std::vector primitive_types{ diff --git a/tests/unit/query_procedure_py_module.cpp b/tests/unit/query_procedure_py_module.cpp index 68459e372..883489643 100644 --- a/tests/unit/query_procedure_py_module.cpp +++ b/tests/unit/query_procedure_py_module.cpp @@ -30,13 +30,13 @@ TEST(PyModule, MgpValueToPyObject) { EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_double, 0.1, &memory), EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_string, "some text", &memory)}; for (auto *val : primitive_values) { - EXPECT_EQ(mgp_list_append(list, val), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_list_append(list, val), mgp_error::MGP_ERROR_NO_ERROR); mgp_value_destroy(val); } } auto *list_val = EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_list, list); auto *map = EXPECT_MGP_NO_ERROR(mgp_map *, mgp_map_make_empty, &memory); - EXPECT_EQ(mgp_map_insert(map, "list", list_val), MGP_ERROR_NO_ERROR); + EXPECT_EQ(mgp_map_insert(map, "list", list_val), mgp_error::MGP_ERROR_NO_ERROR); mgp_value_destroy(list_val); auto *map_val = EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_map, map); auto gil = memgraph::py::EnsureGIL(); @@ -218,7 +218,7 @@ TEST(PyModule, PyPath) { ASSERT_TRUE(edges_it); for (auto *edge = EXPECT_MGP_NO_ERROR(mgp_edge *, mgp_edges_iterator_get, edges_it); edge != nullptr; edge = EXPECT_MGP_NO_ERROR(mgp_edge *, mgp_edges_iterator_next, edges_it)) { - ASSERT_EQ(mgp_path_expand(path, edge), MGP_ERROR_NO_ERROR); + ASSERT_EQ(mgp_path_expand(path, edge), mgp_error::MGP_ERROR_NO_ERROR); } ASSERT_EQ(EXPECT_MGP_NO_ERROR(size_t, mgp_path_size, path), 1); mgp_edges_iterator_destroy(edges_it); diff --git a/tests/unit/query_procedures_mgp_graph.cpp b/tests/unit/query_procedures_mgp_graph.cpp index 25580bc2b..9b00ec4f8 100644 --- a/tests/unit/query_procedures_mgp_graph.cpp +++ b/tests/unit/query_procedures_mgp_graph.cpp @@ -31,7 +31,7 @@ #include "test_utils.hpp" #include "utils/memory.hpp" -#define EXPECT_SUCCESS(...) EXPECT_EQ(__VA_ARGS__, MGP_ERROR_NO_ERROR) +#define EXPECT_SUCCESS(...) EXPECT_EQ(__VA_ARGS__, mgp_error::MGP_ERROR_NO_ERROR) namespace { struct MgpEdgeDeleter { @@ -193,7 +193,7 @@ TEST_F(MgpGraphTest, DetachDeleteVertex) { EXPECT_EQ(CountVertices(read_uncommited_accessor, memgraph::storage::View::NEW), 2); MgpVertexPtr vertex{EXPECT_MGP_NO_ERROR(mgp_vertex *, mgp_graph_get_vertex_by_id, &graph, mgp_vertex_id{vertex_ids.front().AsInt()}, &memory)}; - EXPECT_EQ(mgp_graph_delete_vertex(&graph, vertex.get()), MGP_ERROR_LOGIC_ERROR); + EXPECT_EQ(mgp_graph_delete_vertex(&graph, vertex.get()), mgp_error::MGP_ERROR_LOGIC_ERROR); EXPECT_EQ(CountVertices(read_uncommited_accessor, memgraph::storage::View::NEW), 2); EXPECT_SUCCESS(mgp_graph_detach_delete_vertex(&graph, vertex.get())); EXPECT_EQ(CountVertices(read_uncommited_accessor, memgraph::storage::View::NEW), 1); @@ -212,14 +212,14 @@ TEST_F(MgpGraphTest, CreateDeleteWithImmutableGraph) { mgp_graph immutable_graph = CreateGraph(memgraph::storage::View::OLD); mgp_vertex *raw_vertex{nullptr}; - EXPECT_EQ(mgp_graph_create_vertex(&immutable_graph, &memory, &raw_vertex), MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_graph_create_vertex(&immutable_graph, &memory, &raw_vertex), mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); MgpVertexPtr created_vertex{raw_vertex}; EXPECT_EQ(created_vertex, nullptr); EXPECT_EQ(CountVertices(read_uncommited_accessor, memgraph::storage::View::NEW), 1); MgpVertexPtr vertex_to_delete{EXPECT_MGP_NO_ERROR(mgp_vertex *, mgp_graph_get_vertex_by_id, &immutable_graph, mgp_vertex_id{vertex_id.AsInt()}, &memory)}; ASSERT_NE(vertex_to_delete, nullptr); - EXPECT_EQ(mgp_graph_delete_vertex(&immutable_graph, vertex_to_delete.get()), MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_graph_delete_vertex(&immutable_graph, vertex_to_delete.get()), mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); EXPECT_EQ(CountVertices(read_uncommited_accessor, memgraph::storage::View::NEW), 1); } @@ -398,10 +398,11 @@ TEST_F(MgpGraphTest, ModifyImmutableVertex) { EXPECT_MGP_NO_ERROR(mgp_vertex *, mgp_graph_get_vertex_by_id, &graph, mgp_vertex_id{vertex_id.AsInt()}, &memory)}; EXPECT_EQ(EXPECT_MGP_NO_ERROR(int, mgp_vertex_underlying_graph_is_mutable, vertex.get()), 0); - EXPECT_EQ(mgp_vertex_add_label(vertex.get(), mgp_label{"label"}), MGP_ERROR_IMMUTABLE_OBJECT); - EXPECT_EQ(mgp_vertex_remove_label(vertex.get(), mgp_label{label_to_remove.data()}), MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_vertex_add_label(vertex.get(), mgp_label{"label"}), mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_vertex_remove_label(vertex.get(), mgp_label{label_to_remove.data()}), + mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); MgpValuePtr value{EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_int, 4, &memory)}; - EXPECT_EQ(mgp_vertex_set_property(vertex.get(), "property", value.get()), MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_vertex_set_property(vertex.get(), "property", value.get()), mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); } TEST_F(MgpGraphTest, CreateDeleteEdge) { @@ -452,16 +453,16 @@ TEST_F(MgpGraphTest, CreateDeleteEdgeWithImmutableGraph) { mgp_edge *edge{nullptr}; EXPECT_EQ( mgp_graph_create_edge(&graph, from.get(), to.get(), mgp_edge_type{"NEWLY_CREATED_EDGE_TYPE"}, &memory, &edge), - MGP_ERROR_IMMUTABLE_OBJECT); + mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); CheckEdgeCountBetween(from, to, 1); MgpEdgesIteratorPtr edges_it{ EXPECT_MGP_NO_ERROR(mgp_edges_iterator *, mgp_vertex_iter_out_edges, from.get(), &memory)}; auto *edge_from_it = EXPECT_MGP_NO_ERROR(mgp_edge *, mgp_edges_iterator_get, edges_it.get()); ASSERT_NE(edge_from_it, nullptr); - EXPECT_EQ(mgp_graph_delete_edge(&graph, edge_from_it), MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_graph_delete_edge(&graph, edge_from_it), mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); MgpEdgePtr edge_copy_of_immutable{EXPECT_MGP_NO_ERROR(mgp_edge *, mgp_edge_copy, edge_from_it, &memory)}; - EXPECT_EQ(mgp_graph_delete_edge(&graph, edge_copy_of_immutable.get()), MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_graph_delete_edge(&graph, edge_copy_of_immutable.get()), mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); CheckEdgeCountBetween(from, to, 1); } @@ -616,5 +617,5 @@ TEST_F(MgpGraphTest, EdgeSetPropertyWithImmutableGraph) { ASSERT_NO_FATAL_FAILURE(GetFirstOutEdge(graph, from_vertex_id, edge)); MgpValuePtr value{EXPECT_MGP_NO_ERROR(mgp_value *, mgp_value_make_int, 65, &memory)}; EXPECT_EQ(EXPECT_MGP_NO_ERROR(int, mgp_edge_underlying_graph_is_mutable, edge.get()), 0); - EXPECT_EQ(mgp_edge_set_property(edge.get(), "property", value.get()), MGP_ERROR_IMMUTABLE_OBJECT); + EXPECT_EQ(mgp_edge_set_property(edge.get(), "property", value.get()), mgp_error::MGP_ERROR_IMMUTABLE_OBJECT); } diff --git a/tests/unit/test_utils.hpp b/tests/unit/test_utils.hpp index f6a8ea232..b72c3b077 100644 --- a/tests/unit/test_utils.hpp +++ b/tests/unit/test_utils.hpp @@ -29,7 +29,7 @@ TResult ExpectNoError(const char *file, int line, TFunc func, TArgs &&...args) { static_assert(std::is_trivially_copyable_v); static_assert((std::is_trivially_copyable_v> && ...)); TResult result{}; - EXPECT_EQ(func(args..., &result), MGP_ERROR_NO_ERROR) << fmt::format("Source of error: {}:{}", file, line); + EXPECT_EQ(func(args..., &result), mgp_error::MGP_ERROR_NO_ERROR) << fmt::format("Source of error: {}:{}", file, line); return result; } } // namespace test_utils