From b244c4d6ee7f75e291b96940ff8ee3a71f4bb784 Mon Sep 17 00:00:00 2001 From: jeremy Date: Mon, 28 Nov 2022 17:32:29 +0100 Subject: [PATCH 01/51] Impl of Multiframe and iterators --- src/expr/interpret/frame.hpp | 15 +++ src/query/v2/CMakeLists.txt | 3 +- src/query/v2/bindings/frame.hpp | 5 +- src/query/v2/interpreter.cpp | 115 ++++++++++++++++- src/query/v2/multiframe.cpp | 128 +++++++++++++++++++ src/query/v2/multiframe.hpp | 218 ++++++++++++++++++++++++++++++++ src/query/v2/plan/operator.cpp | 40 +++++- src/query/v2/plan/operator.lcp | 7 +- 8 files changed, 523 insertions(+), 8 deletions(-) create mode 100644 src/query/v2/multiframe.cpp create mode 100644 src/query/v2/multiframe.hpp diff --git a/src/expr/interpret/frame.hpp b/src/expr/interpret/frame.hpp index c0619e50e..6a220b6a0 100644 --- a/src/expr/interpret/frame.hpp +++ b/src/expr/interpret/frame.hpp @@ -42,4 +42,19 @@ class Frame { utils::pmr::vector elems_; }; +template +class FrameWithValidity final : public Frame { + public: + explicit FrameWithValidity(int64_t size) : Frame(size), is_valid_(false) {} + + FrameWithValidity(int64_t size, utils::MemoryResource *memory) : Frame(size, memory), is_valid_(false) {} + + bool IsValid() const { return is_valid_; } + void MakeValid() { is_valid_ = true; } + void MakeInvalid() { is_valid_ = false; } + + private: + bool is_valid_; +}; + } // namespace memgraph::expr diff --git a/src/query/v2/CMakeLists.txt b/src/query/v2/CMakeLists.txt index 3c3f780c8..0c6fa5f46 100644 --- a/src/query/v2/CMakeLists.txt +++ b/src/query/v2/CMakeLists.txt @@ -24,7 +24,8 @@ set(mg_query_v2_sources plan/variable_start_planner.cpp serialization/property_value.cpp bindings/typed_value.cpp - accessors.cpp) + accessors.cpp + multiframe.cpp) find_package(Boost REQUIRED) diff --git a/src/query/v2/bindings/frame.hpp b/src/query/v2/bindings/frame.hpp index f5c425f23..688f14121 100644 --- a/src/query/v2/bindings/frame.hpp +++ b/src/query/v2/bindings/frame.hpp @@ -13,9 +13,10 @@ #include "query/v2/bindings/bindings.hpp" -#include "query/v2/bindings/typed_value.hpp" #include "expr/interpret/frame.hpp" +#include "query/v2/bindings/typed_value.hpp" namespace memgraph::query::v2 { using Frame = memgraph::expr::Frame; -} // namespace memgraph::query::v2 +using FrameWithValidity = memgraph::expr::FrameWithValidity; +} // namespace memgraph::query::v2 diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index 045d94709..99eedccb9 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -41,6 +41,7 @@ #include "query/v2/frontend/ast/ast.hpp" #include "query/v2/frontend/semantic/required_privileges.hpp" #include "query/v2/metadata.hpp" +#include "query/v2/multiframe.hpp" #include "query/v2/plan/planner.hpp" #include "query/v2/plan/profile.hpp" #include "query/v2/plan/vertex_count_cache.hpp" @@ -655,11 +656,15 @@ struct PullPlan { std::optional Pull(AnyStream *stream, std::optional n, const std::vector &output_symbols, std::map *summary); + std::optional PullMultiple(AnyStream *stream, std::optional n, + const std::vector &output_symbols, + std::map *summary); private: std::shared_ptr plan_ = nullptr; plan::UniqueCursorPtr cursor_ = nullptr; - expr::Frame frame_; + expr::FrameWithValidity frame_; + MultiFrame multi_frame_; ExecutionContext ctx_; std::optional memory_limit_; @@ -683,6 +688,7 @@ PullPlan::PullPlan(const std::shared_ptr plan, const Parameters &par : plan_(plan), cursor_(plan->plan().MakeCursor(execution_memory)), frame_(plan->symbol_table().max_position(), execution_memory), + multi_frame_(frame_, kNumberOfFramesInMultiframe, execution_memory), memory_limit_(memory_limit) { ctx_.db_accessor = dba; ctx_.symbol_table = plan->symbol_table(); @@ -699,9 +705,116 @@ PullPlan::PullPlan(const std::shared_ptr plan, const Parameters &par ctx_.edge_ids_alloc = &interpreter_context->edge_ids_alloc; } +std::optional PullPlan::PullMultiple(AnyStream *stream, std::optional n, + const std::vector &output_symbols, + std::map *summary) { + // Set up temporary memory for a single Pull. Initial memory comes from the + // stack. 256 KiB should fit on the stack and should be more than enough for a + // single `Pull`. + MG_ASSERT(!n.has_value(), "should pull all!"); + static constexpr size_t stack_size = 256UL * 1024UL; + char stack_data[stack_size]; + utils::ResourceWithOutOfMemoryException resource_with_exception; + utils::MonotonicBufferResource monotonic_memory(&stack_data[0], stack_size, &resource_with_exception); + // We can throw on every query because a simple queries for deleting will use only + // the stack allocated buffer. + // Also, we want to throw only when the query engine requests more memory and not the storage + // so we add the exception to the allocator. + // TODO (mferencevic): Tune the parameters accordingly. + utils::PoolResource pool_memory(128, 1024, &monotonic_memory); + std::optional maybe_limited_resource; + + if (memory_limit_) { + maybe_limited_resource.emplace(&pool_memory, *memory_limit_); + ctx_.evaluation_context.memory = &*maybe_limited_resource; + } else { + ctx_.evaluation_context.memory = &pool_memory; + } + + // Returns true if a result was pulled. + const auto pull_result = [&]() -> bool { + cursor_->PullMultiple(multi_frame_, ctx_); + return multi_frame_.HasValidFrame(); + }; + + const auto stream_values = [&output_symbols, &stream](Frame &frame) { + // TODO: The streamed values should also probably use the above memory. + std::vector values; + values.reserve(output_symbols.size()); + + for (const auto &symbol : output_symbols) { + values.emplace_back(frame[symbol]); + } + + stream->Result(values); + }; + + // Get the execution time of all possible result pulls and streams. + utils::Timer timer; + + int i = 0; + if (has_unsent_results_ && !output_symbols.empty()) { + // stream unsent results from previous pull + + auto iterator_for_valid_frame_only = multi_frame_.GetItOnConstValidFrames(); + for (auto &frame : iterator_for_valid_frame_only) { + stream_values(frame); + ++i; + } + multi_frame_.ResetAllFramesInvalid(); + } + + for (; !n || i < n;) { + if (!pull_result()) { + break; + } + + if (!output_symbols.empty()) { + auto iterator_for_valid_frame_only = multi_frame_.GetItOnConstValidFrames(); + for (auto &frame : iterator_for_valid_frame_only) { + stream_values(frame); + ++i; + } + } + multi_frame_.ResetAllFramesInvalid(); + } + + // If we finished because we streamed the requested n results, + // we try to pull the next result to see if there is more. + // If there is additional result, we leave the pulled result in the frame + // and set the flag to true. + has_unsent_results_ = i == n && pull_result(); + + execution_time_ += timer.Elapsed(); + + if (has_unsent_results_) { + return std::nullopt; + } + summary->insert_or_assign("plan_execution_time", execution_time_.count()); + // We are finished with pulling all the data, therefore we can send any + // metadata about the results i.e. notifications and statistics + const bool is_any_counter_set = + std::any_of(ctx_.execution_stats.counters.begin(), ctx_.execution_stats.counters.end(), + [](const auto &counter) { return counter > 0; }); + if (is_any_counter_set) { + std::map stats; + for (size_t i = 0; i < ctx_.execution_stats.counters.size(); ++i) { + stats.emplace(ExecutionStatsKeyToString(ExecutionStats::Key(i)), ctx_.execution_stats.counters[i]); + } + summary->insert_or_assign("stats", std::move(stats)); + } + cursor_->Shutdown(); + ctx_.profile_execution_time = execution_time_; + return GetStatsWithTotalTime(ctx_); +} + std::optional PullPlan::Pull(AnyStream *stream, std::optional n, const std::vector &output_symbols, std::map *summary) { + auto should_pull_multiple = false; // #NoCommit + if (should_pull_multiple) { + return PullMultiple(stream, n, output_symbols, summary); + } // Set up temporary memory for a single Pull. Initial memory comes from the // stack. 256 KiB should fit on the stack and should be more than enough for a // single `Pull`. diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp new file mode 100644 index 000000000..e6e3d3a98 --- /dev/null +++ b/src/query/v2/multiframe.cpp @@ -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. + +#include "query/v2/multiframe.hpp" + +#include "query/v2/bindings/frame.hpp" +#include "utils/pmr/vector.hpp" + +namespace memgraph::query::v2 { + +MultiFrame::MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, + utils::MemoryResource *execution_memory) + : default_frame_(default_frame), + frames_(utils::pmr::vector(number_of_frames, default_frame, execution_memory)) { + MG_ASSERT(number_of_frames > 0); + MG_ASSERT(!default_frame.IsValid()); +} + +MultiFrame::~MultiFrame() = default; + +MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_frame_) { + /* + #NoCommit maybe not needed + Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any + difference? + */ + frames_.reserve(other.frames_.size()); + std::transform(other.frames_.begin(), other.frames_.end(), std::back_inserter(frames_), + [&default_frame = default_frame_](const auto &other_frame) { + if (other_frame.IsValid()) { + return other_frame; + } else { + return default_frame; + } + }); +} + +MultiFrame::MultiFrame(MultiFrame &&other) noexcept : default_frame_(std::move(other.default_frame_)) { + /* + #NoCommit maybe not needed + Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any + difference? + */ + frames_.reserve(other.frames_.size()); + std::transform(make_move_iterator(other.frames_.begin()), make_move_iterator(other.frames_.end()), + std::back_inserter(frames_), [&default_frame = default_frame_](const auto &other_frame) { + if (other_frame.IsValid()) { + return other_frame; + } else { + return default_frame; + } + }); +} + +void MultiFrame::ResetAllFramesInvalid() noexcept { + std::for_each(frames_.begin(), frames_.end(), [](auto &frame) { frame.MakeInvalid(); }); +} + +bool MultiFrame::HasValidFrame() const noexcept { + return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return frame.IsValid(); }); +} + +void MultiFrame::DefragmentValidFrames() noexcept { + /* + from: https://en.cppreference.com/w/cpp/algorithm/remove + "Removing is done by shifting (by means of copy assignment (until C++11)move assignment (since C++11)) the elements + in the range in such a way that the elements that are not to be removed appear in the beginning of the range. + Relative order of the elements that remain is preserved and the physical size of the container is unchanged." + */ + std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); }); +} + +ItOnConstValidFrames MultiFrame::GetItOnConstValidFrames() { return ItOnConstValidFrames(*this); } + +ItOnNonConstValidFrames MultiFrame::GetItOnNonConstValidFrames() { return ItOnNonConstValidFrames(*this); } + +ItOnNonConstInvalidFrames MultiFrame::GetItOnNonConstInvalidFrames() { return ItOnNonConstInvalidFrames(*this); } + +ItOnConstValidFrames::ItOnConstValidFrames(MultiFrame &multiframe) : multiframe_(multiframe) {} + +ItOnConstValidFrames::~ItOnConstValidFrames() = default; + +ItOnConstValidFrames::Iterator ItOnConstValidFrames::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ItOnConstValidFrames::Iterator ItOnConstValidFrames::end() { + return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); +} + +ItOnNonConstValidFrames::ItOnNonConstValidFrames(MultiFrame &multiframe) : multiframe_(multiframe) {} + +ItOnNonConstValidFrames::~ItOnNonConstValidFrames() { + // #NoCommit possible optimisation: only DefragmentValidFrames if one frame has been invalidated? Only if does not + // cost too much to store it + multiframe_.DefragmentValidFrames(); +} + +ItOnNonConstValidFrames::Iterator ItOnNonConstValidFrames::begin() { return Iterator(&multiframe_.frames_[0], *this); } + +ItOnNonConstValidFrames::Iterator ItOnNonConstValidFrames::end() { + return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); +} + +ItOnNonConstInvalidFrames::ItOnNonConstInvalidFrames(MultiFrame &multiframe) : multiframe_(multiframe) {} + +ItOnNonConstInvalidFrames::~ItOnNonConstInvalidFrames() = default; + +ItOnNonConstInvalidFrames::Iterator ItOnNonConstInvalidFrames::begin() { + for (auto idx = 0UL; idx < multiframe_.frames_.size(); ++idx) { + if (!multiframe_.frames_[idx].IsValid()) { + return Iterator(&multiframe_.frames_[idx]); + } + } + + return end(); +} + +ItOnNonConstInvalidFrames::Iterator ItOnNonConstInvalidFrames::end() { + return Iterator(&multiframe_.frames_[multiframe_.frames_.size()]); +} + +} // namespace memgraph::query::v2 diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp new file mode 100644 index 000000000..5dd5d0e6b --- /dev/null +++ b/src/query/v2/multiframe.hpp @@ -0,0 +1,218 @@ +// 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 "query/v2/bindings/frame.hpp" + +namespace memgraph::query::v2 { +constexpr unsigned long kNumberOfFramesInMultiframe = 1000; // #NoCommit have it configurable + +class ItOnConstValidFrames; +class ItOnNonConstValidFrames; +class ItOnNonConstInvalidFrames; + +class MultiFrame { + public: + friend class ItOnConstValidFrames; + friend class ItOnNonConstValidFrames; + friend class ItOnNonConstInvalidFrames; + + MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); + ~MultiFrame(); + + MultiFrame(const MultiFrame &other); // copy constructor + MultiFrame(MultiFrame &&other) noexcept; // move constructor + MultiFrame &operator=(const MultiFrame &other) = delete; + MultiFrame &operator=(MultiFrame &&other) noexcept = delete; + + /*! + Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid + state in the multiframe. + Iteration goes in a deterministic order. + One can't modify the validity of the frame with this implementation. + */ + ItOnConstValidFrames GetItOnConstValidFrames(); + + /*! + Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid + state in the multiframe. + Iteration goes in a deterministic order. + One can modify the validity of the frame with this implementation. + If you do not plan to modify the validity of the frames, use GetItOnConstValidFrames instead as this is faster. + */ + ItOnNonConstValidFrames GetItOnNonConstValidFrames(); + + /*! + Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in an invalid + state in the multiframe. + Iteration goes in a deterministic order. + One can modify the validity of the frame with this implementation. + */ + ItOnNonConstInvalidFrames GetItOnNonConstInvalidFrames(); + + void ResetAllFramesInvalid() noexcept; + + bool HasValidFrame() const noexcept; + + inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); } + + private: + void DefragmentValidFrames() noexcept; + + FrameWithValidity default_frame_; + utils::pmr::vector frames_ = + utils::pmr::vector(0, FrameWithValidity{1}, utils::NewDeleteResource()); +}; + +class ItOnConstValidFrames { + public: + ItOnConstValidFrames(MultiFrame &multiframe); + + ~ItOnConstValidFrames(); + ItOnConstValidFrames(const ItOnConstValidFrames &other) = delete; // copy constructor + ItOnConstValidFrames(ItOnConstValidFrames &&other) noexcept = delete; // move constructor + ItOnConstValidFrames &operator=(const ItOnConstValidFrames &other) = delete; // copy assignment + ItOnConstValidFrames &operator=(ItOnConstValidFrames &&other) noexcept = delete; // move assignment + + struct Iterator { + using iterator_category = std::forward_iterator_tag; + using difference_type = std::ptrdiff_t; + using value_type = Frame; + using pointer = value_type *; + using reference = Frame &; + using internal_ptr = FrameWithValidity *; + + Iterator(internal_ptr ptr, ItOnConstValidFrames &iterator_wrapper) + : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + + reference operator*() const { return *ptr_; } + pointer operator->() { return ptr_; } + + // Prefix increment + Iterator &operator++() { + do { + ptr_++; + } while (!this->ptr_->IsValid() && *this != iterator_wrapper_.end()); + + return *this; + } + + friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; + friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + + private: + internal_ptr ptr_; + ItOnConstValidFrames &iterator_wrapper_; + }; + + Iterator begin(); + Iterator end(); + + private: + MultiFrame &multiframe_; +}; + +class ItOnNonConstValidFrames { + public: + ItOnNonConstValidFrames(MultiFrame &multiframe); + + ~ItOnNonConstValidFrames(); + ItOnNonConstValidFrames(const ItOnNonConstValidFrames &other) = delete; // copy constructor + ItOnNonConstValidFrames(ItOnNonConstValidFrames &&other) noexcept = delete; // move constructor + ItOnNonConstValidFrames &operator=(const ItOnNonConstValidFrames &other) = delete; // copy assignment + ItOnNonConstValidFrames &operator=(ItOnNonConstValidFrames &&other) noexcept = delete; // move assignment + + struct Iterator { + using iterator_category = std::forward_iterator_tag; + using difference_type = std::ptrdiff_t; + using value_type = FrameWithValidity; + using pointer = value_type *; + using reference = FrameWithValidity &; + using internal_ptr = FrameWithValidity *; + + Iterator(internal_ptr ptr, ItOnNonConstValidFrames &iterator_wrapper) + : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + + reference operator*() const { return *ptr_; } + pointer operator->() { return ptr_; } + + // Prefix increment + Iterator &operator++() { + do { + ptr_++; + } while (!this->ptr_->IsValid() && *this != iterator_wrapper_.end()); + + return *this; + } + + friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; + friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + + private: + internal_ptr ptr_; + ItOnNonConstValidFrames &iterator_wrapper_; + }; + + Iterator begin(); + Iterator end(); + + private: + MultiFrame &multiframe_; +}; + +class ItOnNonConstInvalidFrames { + public: + ItOnNonConstInvalidFrames(MultiFrame &multiframe); + ~ItOnNonConstInvalidFrames(); + + ItOnNonConstInvalidFrames(const ItOnNonConstInvalidFrames &other) = delete; // copy constructor + ItOnNonConstInvalidFrames(ItOnNonConstInvalidFrames &&other) noexcept = delete; // move constructor + ItOnNonConstInvalidFrames &operator=(const ItOnNonConstInvalidFrames &other) = delete; // copy assignment + ItOnNonConstInvalidFrames &operator=(ItOnNonConstInvalidFrames &&other) noexcept = delete; // move assignment + + struct Iterator { + using iterator_category = std::forward_iterator_tag; + using difference_type = std::ptrdiff_t; + using value_type = FrameWithValidity; + using pointer = value_type *; + using reference = FrameWithValidity &; + using internal_ptr = FrameWithValidity *; + + Iterator(internal_ptr ptr) : ptr_(ptr) {} + + reference operator*() const { return *ptr_; } + pointer operator->() { return ptr_; } + + // Prefix increment + Iterator &operator++() { + ptr_->MakeValid(); + ptr_++; + return *this; + } + + friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; + friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + + private: + internal_ptr ptr_; + }; + + Iterator begin(); + Iterator end(); + + private: + MultiFrame &multiframe_; +}; + +} // namespace memgraph::query::v2 diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index 068ca9192..f7044beac 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -257,13 +257,30 @@ class DistributedCreateNodeCursor : public Cursor { bool Once::OnceCursor::Pull(Frame &, ExecutionContext &context) { SCOPED_PROFILE_OP("Once"); - if (!did_pull_) { - did_pull_ = true; + if (pull_count_ < 1) { + pull_count_++; return true; } return false; } +void Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) { + SCOPED_PROFILE_OP("OnceMF"); + + auto iterator_for_valid_frame_only = multi_frame.GetItOnNonConstInvalidFrames(); + auto first_it = iterator_for_valid_frame_only.begin(); + MG_ASSERT(first_it != iterator_for_valid_frame_only.end()); + if (pull_count_ < 1) { + auto *memory_resource = multi_frame.GetMemoryResource(); + auto &frame = *first_it; + frame.MakeValid(); + for (auto &value : frame.elems()) { + value = TypedValue{memory_resource}; + } + pull_count_++; + } +} + UniqueCursorPtr Once::MakeCursor(utils::MemoryResource *mem) const { EventCounter::IncrementCounter(EventCounter::OnceOperator); @@ -274,7 +291,7 @@ WITHOUT_SINGLE_INPUT(Once); void Once::OnceCursor::Shutdown() {} -void Once::OnceCursor::Reset() { did_pull_ = false; } +void Once::OnceCursor::Reset() { pull_count_ = 0; } CreateNode::CreateNode(const std::shared_ptr &input, const NodeCreationInfo &node_info) : input_(input ? input : std::make_shared()), node_info_(node_info) {} @@ -766,6 +783,23 @@ bool Produce::ProduceCursor::Pull(Frame &frame, ExecutionContext &context) { return false; } +void Produce::ProduceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) { + SCOPED_PROFILE_OP("ProduceMF"); + + input_cursor_->PullMultiple(multi_frame, context); + + auto iterator_for_valid_frame_only = multi_frame.GetItOnConstValidFrames(); + for (auto &frame : iterator_for_valid_frame_only) { + // Produce should always yield the latest results. + ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, + context.shard_request_manager, storage::v3::View::NEW); + + for (auto *named_expr : self_.named_expressions_) { + named_expr->Accept(evaluator); + } + } +}; + void Produce::ProduceCursor::Shutdown() { input_cursor_->Shutdown(); } void Produce::ProduceCursor::Reset() { input_cursor_->Reset(); } diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp index d6277809c..eb83a9fb2 100644 --- a/src/query/v2/plan/operator.lcp +++ b/src/query/v2/plan/operator.lcp @@ -28,6 +28,7 @@ #include "query/v2/bindings/typed_value.hpp" #include "query/v2/bindings/frame.hpp" #include "query/v2/bindings/symbol_table.hpp" +#include "query/v2/multiframe.hpp" #include "storage/v3/id_types.hpp" #include "utils/bound.hpp" #include "utils/fnv.hpp" @@ -71,6 +72,8 @@ class Cursor { /// @throws QueryRuntimeException if something went wrong with execution virtual bool Pull(Frame &, ExecutionContext &) = 0; + virtual void PullMultiple(MultiFrame &, ExecutionContext &) { LOG_FATAL("PullMultipleIsNotImplemented"); } + /// Resets the Cursor to its initial state. virtual void Reset() = 0; @@ -332,12 +335,13 @@ and false on every following Pull.") class OnceCursor : public Cursor { public: OnceCursor() {} + void PullMultiple(MultiFrame &, ExecutionContext &) override; bool Pull(Frame &, ExecutionContext &) override; void Shutdown() override; void Reset() override; private: - bool did_pull_{false}; + size_t pull_count_{false}; }; cpp<#) (:serialize (:slk)) @@ -1207,6 +1211,7 @@ RETURN clause) the Produce's pull succeeds exactly once.") public: ProduceCursor(const Produce &, utils::MemoryResource *); bool Pull(Frame &, ExecutionContext &) override; + void PullMultiple(MultiFrame &, ExecutionContext &) override; void Shutdown() override; void Reset() override; From 11119e540644f4e7b4f41bf893ad39a9d093f403 Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 10:32:52 +0100 Subject: [PATCH 02/51] Rename ItOnConstValidFrames->ValidFramesReader --- src/query/v2/interpreter.cpp | 4 ++-- src/query/v2/multiframe.cpp | 10 +++++----- src/query/v2/multiframe.hpp | 27 +++++++++++++-------------- src/query/v2/plan/operator.cpp | 2 +- 4 files changed, 21 insertions(+), 22 deletions(-) diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index aa4d39de3..78a560662 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -756,7 +756,7 @@ std::optional PullPlan::PullMultiple(AnyStrea if (has_unsent_results_ && !output_symbols.empty()) { // stream unsent results from previous pull - auto iterator_for_valid_frame_only = multi_frame_.GetItOnConstValidFrames(); + auto iterator_for_valid_frame_only = multi_frame_.GetValidFramesReader(); for (auto &frame : iterator_for_valid_frame_only) { stream_values(frame); ++i; @@ -770,7 +770,7 @@ std::optional PullPlan::PullMultiple(AnyStrea } if (!output_symbols.empty()) { - auto iterator_for_valid_frame_only = multi_frame_.GetItOnConstValidFrames(); + auto iterator_for_valid_frame_only = multi_frame_.GetValidFramesReader(); for (auto &frame : iterator_for_valid_frame_only) { stream_values(frame); ++i; diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index e6e3d3a98..0109a9e39 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -78,18 +78,18 @@ void MultiFrame::DefragmentValidFrames() noexcept { std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); }); } -ItOnConstValidFrames MultiFrame::GetItOnConstValidFrames() { return ItOnConstValidFrames(*this); } +ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader(*this); } ItOnNonConstValidFrames MultiFrame::GetItOnNonConstValidFrames() { return ItOnNonConstValidFrames(*this); } ItOnNonConstInvalidFrames MultiFrame::GetItOnNonConstInvalidFrames() { return ItOnNonConstInvalidFrames(*this); } -ItOnConstValidFrames::ItOnConstValidFrames(MultiFrame &multiframe) : multiframe_(multiframe) {} +ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multiframe) {} -ItOnConstValidFrames::~ItOnConstValidFrames() = default; +ValidFramesReader::~ValidFramesReader() = default; -ItOnConstValidFrames::Iterator ItOnConstValidFrames::begin() { return Iterator(&multiframe_.frames_[0], *this); } -ItOnConstValidFrames::Iterator ItOnConstValidFrames::end() { +ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ValidFramesReader::Iterator ValidFramesReader::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 5dd5d0e6b..0f325f525 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -18,13 +18,13 @@ namespace memgraph::query::v2 { constexpr unsigned long kNumberOfFramesInMultiframe = 1000; // #NoCommit have it configurable -class ItOnConstValidFrames; +class ValidFramesReader; class ItOnNonConstValidFrames; class ItOnNonConstInvalidFrames; class MultiFrame { public: - friend class ItOnConstValidFrames; + friend class ValidFramesReader; friend class ItOnNonConstValidFrames; friend class ItOnNonConstInvalidFrames; @@ -42,14 +42,14 @@ class MultiFrame { Iteration goes in a deterministic order. One can't modify the validity of the frame with this implementation. */ - ItOnConstValidFrames GetItOnConstValidFrames(); + ValidFramesReader GetValidFramesReader(); /*! Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid state in the multiframe. Iteration goes in a deterministic order. One can modify the validity of the frame with this implementation. - If you do not plan to modify the validity of the frames, use GetItOnConstValidFrames instead as this is faster. + If you do not plan to modify the validity of the frames, use GetReader instead as this is faster. */ ItOnNonConstValidFrames GetItOnNonConstValidFrames(); @@ -75,15 +75,15 @@ class MultiFrame { utils::pmr::vector(0, FrameWithValidity{1}, utils::NewDeleteResource()); }; -class ItOnConstValidFrames { +class ValidFramesReader { public: - ItOnConstValidFrames(MultiFrame &multiframe); + ValidFramesReader(MultiFrame &multiframe); - ~ItOnConstValidFrames(); - ItOnConstValidFrames(const ItOnConstValidFrames &other) = delete; // copy constructor - ItOnConstValidFrames(ItOnConstValidFrames &&other) noexcept = delete; // move constructor - ItOnConstValidFrames &operator=(const ItOnConstValidFrames &other) = delete; // copy assignment - ItOnConstValidFrames &operator=(ItOnConstValidFrames &&other) noexcept = delete; // move assignment + ~ValidFramesReader(); + ValidFramesReader(const ValidFramesReader &other) = delete; // copy constructor + ValidFramesReader(ValidFramesReader &&other) noexcept = delete; // move constructor + ValidFramesReader &operator=(const ValidFramesReader &other) = delete; // copy assignment + ValidFramesReader &operator=(ValidFramesReader &&other) noexcept = delete; // move assignment struct Iterator { using iterator_category = std::forward_iterator_tag; @@ -93,8 +93,7 @@ class ItOnConstValidFrames { using reference = Frame &; using internal_ptr = FrameWithValidity *; - Iterator(internal_ptr ptr, ItOnConstValidFrames &iterator_wrapper) - : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + Iterator(internal_ptr ptr, ValidFramesReader &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } @@ -113,7 +112,7 @@ class ItOnConstValidFrames { private: internal_ptr ptr_; - ItOnConstValidFrames &iterator_wrapper_; + ValidFramesReader &iterator_wrapper_; }; Iterator begin(); diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index 81e466d82..bf85c13ad 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -768,7 +768,7 @@ void Produce::ProduceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionCont input_cursor_->PullMultiple(multi_frame, context); - auto iterator_for_valid_frame_only = multi_frame.GetItOnConstValidFrames(); + auto iterator_for_valid_frame_only = multi_frame.GetValidFramesReader(); for (auto &frame : iterator_for_valid_frame_only) { // Produce should always yield the latest results. ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, From 7c37ed23138c0d9c854a4f847044b563ccc5a4fe Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 10:38:01 +0100 Subject: [PATCH 03/51] Rename ItOnNonConstValidFrames -> ValidFramesInvalidator --- src/query/v2/multiframe.cpp | 10 +++++----- src/query/v2/multiframe.hpp | 26 +++++++++++++------------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 0109a9e39..9101a51b0 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -80,7 +80,7 @@ void MultiFrame::DefragmentValidFrames() noexcept { ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader(*this); } -ItOnNonConstValidFrames MultiFrame::GetItOnNonConstValidFrames() { return ItOnNonConstValidFrames(*this); } +ValidFramesInvalidator MultiFrame::GetValidFramesInvalidator() { return ValidFramesInvalidator(*this); } ItOnNonConstInvalidFrames MultiFrame::GetItOnNonConstInvalidFrames() { return ItOnNonConstInvalidFrames(*this); } @@ -93,17 +93,17 @@ ValidFramesReader::Iterator ValidFramesReader::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } -ItOnNonConstValidFrames::ItOnNonConstValidFrames(MultiFrame &multiframe) : multiframe_(multiframe) {} +ValidFramesInvalidator::ValidFramesInvalidator(MultiFrame &multiframe) : multiframe_(multiframe) {} -ItOnNonConstValidFrames::~ItOnNonConstValidFrames() { +ValidFramesInvalidator::~ValidFramesInvalidator() { // #NoCommit possible optimisation: only DefragmentValidFrames if one frame has been invalidated? Only if does not // cost too much to store it multiframe_.DefragmentValidFrames(); } -ItOnNonConstValidFrames::Iterator ItOnNonConstValidFrames::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ValidFramesInvalidator::Iterator ValidFramesInvalidator::begin() { return Iterator(&multiframe_.frames_[0], *this); } -ItOnNonConstValidFrames::Iterator ItOnNonConstValidFrames::end() { +ValidFramesInvalidator::Iterator ValidFramesInvalidator::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 0f325f525..ab373c626 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -19,13 +19,13 @@ namespace memgraph::query::v2 { constexpr unsigned long kNumberOfFramesInMultiframe = 1000; // #NoCommit have it configurable class ValidFramesReader; -class ItOnNonConstValidFrames; +class ValidFramesInvalidator; class ItOnNonConstInvalidFrames; class MultiFrame { public: friend class ValidFramesReader; - friend class ItOnNonConstValidFrames; + friend class ValidFramesInvalidator; friend class ItOnNonConstInvalidFrames; MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); @@ -49,9 +49,9 @@ class MultiFrame { state in the multiframe. Iteration goes in a deterministic order. One can modify the validity of the frame with this implementation. - If you do not plan to modify the validity of the frames, use GetReader instead as this is faster. + If you do not plan to modify the validity of the frames, use GetValidFramesReader instead as this is faster. */ - ItOnNonConstValidFrames GetItOnNonConstValidFrames(); + ValidFramesInvalidator GetValidFramesInvalidator(); /*! Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in an invalid @@ -122,15 +122,15 @@ class ValidFramesReader { MultiFrame &multiframe_; }; -class ItOnNonConstValidFrames { +class ValidFramesInvalidator { public: - ItOnNonConstValidFrames(MultiFrame &multiframe); + ValidFramesInvalidator(MultiFrame &multiframe); - ~ItOnNonConstValidFrames(); - ItOnNonConstValidFrames(const ItOnNonConstValidFrames &other) = delete; // copy constructor - ItOnNonConstValidFrames(ItOnNonConstValidFrames &&other) noexcept = delete; // move constructor - ItOnNonConstValidFrames &operator=(const ItOnNonConstValidFrames &other) = delete; // copy assignment - ItOnNonConstValidFrames &operator=(ItOnNonConstValidFrames &&other) noexcept = delete; // move assignment + ~ValidFramesInvalidator(); + ValidFramesInvalidator(const ValidFramesInvalidator &other) = delete; // copy constructor + ValidFramesInvalidator(ValidFramesInvalidator &&other) noexcept = delete; // move constructor + ValidFramesInvalidator &operator=(const ValidFramesInvalidator &other) = delete; // copy assignment + ValidFramesInvalidator &operator=(ValidFramesInvalidator &&other) noexcept = delete; // move assignment struct Iterator { using iterator_category = std::forward_iterator_tag; @@ -140,7 +140,7 @@ class ItOnNonConstValidFrames { using reference = FrameWithValidity &; using internal_ptr = FrameWithValidity *; - Iterator(internal_ptr ptr, ItOnNonConstValidFrames &iterator_wrapper) + Iterator(internal_ptr ptr, ValidFramesInvalidator &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} reference operator*() const { return *ptr_; } @@ -160,7 +160,7 @@ class ItOnNonConstValidFrames { private: internal_ptr ptr_; - ItOnNonConstValidFrames &iterator_wrapper_; + ValidFramesInvalidator &iterator_wrapper_; }; Iterator begin(); From 8f19ce88d95dba27c293ef4eef06d3d6364566d4 Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 10:40:52 +0100 Subject: [PATCH 04/51] Rename ItOnNonConstInvalidFrames->InvalidFramesPopulator --- src/query/v2/multiframe.cpp | 10 +++++----- src/query/v2/multiframe.hpp | 20 ++++++++++---------- src/query/v2/plan/operator.cpp | 2 +- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 9101a51b0..d07a4deaf 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -82,7 +82,7 @@ ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader( ValidFramesInvalidator MultiFrame::GetValidFramesInvalidator() { return ValidFramesInvalidator(*this); } -ItOnNonConstInvalidFrames MultiFrame::GetItOnNonConstInvalidFrames() { return ItOnNonConstInvalidFrames(*this); } +InvalidFramesPopulator MultiFrame::GetInvalidFramesPopulator() { return InvalidFramesPopulator(*this); } ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multiframe) {} @@ -107,11 +107,11 @@ ValidFramesInvalidator::Iterator ValidFramesInvalidator::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } -ItOnNonConstInvalidFrames::ItOnNonConstInvalidFrames(MultiFrame &multiframe) : multiframe_(multiframe) {} +InvalidFramesPopulator::InvalidFramesPopulator(MultiFrame &multiframe) : multiframe_(multiframe) {} -ItOnNonConstInvalidFrames::~ItOnNonConstInvalidFrames() = default; +InvalidFramesPopulator::~InvalidFramesPopulator() = default; -ItOnNonConstInvalidFrames::Iterator ItOnNonConstInvalidFrames::begin() { +InvalidFramesPopulator::Iterator InvalidFramesPopulator::begin() { for (auto idx = 0UL; idx < multiframe_.frames_.size(); ++idx) { if (!multiframe_.frames_[idx].IsValid()) { return Iterator(&multiframe_.frames_[idx]); @@ -121,7 +121,7 @@ ItOnNonConstInvalidFrames::Iterator ItOnNonConstInvalidFrames::begin() { return end(); } -ItOnNonConstInvalidFrames::Iterator ItOnNonConstInvalidFrames::end() { +InvalidFramesPopulator::Iterator InvalidFramesPopulator::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()]); } diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index ab373c626..23d14a9c3 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -20,13 +20,13 @@ constexpr unsigned long kNumberOfFramesInMultiframe = 1000; // #NoCommit have i class ValidFramesReader; class ValidFramesInvalidator; -class ItOnNonConstInvalidFrames; +class InvalidFramesPopulator; class MultiFrame { public: friend class ValidFramesReader; friend class ValidFramesInvalidator; - friend class ItOnNonConstInvalidFrames; + friend class InvalidFramesPopulator; MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); ~MultiFrame(); @@ -59,7 +59,7 @@ class MultiFrame { Iteration goes in a deterministic order. One can modify the validity of the frame with this implementation. */ - ItOnNonConstInvalidFrames GetItOnNonConstInvalidFrames(); + InvalidFramesPopulator GetInvalidFramesPopulator(); void ResetAllFramesInvalid() noexcept; @@ -170,15 +170,15 @@ class ValidFramesInvalidator { MultiFrame &multiframe_; }; -class ItOnNonConstInvalidFrames { +class InvalidFramesPopulator { public: - ItOnNonConstInvalidFrames(MultiFrame &multiframe); - ~ItOnNonConstInvalidFrames(); + InvalidFramesPopulator(MultiFrame &multiframe); + ~InvalidFramesPopulator(); - ItOnNonConstInvalidFrames(const ItOnNonConstInvalidFrames &other) = delete; // copy constructor - ItOnNonConstInvalidFrames(ItOnNonConstInvalidFrames &&other) noexcept = delete; // move constructor - ItOnNonConstInvalidFrames &operator=(const ItOnNonConstInvalidFrames &other) = delete; // copy assignment - ItOnNonConstInvalidFrames &operator=(ItOnNonConstInvalidFrames &&other) noexcept = delete; // move assignment + InvalidFramesPopulator(const InvalidFramesPopulator &other) = delete; // copy constructor + InvalidFramesPopulator(InvalidFramesPopulator &&other) noexcept = delete; // move constructor + InvalidFramesPopulator &operator=(const InvalidFramesPopulator &other) = delete; // copy assignment + InvalidFramesPopulator &operator=(InvalidFramesPopulator &&other) noexcept = delete; // move assignment struct Iterator { using iterator_category = std::forward_iterator_tag; diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index bf85c13ad..6f4747e2b 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -268,7 +268,7 @@ bool Once::OnceCursor::Pull(Frame &, ExecutionContext &context) { void Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) { SCOPED_PROFILE_OP("OnceMF"); - auto iterator_for_valid_frame_only = multi_frame.GetItOnNonConstInvalidFrames(); + auto iterator_for_valid_frame_only = multi_frame.GetValidFramesInvalidator(); auto first_it = iterator_for_valid_frame_only.begin(); MG_ASSERT(first_it != iterator_for_valid_frame_only.end()); if (pull_count_ < 1) { From e946eb50d2115777927e4aeeb28e77cf3dfa8310 Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 11:05:11 +0100 Subject: [PATCH 05/51] Add version ValidFramesModifier to distinguish between reading-only and reading+modifying --- src/query/v2/interpreter.cpp | 2 +- src/query/v2/multiframe.cpp | 11 ++++++ src/query/v2/multiframe.hpp | 71 +++++++++++++++++++++++++++++++--- src/query/v2/plan/operator.cpp | 2 +- 4 files changed, 78 insertions(+), 8 deletions(-) diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index 78a560662..86a967de4 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -737,7 +737,7 @@ std::optional PullPlan::PullMultiple(AnyStrea return multi_frame_.HasValidFrame(); }; - const auto stream_values = [&output_symbols, &stream](Frame &frame) { + const auto stream_values = [&output_symbols, &stream](const Frame &frame) { // TODO: The streamed values should also probably use the above memory. std::vector values; values.reserve(output_symbols.size()); diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index d07a4deaf..2d0918b9c 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -80,6 +80,8 @@ void MultiFrame::DefragmentValidFrames() noexcept { ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader(*this); } +ValidFramesModifier MultiFrame::GetValidFramesModifier() { return ValidFramesModifier(*this); } + ValidFramesInvalidator MultiFrame::GetValidFramesInvalidator() { return ValidFramesInvalidator(*this); } InvalidFramesPopulator MultiFrame::GetInvalidFramesPopulator() { return InvalidFramesPopulator(*this); } @@ -93,6 +95,15 @@ ValidFramesReader::Iterator ValidFramesReader::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } +ValidFramesModifier::ValidFramesModifier(MultiFrame &multiframe) : multiframe_(multiframe) {} + +ValidFramesModifier::~ValidFramesModifier() = default; + +ValidFramesModifier::Iterator ValidFramesModifier::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ValidFramesModifier::Iterator ValidFramesModifier::end() { + return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); +} + ValidFramesInvalidator::ValidFramesInvalidator(MultiFrame &multiframe) : multiframe_(multiframe) {} ValidFramesInvalidator::~ValidFramesInvalidator() { diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 23d14a9c3..5b4af87e3 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -18,14 +18,16 @@ namespace memgraph::query::v2 { constexpr unsigned long kNumberOfFramesInMultiframe = 1000; // #NoCommit have it configurable -class ValidFramesReader; class ValidFramesInvalidator; +class ValidFramesModifier; +class ValidFramesReader; class InvalidFramesPopulator; class MultiFrame { public: - friend class ValidFramesReader; friend class ValidFramesInvalidator; + friend class ValidFramesModifier; + friend class ValidFramesReader; friend class InvalidFramesPopulator; MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); @@ -40,16 +42,25 @@ class MultiFrame { Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid state in the multiframe. Iteration goes in a deterministic order. - One can't modify the validity of the frame with this implementation. + One can't modify the validity of the frame nor its content with this implementation. */ ValidFramesReader GetValidFramesReader(); + /*! + Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid + state in the multiframe. + Iteration goes in a deterministic order. + One can't modify the validity of the frame with this implementation. One can modify its content. + */ + ValidFramesModifier GetValidFramesModifier(); + /*! Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid state in the multiframe. Iteration goes in a deterministic order. One can modify the validity of the frame with this implementation. - If you do not plan to modify the validity of the frames, use GetValidFramesReader instead as this is faster. + If you do not plan to modify the validity of the frames, use GetValidFramesReader/GetValidFramesModifer instead as + this is faster. */ ValidFramesInvalidator GetValidFramesInvalidator(); @@ -88,9 +99,9 @@ class ValidFramesReader { struct Iterator { using iterator_category = std::forward_iterator_tag; using difference_type = std::ptrdiff_t; - using value_type = Frame; + using value_type = const Frame; using pointer = value_type *; - using reference = Frame &; + using reference = const Frame &; using internal_ptr = FrameWithValidity *; Iterator(internal_ptr ptr, ValidFramesReader &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} @@ -122,6 +133,54 @@ class ValidFramesReader { MultiFrame &multiframe_; }; +class ValidFramesModifier { + public: + ValidFramesModifier(MultiFrame &multiframe); + + ~ValidFramesModifier(); + ValidFramesModifier(const ValidFramesModifier &other) = delete; // copy constructor + ValidFramesModifier(ValidFramesModifier &&other) noexcept = delete; // move constructor + ValidFramesModifier &operator=(const ValidFramesModifier &other) = delete; // copy assignment + ValidFramesModifier &operator=(ValidFramesModifier &&other) noexcept = delete; // move assignment + + struct Iterator { + using iterator_category = std::forward_iterator_tag; + using difference_type = std::ptrdiff_t; + using value_type = Frame; + using pointer = value_type *; + using reference = Frame &; + using internal_ptr = FrameWithValidity *; + + Iterator(internal_ptr ptr, ValidFramesModifier &iterator_wrapper) + : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + + reference operator*() const { return *ptr_; } + pointer operator->() { return ptr_; } + + // Prefix increment + Iterator &operator++() { + do { + ptr_++; + } while (!this->ptr_->IsValid() && *this != iterator_wrapper_.end()); + + return *this; + } + + friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; + friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + + private: + internal_ptr ptr_; + ValidFramesModifier &iterator_wrapper_; + }; + + Iterator begin(); + Iterator end(); + + private: + MultiFrame &multiframe_; +}; + class ValidFramesInvalidator { public: ValidFramesInvalidator(MultiFrame &multiframe); diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index 6f4747e2b..c71d70701 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -768,7 +768,7 @@ void Produce::ProduceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionCont input_cursor_->PullMultiple(multi_frame, context); - auto iterator_for_valid_frame_only = multi_frame.GetValidFramesReader(); + auto iterator_for_valid_frame_only = multi_frame.GetValidFramesModifier(); for (auto &frame : iterator_for_valid_frame_only) { // Produce should always yield the latest results. ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, From cf388d80fc6ffbc8adf5f7d5c7c0f2bcb6ffc68d Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Tue, 29 Nov 2022 12:29:52 +0100 Subject: [PATCH 06/51] Update src/query/v2/plan/operator.lcp --- src/query/v2/plan/operator.lcp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp index eb83a9fb2..9f8b7d6db 100644 --- a/src/query/v2/plan/operator.lcp +++ b/src/query/v2/plan/operator.lcp @@ -341,7 +341,7 @@ and false on every following Pull.") void Reset() override; private: - size_t pull_count_{false}; + size_t pull_count_{0}; }; cpp<#) (:serialize (:slk)) From cead1bcb215f0e1537b928dad7325b8d05c31120 Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 13:49:37 +0100 Subject: [PATCH 07/51] Rename ValidFramesInvalidator->ValidFramesConsumer --- src/query/v2/multiframe.cpp | 10 +++++----- src/query/v2/multiframe.hpp | 24 ++++++++++++------------ src/query/v2/plan/operator.cpp | 6 +++--- 3 files changed, 20 insertions(+), 20 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 2d0918b9c..65829fe5e 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -82,7 +82,7 @@ ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader( ValidFramesModifier MultiFrame::GetValidFramesModifier() { return ValidFramesModifier(*this); } -ValidFramesInvalidator MultiFrame::GetValidFramesInvalidator() { return ValidFramesInvalidator(*this); } +ValidFramesConsumer MultiFrame::GetValidFramesConsumer() { return ValidFramesConsumer(*this); } InvalidFramesPopulator MultiFrame::GetInvalidFramesPopulator() { return InvalidFramesPopulator(*this); } @@ -104,17 +104,17 @@ ValidFramesModifier::Iterator ValidFramesModifier::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } -ValidFramesInvalidator::ValidFramesInvalidator(MultiFrame &multiframe) : multiframe_(multiframe) {} +ValidFramesConsumer::ValidFramesConsumer(MultiFrame &multiframe) : multiframe_(multiframe) {} -ValidFramesInvalidator::~ValidFramesInvalidator() { +ValidFramesConsumer::~ValidFramesConsumer() { // #NoCommit possible optimisation: only DefragmentValidFrames if one frame has been invalidated? Only if does not // cost too much to store it multiframe_.DefragmentValidFrames(); } -ValidFramesInvalidator::Iterator ValidFramesInvalidator::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ValidFramesConsumer::Iterator ValidFramesConsumer::begin() { return Iterator(&multiframe_.frames_[0], *this); } -ValidFramesInvalidator::Iterator ValidFramesInvalidator::end() { +ValidFramesConsumer::Iterator ValidFramesConsumer::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 5b4af87e3..aee4052a6 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -18,14 +18,14 @@ namespace memgraph::query::v2 { constexpr unsigned long kNumberOfFramesInMultiframe = 1000; // #NoCommit have it configurable -class ValidFramesInvalidator; +class ValidFramesConsumer; class ValidFramesModifier; class ValidFramesReader; class InvalidFramesPopulator; class MultiFrame { public: - friend class ValidFramesInvalidator; + friend class ValidFramesConsumer; friend class ValidFramesModifier; friend class ValidFramesReader; friend class InvalidFramesPopulator; @@ -62,7 +62,7 @@ class MultiFrame { If you do not plan to modify the validity of the frames, use GetValidFramesReader/GetValidFramesModifer instead as this is faster. */ - ValidFramesInvalidator GetValidFramesInvalidator(); + ValidFramesConsumer GetValidFramesConsumer(); /*! Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in an invalid @@ -181,15 +181,15 @@ class ValidFramesModifier { MultiFrame &multiframe_; }; -class ValidFramesInvalidator { +class ValidFramesConsumer { public: - ValidFramesInvalidator(MultiFrame &multiframe); + ValidFramesConsumer(MultiFrame &multiframe); - ~ValidFramesInvalidator(); - ValidFramesInvalidator(const ValidFramesInvalidator &other) = delete; // copy constructor - ValidFramesInvalidator(ValidFramesInvalidator &&other) noexcept = delete; // move constructor - ValidFramesInvalidator &operator=(const ValidFramesInvalidator &other) = delete; // copy assignment - ValidFramesInvalidator &operator=(ValidFramesInvalidator &&other) noexcept = delete; // move assignment + ~ValidFramesConsumer(); + ValidFramesConsumer(const ValidFramesConsumer &other) = delete; // copy constructor + ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete; // move constructor + ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete; // copy assignment + ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete; // move assignment struct Iterator { using iterator_category = std::forward_iterator_tag; @@ -199,7 +199,7 @@ class ValidFramesInvalidator { using reference = FrameWithValidity &; using internal_ptr = FrameWithValidity *; - Iterator(internal_ptr ptr, ValidFramesInvalidator &iterator_wrapper) + Iterator(internal_ptr ptr, ValidFramesConsumer &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} reference operator*() const { return *ptr_; } @@ -219,7 +219,7 @@ class ValidFramesInvalidator { private: internal_ptr ptr_; - ValidFramesInvalidator &iterator_wrapper_; + ValidFramesConsumer &iterator_wrapper_; }; Iterator begin(); diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index 50cc57ec6..0f1415694 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -268,7 +268,7 @@ bool Once::OnceCursor::Pull(Frame &, ExecutionContext &context) { void Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) { SCOPED_PROFILE_OP("OnceMF"); - auto iterator_for_valid_frame_only = multi_frame.GetValidFramesInvalidator(); + auto iterator_for_valid_frame_only = multi_frame.GetValidFramesConsumer(); auto first_it = iterator_for_valid_frame_only.begin(); MG_ASSERT(first_it != iterator_for_valid_frame_only.end()); if (pull_count_ < 1) { @@ -772,8 +772,8 @@ void Produce::ProduceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionCont auto iterator_for_valid_frame_only = multi_frame.GetValidFramesModifier(); for (auto &frame : iterator_for_valid_frame_only) { // Produce should always yield the latest results. - ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, - context.shard_request_manager, storage::v3::View::NEW); + ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router, + storage::v3::View::NEW); for (auto *named_expr : self_.named_expressions_) { named_expr->Accept(evaluator); From bc32a3d305ee772ce3e45d98c1318aba4cc53a0a Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 13:52:29 +0100 Subject: [PATCH 08/51] Adapt comment --- src/query/v2/interpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index fa80ee487..891699312 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -811,7 +811,7 @@ std::optional PullPlan::PullMultiple(AnyStrea std::optional PullPlan::Pull(AnyStream *stream, std::optional n, const std::vector &output_symbols, std::map *summary) { - auto should_pull_multiple = false; // #NoCommit + auto should_pull_multiple = false; // TODO on the long term, we will only use PullMultiple if (should_pull_multiple) { return PullMultiple(stream, n, output_symbols, summary); } From aace5db8ccb48a2901da134bb432b1c5048bd50a Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 13:53:08 +0100 Subject: [PATCH 09/51] Adapt comment --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index aee4052a6..b004bd96f 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -16,7 +16,7 @@ #include "query/v2/bindings/frame.hpp" namespace memgraph::query::v2 { -constexpr unsigned long kNumberOfFramesInMultiframe = 1000; // #NoCommit have it configurable +constexpr uint64 kNumberOfFramesInMultiframe = 1000; // TODO have it configurable class ValidFramesConsumer; class ValidFramesModifier; From 3c0e38aacb8ea10eda1a8d64578868da10170aff Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 13:54:26 +0100 Subject: [PATCH 10/51] Adapt comment --- src/query/v2/multiframe.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 65829fe5e..022e0725d 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -28,7 +28,7 @@ MultiFrame::~MultiFrame() = default; MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_frame_) { /* - #NoCommit maybe not needed + TODO Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any difference? */ @@ -45,7 +45,7 @@ MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_f MultiFrame::MultiFrame(MultiFrame &&other) noexcept : default_frame_(std::move(other.default_frame_)) { /* - #NoCommit maybe not needed + TODO Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any difference? */ @@ -107,7 +107,7 @@ ValidFramesModifier::Iterator ValidFramesModifier::end() { ValidFramesConsumer::ValidFramesConsumer(MultiFrame &multiframe) : multiframe_(multiframe) {} ValidFramesConsumer::~ValidFramesConsumer() { - // #NoCommit possible optimisation: only DefragmentValidFrames if one frame has been invalidated? Only if does not + // TODO Possible optimisation: only DefragmentValidFrames if one frame has been invalidated? Only if does not // cost too much to store it multiframe_.DefragmentValidFrames(); } From 8c5edaaeb95512fe155f6791b15fef4b817523ec Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 14:07:15 +0100 Subject: [PATCH 11/51] Update type --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index b004bd96f..64c41f43d 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -16,7 +16,7 @@ #include "query/v2/bindings/frame.hpp" namespace memgraph::query::v2 { -constexpr uint64 kNumberOfFramesInMultiframe = 1000; // TODO have it configurable +constexpr uint64_t kNumberOfFramesInMultiframe = 1000; // TODO have it configurable class ValidFramesConsumer; class ValidFramesModifier; From 86f7b82bdc86d2992dad91acdb03ea91b619fbea Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 14:26:17 +0100 Subject: [PATCH 12/51] Clang tidy --- src/query/v2/interpreter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index 891699312..bfa7dfe0e 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -757,7 +757,7 @@ std::optional PullPlan::PullMultiple(AnyStrea // stream unsent results from previous pull auto iterator_for_valid_frame_only = multi_frame_.GetValidFramesReader(); - for (auto &frame : iterator_for_valid_frame_only) { + for (const auto &frame : iterator_for_valid_frame_only) { stream_values(frame); ++i; } @@ -771,7 +771,7 @@ std::optional PullPlan::PullMultiple(AnyStrea if (!output_symbols.empty()) { auto iterator_for_valid_frame_only = multi_frame_.GetValidFramesReader(); - for (auto &frame : iterator_for_valid_frame_only) { + for (const auto &frame : iterator_for_valid_frame_only) { stream_values(frame); ++i; } From 9faa206f95b582674baf9ec4dbbccf944c2ea62e Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 15:03:03 +0100 Subject: [PATCH 13/51] Clang tidy --- src/query/v2/multiframe.cpp | 33 ++++++++++++++------------------- src/query/v2/multiframe.hpp | 10 +++++----- 2 files changed, 19 insertions(+), 24 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 022e0725d..5c39390c9 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -37,13 +37,12 @@ MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_f [&default_frame = default_frame_](const auto &other_frame) { if (other_frame.IsValid()) { return other_frame; - } else { - return default_frame; } + return default_frame; }); } -MultiFrame::MultiFrame(MultiFrame &&other) noexcept : default_frame_(std::move(other.default_frame_)) { +MultiFrame::MultiFrame(MultiFrame &&other) : default_frame_(std::move(other.default_frame_)) { /* TODO Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any @@ -54,9 +53,8 @@ MultiFrame::MultiFrame(MultiFrame &&other) noexcept : default_frame_(std::move(o std::back_inserter(frames_), [&default_frame = default_frame_](const auto &other_frame) { if (other_frame.IsValid()) { return other_frame; - } else { - return default_frame; } + return default_frame; }); } @@ -68,38 +66,35 @@ bool MultiFrame::HasValidFrame() const noexcept { return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return frame.IsValid(); }); } -void MultiFrame::DefragmentValidFrames() noexcept { +void MultiFrame::DefragmentValidFrames() { /* from: https://en.cppreference.com/w/cpp/algorithm/remove "Removing is done by shifting (by means of copy assignment (until C++11)move assignment (since C++11)) the elements in the range in such a way that the elements that are not to be removed appear in the beginning of the range. Relative order of the elements that remain is preserved and the physical size of the container is unchanged." */ - std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); }); + [[maybe_unused]] const auto it = + std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); }); } -ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader(*this); } +ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader{*this}; } -ValidFramesModifier MultiFrame::GetValidFramesModifier() { return ValidFramesModifier(*this); } +ValidFramesModifier MultiFrame::GetValidFramesModifier() { return ValidFramesModifier{*this}; } -ValidFramesConsumer MultiFrame::GetValidFramesConsumer() { return ValidFramesConsumer(*this); } +ValidFramesConsumer MultiFrame::GetValidFramesConsumer() { return ValidFramesConsumer{*this}; } -InvalidFramesPopulator MultiFrame::GetInvalidFramesPopulator() { return InvalidFramesPopulator(*this); } +InvalidFramesPopulator MultiFrame::GetInvalidFramesPopulator() { return InvalidFramesPopulator{*this}; } ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multiframe) {} -ValidFramesReader::~ValidFramesReader() = default; - -ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesReader::Iterator ValidFramesReader::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } ValidFramesModifier::ValidFramesModifier(MultiFrame &multiframe) : multiframe_(multiframe) {} -ValidFramesModifier::~ValidFramesModifier() = default; - -ValidFramesModifier::Iterator ValidFramesModifier::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ValidFramesModifier::Iterator ValidFramesModifier::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesModifier::Iterator ValidFramesModifier::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); } @@ -112,7 +107,7 @@ ValidFramesConsumer::~ValidFramesConsumer() { multiframe_.DefragmentValidFrames(); } -ValidFramesConsumer::Iterator ValidFramesConsumer::begin() { return Iterator(&multiframe_.frames_[0], *this); } +ValidFramesConsumer::Iterator ValidFramesConsumer::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesConsumer::Iterator ValidFramesConsumer::end() { return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); @@ -133,7 +128,7 @@ InvalidFramesPopulator::Iterator InvalidFramesPopulator::begin() { } InvalidFramesPopulator::Iterator InvalidFramesPopulator::end() { - return Iterator(&multiframe_.frames_[multiframe_.frames_.size()]); + return Iterator{&multiframe_.frames_[multiframe_.frames_.size()]}; } } // namespace memgraph::query::v2 diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 64c41f43d..3fe95888a 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -33,8 +33,8 @@ class MultiFrame { MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); ~MultiFrame(); - MultiFrame(const MultiFrame &other); // copy constructor - MultiFrame(MultiFrame &&other) noexcept; // move constructor + MultiFrame(const MultiFrame &other); // copy constructor + MultiFrame(MultiFrame &&other); // move constructor MultiFrame &operator=(const MultiFrame &other) = delete; MultiFrame &operator=(MultiFrame &&other) noexcept = delete; @@ -79,7 +79,7 @@ class MultiFrame { inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); } private: - void DefragmentValidFrames() noexcept; + void DefragmentValidFrames(); FrameWithValidity default_frame_; utils::pmr::vector frames_ = @@ -90,7 +90,7 @@ class ValidFramesReader { public: ValidFramesReader(MultiFrame &multiframe); - ~ValidFramesReader(); + ~ValidFramesReader() = default; ValidFramesReader(const ValidFramesReader &other) = delete; // copy constructor ValidFramesReader(ValidFramesReader &&other) noexcept = delete; // move constructor ValidFramesReader &operator=(const ValidFramesReader &other) = delete; // copy assignment @@ -137,7 +137,7 @@ class ValidFramesModifier { public: ValidFramesModifier(MultiFrame &multiframe); - ~ValidFramesModifier(); + ~ValidFramesModifier() = default; ValidFramesModifier(const ValidFramesModifier &other) = delete; // copy constructor ValidFramesModifier(ValidFramesModifier &&other) noexcept = delete; // move constructor ValidFramesModifier &operator=(const ValidFramesModifier &other) = delete; // copy assignment From cc3bcf1dc2337ef869977f032f1449cf45e63656 Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 15:43:24 +0100 Subject: [PATCH 14/51] Clang tidy --- src/query/v2/multiframe.cpp | 14 +++++--------- src/query/v2/multiframe.hpp | 18 +++++++++--------- 2 files changed, 14 insertions(+), 18 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 5c39390c9..dba5d3ee5 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -24,8 +24,6 @@ MultiFrame::MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, MG_ASSERT(!default_frame.IsValid()); } -MultiFrame::~MultiFrame() = default; - MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_frame_) { /* TODO @@ -42,7 +40,7 @@ MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_f }); } -MultiFrame::MultiFrame(MultiFrame &&other) : default_frame_(std::move(other.default_frame_)) { +MultiFrame::MultiFrame(MultiFrame &&other) noexcept : default_frame_(std::move(other.default_frame_)) { /* TODO Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any @@ -89,14 +87,14 @@ ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multi ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesReader::Iterator ValidFramesReader::end() { - return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); + return Iterator{&multiframe_.frames_[multiframe_.frames_.size()], *this}; } ValidFramesModifier::ValidFramesModifier(MultiFrame &multiframe) : multiframe_(multiframe) {} ValidFramesModifier::Iterator ValidFramesModifier::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesModifier::Iterator ValidFramesModifier::end() { - return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); + return Iterator{&multiframe_.frames_[multiframe_.frames_.size()], *this}; } ValidFramesConsumer::ValidFramesConsumer(MultiFrame &multiframe) : multiframe_(multiframe) {} @@ -110,17 +108,15 @@ ValidFramesConsumer::~ValidFramesConsumer() { ValidFramesConsumer::Iterator ValidFramesConsumer::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesConsumer::Iterator ValidFramesConsumer::end() { - return Iterator(&multiframe_.frames_[multiframe_.frames_.size()], *this); + return Iterator{&multiframe_.frames_[multiframe_.frames_.size()], *this}; } InvalidFramesPopulator::InvalidFramesPopulator(MultiFrame &multiframe) : multiframe_(multiframe) {} -InvalidFramesPopulator::~InvalidFramesPopulator() = default; - InvalidFramesPopulator::Iterator InvalidFramesPopulator::begin() { for (auto idx = 0UL; idx < multiframe_.frames_.size(); ++idx) { if (!multiframe_.frames_[idx].IsValid()) { - return Iterator(&multiframe_.frames_[idx]); + return Iterator{&multiframe_.frames_[idx]}; } } diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 3fe95888a..9ed707002 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -31,10 +31,10 @@ class MultiFrame { friend class InvalidFramesPopulator; MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); - ~MultiFrame(); + ~MultiFrame() = default; - MultiFrame(const MultiFrame &other); // copy constructor - MultiFrame(MultiFrame &&other); // move constructor + MultiFrame(const MultiFrame &other); // copy constructor + MultiFrame(MultiFrame &&other) noexcept; // move constructor MultiFrame &operator=(const MultiFrame &other) = delete; MultiFrame &operator=(MultiFrame &&other) noexcept = delete; @@ -88,7 +88,7 @@ class MultiFrame { class ValidFramesReader { public: - ValidFramesReader(MultiFrame &multiframe); + explicit ValidFramesReader(MultiFrame &multiframe); ~ValidFramesReader() = default; ValidFramesReader(const ValidFramesReader &other) = delete; // copy constructor @@ -135,7 +135,7 @@ class ValidFramesReader { class ValidFramesModifier { public: - ValidFramesModifier(MultiFrame &multiframe); + explicit ValidFramesModifier(MultiFrame &multiframe); ~ValidFramesModifier() = default; ValidFramesModifier(const ValidFramesModifier &other) = delete; // copy constructor @@ -183,7 +183,7 @@ class ValidFramesModifier { class ValidFramesConsumer { public: - ValidFramesConsumer(MultiFrame &multiframe); + explicit ValidFramesConsumer(MultiFrame &multiframe); ~ValidFramesConsumer(); ValidFramesConsumer(const ValidFramesConsumer &other) = delete; // copy constructor @@ -231,8 +231,8 @@ class ValidFramesConsumer { class InvalidFramesPopulator { public: - InvalidFramesPopulator(MultiFrame &multiframe); - ~InvalidFramesPopulator(); + explicit InvalidFramesPopulator(MultiFrame &multiframe); + ~InvalidFramesPopulator() = default; InvalidFramesPopulator(const InvalidFramesPopulator &other) = delete; // copy constructor InvalidFramesPopulator(InvalidFramesPopulator &&other) noexcept = delete; // move constructor @@ -247,7 +247,7 @@ class InvalidFramesPopulator { using reference = FrameWithValidity &; using internal_ptr = FrameWithValidity *; - Iterator(internal_ptr ptr) : ptr_(ptr) {} + explicit Iterator(internal_ptr ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } From f107ef8aea4f8dabd210a494b378baf842d48c3f Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 16:12:50 +0100 Subject: [PATCH 15/51] Default destructor in header --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 9ed707002..e26b138c8 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -185,7 +185,7 @@ class ValidFramesConsumer { public: explicit ValidFramesConsumer(MultiFrame &multiframe); - ~ValidFramesConsumer(); + ~ValidFramesConsumer() = default; ValidFramesConsumer(const ValidFramesConsumer &other) = delete; // copy constructor ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete; // move constructor ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete; // copy assignment From 00a4127e4e563eac97d4dda487fceac9ebab7dce Mon Sep 17 00:00:00 2001 From: jeremy Date: Tue, 29 Nov 2022 17:15:52 +0100 Subject: [PATCH 16/51] Remove incorrect = default --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index e26b138c8..9ed707002 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -185,7 +185,7 @@ class ValidFramesConsumer { public: explicit ValidFramesConsumer(MultiFrame &multiframe); - ~ValidFramesConsumer() = default; + ~ValidFramesConsumer(); ValidFramesConsumer(const ValidFramesConsumer &other) = delete; // copy constructor ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete; // move constructor ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete; // copy assignment From 9c0c0a2d1cf134197bebb5af25b767d88f498035 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 10:14:07 +0100 Subject: [PATCH 17/51] Add clang warning suppress --- src/query/v2/multiframe.cpp | 21 +++++---------------- src/query/v2/multiframe.hpp | 2 +- 2 files changed, 6 insertions(+), 17 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index dba5d3ee5..fea74d141 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -40,21 +40,9 @@ MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_f }); } -MultiFrame::MultiFrame(MultiFrame &&other) noexcept : default_frame_(std::move(other.default_frame_)) { - /* - TODO - Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any - difference? - */ - frames_.reserve(other.frames_.size()); - std::transform(make_move_iterator(other.frames_.begin()), make_move_iterator(other.frames_.end()), - std::back_inserter(frames_), [&default_frame = default_frame_](const auto &other_frame) { - if (other_frame.IsValid()) { - return other_frame; - } - return default_frame; - }); -} +// NOLINTNEXTLINE (bugprone-exception-escape) +MultiFrame::MultiFrame(MultiFrame &&other) noexcept + : default_frame_(std::move(other.default_frame_)), frames_(std::move(other.frames_)) {} void MultiFrame::ResetAllFramesInvalid() noexcept { std::for_each(frames_.begin(), frames_.end(), [](auto &frame) { frame.MakeInvalid(); }); @@ -64,7 +52,8 @@ bool MultiFrame::HasValidFrame() const noexcept { return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return frame.IsValid(); }); } -void MultiFrame::DefragmentValidFrames() { +// NOLINTNEXTLINE (bugprone-exception-escape) +void MultiFrame::DefragmentValidFrames() noexcept { /* from: https://en.cppreference.com/w/cpp/algorithm/remove "Removing is done by shifting (by means of copy assignment (until C++11)move assignment (since C++11)) the elements diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 9ed707002..e7c5aa8cd 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -79,7 +79,7 @@ class MultiFrame { inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); } private: - void DefragmentValidFrames(); + void DefragmentValidFrames() noexcept; FrameWithValidity default_frame_; utils::pmr::vector frames_ = From a10c254caaad47c831320a72f9a2a6720ea6407e Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 11:43:16 +0100 Subject: [PATCH 18/51] Add // NOLINTNEXTLINE to correct place --- src/query/v2/multiframe.cpp | 3 ++- src/query/v2/multiframe.hpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index fea74d141..4573beaea 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -88,7 +88,8 @@ ValidFramesModifier::Iterator ValidFramesModifier::end() { ValidFramesConsumer::ValidFramesConsumer(MultiFrame &multiframe) : multiframe_(multiframe) {} -ValidFramesConsumer::~ValidFramesConsumer() { +// NOLINTNEXTLINE (bugprone-exception-escape) +ValidFramesConsumer::~ValidFramesConsumer() noexcept { // TODO Possible optimisation: only DefragmentValidFrames if one frame has been invalidated? Only if does not // cost too much to store it multiframe_.DefragmentValidFrames(); diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index e7c5aa8cd..7c59300de 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -79,6 +79,7 @@ class MultiFrame { inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); } private: + // NOLINTNEXTLINE (bugprone-exception-escape) void DefragmentValidFrames() noexcept; FrameWithValidity default_frame_; @@ -185,7 +186,7 @@ class ValidFramesConsumer { public: explicit ValidFramesConsumer(MultiFrame &multiframe); - ~ValidFramesConsumer(); + ~ValidFramesConsumer() noexcept; ValidFramesConsumer(const ValidFramesConsumer &other) = delete; // copy constructor ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete; // move constructor ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete; // copy assignment From 02ca6734c1b6467460f5441a6566e4b9ecbd83b5 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 14:11:38 +0100 Subject: [PATCH 19/51] Correct comment to follow common style --- src/query/v2/multiframe.hpp | 51 ++++++++++++++++++------------------- 1 file changed, 25 insertions(+), 26 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 7c59300de..5ea8d1c65 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -38,38 +38,37 @@ class MultiFrame { MultiFrame &operator=(const MultiFrame &other) = delete; MultiFrame &operator=(MultiFrame &&other) noexcept = delete; - /*! - Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid - state in the multiframe. - Iteration goes in a deterministic order. - One can't modify the validity of the frame nor its content with this implementation. - */ + /* + * Returns a object on which one can iterate in a for-loop. By doing so, you will only get Frames that are in a valid + * state in the MultiFrame. + * Iteration goes in a deterministic order. + * One can't modify the validity of the Frame nor its content with this implementation. + */ ValidFramesReader GetValidFramesReader(); - /*! - Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid - state in the multiframe. - Iteration goes in a deterministic order. - One can't modify the validity of the frame with this implementation. One can modify its content. - */ + /* + * Returns a object on which one can iterate in a for-loop. By doing so, you will only get Frames that are in a valid + * state in the MultiFrame. + * Iteration goes in a deterministic order. + * One can't modify the validity of the Frame with this implementation. One can modify its content. + */ ValidFramesModifier GetValidFramesModifier(); - /*! - Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in a valid - state in the multiframe. - Iteration goes in a deterministic order. - One can modify the validity of the frame with this implementation. - If you do not plan to modify the validity of the frames, use GetValidFramesReader/GetValidFramesModifer instead as - this is faster. - */ + /* + * Returns a object on which one can iterate in a for-loop. By doing so, you will only get Frames that are in a valid + * state in the MultiFrame. + * Iteration goes in a deterministic order. + * One can modify the validity of the Frame with this implementation. + * If you do not plan to modify the validity of the Frames, use GetValidFramesReader/GetValidFramesModifer instead as + * this is faster. + */ ValidFramesConsumer GetValidFramesConsumer(); - /*! - Returns a object on which one can iterate in a for-loop. By doing so, you will only get frames that are in an invalid - state in the multiframe. - Iteration goes in a deterministic order. - One can modify the validity of the frame with this implementation. - */ + /* + * Returns a object on which one can iterate in a for-loop. By doing so, you will only get Frames that are in an + * invalid state in the MultiFrame. Iteration goes in a deterministic order. One can modify the validity of + * the Frame with this implementation. + */ InvalidFramesPopulator GetInvalidFramesPopulator(); void ResetAllFramesInvalid() noexcept; From 55008a2927bcfe2e596783f9bf17a079c5ab9093 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 14:17:01 +0100 Subject: [PATCH 20/51] Rename func ResetAllFramesInvalid->MakeAllFramesInvalid --- src/query/v2/interpreter.cpp | 4 ++-- src/query/v2/multiframe.cpp | 2 +- src/query/v2/multiframe.hpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index bfa7dfe0e..bf28ef875 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -761,7 +761,7 @@ std::optional PullPlan::PullMultiple(AnyStrea stream_values(frame); ++i; } - multi_frame_.ResetAllFramesInvalid(); + multi_frame_.MakeAllFramesInvalid(); } for (; !n || i < n;) { @@ -776,7 +776,7 @@ std::optional PullPlan::PullMultiple(AnyStrea ++i; } } - multi_frame_.ResetAllFramesInvalid(); + multi_frame_.MakeAllFramesInvalid(); } // If we finished because we streamed the requested n results, diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 4573beaea..3a180ab89 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -44,7 +44,7 @@ MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_f MultiFrame::MultiFrame(MultiFrame &&other) noexcept : default_frame_(std::move(other.default_frame_)), frames_(std::move(other.frames_)) {} -void MultiFrame::ResetAllFramesInvalid() noexcept { +void MultiFrame::MakeAllFramesInvalid() noexcept { std::for_each(frames_.begin(), frames_.end(), [](auto &frame) { frame.MakeInvalid(); }); } diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 5ea8d1c65..27d010fd4 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -71,7 +71,7 @@ class MultiFrame { */ InvalidFramesPopulator GetInvalidFramesPopulator(); - void ResetAllFramesInvalid() noexcept; + void MakeAllFramesInvalid() noexcept; bool HasValidFrame() const noexcept; From 969b8f0da79ff238d87b9ed1afa7c35447ddd8c0 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 14:32:19 +0100 Subject: [PATCH 21/51] Remove un-necessary internal_ptr --- src/query/v2/multiframe.hpp | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 27d010fd4..fb6cbe9e6 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -102,9 +102,9 @@ class ValidFramesReader { using value_type = const Frame; using pointer = value_type *; using reference = const Frame &; - using internal_ptr = FrameWithValidity *; - Iterator(internal_ptr ptr, ValidFramesReader &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + Iterator(FrameWithValidity *ptr, ValidFramesReader &iterator_wrapper) + : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } @@ -122,7 +122,7 @@ class ValidFramesReader { friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; private: - internal_ptr ptr_; + FrameWithValidity *ptr_; ValidFramesReader &iterator_wrapper_; }; @@ -149,9 +149,8 @@ class ValidFramesModifier { using value_type = Frame; using pointer = value_type *; using reference = Frame &; - using internal_ptr = FrameWithValidity *; - Iterator(internal_ptr ptr, ValidFramesModifier &iterator_wrapper) + Iterator(FrameWithValidity *ptr, ValidFramesModifier &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} reference operator*() const { return *ptr_; } @@ -170,7 +169,7 @@ class ValidFramesModifier { friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; private: - internal_ptr ptr_; + FrameWithValidity *ptr_; ValidFramesModifier &iterator_wrapper_; }; @@ -197,9 +196,8 @@ class ValidFramesConsumer { using value_type = FrameWithValidity; using pointer = value_type *; using reference = FrameWithValidity &; - using internal_ptr = FrameWithValidity *; - Iterator(internal_ptr ptr, ValidFramesConsumer &iterator_wrapper) + Iterator(FrameWithValidity *ptr, ValidFramesConsumer &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} reference operator*() const { return *ptr_; } @@ -218,7 +216,7 @@ class ValidFramesConsumer { friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; private: - internal_ptr ptr_; + FrameWithValidity *ptr_; ValidFramesConsumer &iterator_wrapper_; }; @@ -245,9 +243,8 @@ class InvalidFramesPopulator { using value_type = FrameWithValidity; using pointer = value_type *; using reference = FrameWithValidity &; - using internal_ptr = FrameWithValidity *; - explicit Iterator(internal_ptr ptr) : ptr_(ptr) {} + explicit Iterator(FrameWithValidity *ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } @@ -263,7 +260,7 @@ class InvalidFramesPopulator { friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; private: - internal_ptr ptr_; + FrameWithValidity *ptr_; }; Iterator begin(); From 072bc58b1eac7462ee9de6b53c287253d2c51a3c Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 14:50:28 +0100 Subject: [PATCH 22/51] Reverse condition in while() + comment --- src/query/v2/multiframe.cpp | 10 ++++++++++ src/query/v2/multiframe.hpp | 6 +++--- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 3a180ab89..7873969ac 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -16,6 +16,16 @@ namespace memgraph::query::v2 { +// #NoCommit uncomment https://github.com/memgraph/memgraph/pull/676#discussion_r1035704661 +// static_assert(std::forward_iterator && +// std::equality_comparable); +// static_assert(std::forward_iterator && +// std::equality_comparable); +// static_assert(std::forward_iterator && +// std::equality_comparable); +// static_assert(std::forward_iterator && +// std::equality_comparable); + MultiFrame::MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory) : default_frame_(default_frame), diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index fb6cbe9e6..9186d739c 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -113,7 +113,7 @@ class ValidFramesReader { Iterator &operator++() { do { ptr_++; - } while (!this->ptr_->IsValid() && *this != iterator_wrapper_.end()); + } while (*this != iterator_wrapper_.end() && !this->ptr_->IsValid()); return *this; } @@ -160,7 +160,7 @@ class ValidFramesModifier { Iterator &operator++() { do { ptr_++; - } while (!this->ptr_->IsValid() && *this != iterator_wrapper_.end()); + } while (*this != iterator_wrapper_.end() && !this->ptr_->IsValid()); return *this; } @@ -207,7 +207,7 @@ class ValidFramesConsumer { Iterator &operator++() { do { ptr_++; - } while (!this->ptr_->IsValid() && *this != iterator_wrapper_.end()); + } while (*this != iterator_wrapper_.end() && !this->ptr_->IsValid()); return *this; } From deb31e4b77045df14b4ec70d8b6e82a1d960509d Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 16:00:24 +0100 Subject: [PATCH 23/51] Multiframe only expects size of frame instead of default frame --- src/query/v2/interpreter.cpp | 2 +- src/query/v2/multiframe.cpp | 9 ++++----- src/query/v2/multiframe.hpp | 2 +- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index bf28ef875..b5f916e9f 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -688,7 +688,7 @@ PullPlan::PullPlan(const std::shared_ptr plan, const Parameters &par : plan_(plan), cursor_(plan->plan().MakeCursor(execution_memory)), frame_(plan->symbol_table().max_position(), execution_memory), - multi_frame_(frame_, kNumberOfFramesInMultiframe, execution_memory), + multi_frame_(plan->symbol_table().max_position(), kNumberOfFramesInMultiframe, execution_memory), memory_limit_(memory_limit) { ctx_.db_accessor = dba; ctx_.symbol_table = plan->symbol_table(); diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 7873969ac..640e63510 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -26,12 +26,11 @@ namespace memgraph::query::v2 { // static_assert(std::forward_iterator && // std::equality_comparable); -MultiFrame::MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, - utils::MemoryResource *execution_memory) - : default_frame_(default_frame), - frames_(utils::pmr::vector(number_of_frames, default_frame, execution_memory)) { +MultiFrame::MultiFrame(size_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory) + : default_frame_(FrameWithValidity(size_of_frame, execution_memory)), + frames_(utils::pmr::vector(number_of_frames, default_frame_, execution_memory)) { MG_ASSERT(number_of_frames > 0); - MG_ASSERT(!default_frame.IsValid()); + MG_ASSERT(!default_frame_.IsValid()); } MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_frame_) { diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 9186d739c..8f6109ab4 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -30,7 +30,7 @@ class MultiFrame { friend class ValidFramesReader; friend class InvalidFramesPopulator; - MultiFrame(FrameWithValidity default_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); + MultiFrame(size_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); ~MultiFrame() = default; MultiFrame(const MultiFrame &other); // copy constructor From a2027fc6ac8e3fcd08c6d726ab4fb26d412598d9 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 16:09:06 +0100 Subject: [PATCH 24/51] Remove default_frame --- src/query/v2/multiframe.cpp | 22 +++++----------------- src/query/v2/multiframe.hpp | 1 - 2 files changed, 5 insertions(+), 18 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 640e63510..a4ec3be4f 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -27,31 +27,19 @@ namespace memgraph::query::v2 { // std::equality_comparable); MultiFrame::MultiFrame(size_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory) - : default_frame_(FrameWithValidity(size_of_frame, execution_memory)), - frames_(utils::pmr::vector(number_of_frames, default_frame_, execution_memory)) { + : frames_(utils::pmr::vector( + number_of_frames, FrameWithValidity(size_of_frame, execution_memory), execution_memory)) { MG_ASSERT(number_of_frames > 0); - MG_ASSERT(!default_frame_.IsValid()); } -MultiFrame::MultiFrame(const MultiFrame &other) : default_frame_(other.default_frame_) { - /* - TODO - Do we just copy all frames or do we make distinctions between valid and not valid frames? Does it make any - difference? - */ +MultiFrame::MultiFrame(const MultiFrame &other) { frames_.reserve(other.frames_.size()); std::transform(other.frames_.begin(), other.frames_.end(), std::back_inserter(frames_), - [&default_frame = default_frame_](const auto &other_frame) { - if (other_frame.IsValid()) { - return other_frame; - } - return default_frame; - }); + [](const auto &other_frame) { return other_frame; }); } // NOLINTNEXTLINE (bugprone-exception-escape) -MultiFrame::MultiFrame(MultiFrame &&other) noexcept - : default_frame_(std::move(other.default_frame_)), frames_(std::move(other.frames_)) {} +MultiFrame::MultiFrame(MultiFrame &&other) noexcept : frames_(std::move(other.frames_)) {} void MultiFrame::MakeAllFramesInvalid() noexcept { std::for_each(frames_.begin(), frames_.end(), [](auto &frame) { frame.MakeInvalid(); }); diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 8f6109ab4..0da5c1717 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -81,7 +81,6 @@ class MultiFrame { // NOLINTNEXTLINE (bugprone-exception-escape) void DefragmentValidFrames() noexcept; - FrameWithValidity default_frame_; utils::pmr::vector frames_ = utils::pmr::vector(0, FrameWithValidity{1}, utils::NewDeleteResource()); }; From 56556f7c2d41d221cb069f7a620c5f436a60d757 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 16:22:39 +0100 Subject: [PATCH 25/51] Update incorrect de-referencing --- src/query/v2/multiframe.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index a4ec3be4f..5bae5c108 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -73,14 +73,14 @@ ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multi ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesReader::Iterator ValidFramesReader::end() { - return Iterator{&multiframe_.frames_[multiframe_.frames_.size()], *this}; + return Iterator{multiframe_.frames_.data() + multiframe_.frames_.size(), *this}; } ValidFramesModifier::ValidFramesModifier(MultiFrame &multiframe) : multiframe_(multiframe) {} ValidFramesModifier::Iterator ValidFramesModifier::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesModifier::Iterator ValidFramesModifier::end() { - return Iterator{&multiframe_.frames_[multiframe_.frames_.size()], *this}; + return Iterator{multiframe_.frames_.data() + multiframe_.frames_.size(), *this}; } ValidFramesConsumer::ValidFramesConsumer(MultiFrame &multiframe) : multiframe_(multiframe) {} @@ -95,7 +95,7 @@ ValidFramesConsumer::~ValidFramesConsumer() noexcept { ValidFramesConsumer::Iterator ValidFramesConsumer::begin() { return Iterator{&multiframe_.frames_[0], *this}; } ValidFramesConsumer::Iterator ValidFramesConsumer::end() { - return Iterator{&multiframe_.frames_[multiframe_.frames_.size()], *this}; + return Iterator{multiframe_.frames_.data() + multiframe_.frames_.size(), *this}; } InvalidFramesPopulator::InvalidFramesPopulator(MultiFrame &multiframe) : multiframe_(multiframe) {} @@ -111,7 +111,7 @@ InvalidFramesPopulator::Iterator InvalidFramesPopulator::begin() { } InvalidFramesPopulator::Iterator InvalidFramesPopulator::end() { - return Iterator{&multiframe_.frames_[multiframe_.frames_.size()]}; + return Iterator{multiframe_.frames_.data() + multiframe_.frames_.size()}; } } // namespace memgraph::query::v2 From 38f3a4cacbd4ed1c4b37c7fb7c0b6c1c7bc1661c Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 16:28:01 +0100 Subject: [PATCH 26/51] Use range for loop instead of idx based --- src/query/v2/multiframe.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 5bae5c108..e6e8685fc 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -101,12 +101,11 @@ ValidFramesConsumer::Iterator ValidFramesConsumer::end() { InvalidFramesPopulator::InvalidFramesPopulator(MultiFrame &multiframe) : multiframe_(multiframe) {} InvalidFramesPopulator::Iterator InvalidFramesPopulator::begin() { - for (auto idx = 0UL; idx < multiframe_.frames_.size(); ++idx) { - if (!multiframe_.frames_[idx].IsValid()) { - return Iterator{&multiframe_.frames_[idx]}; + for (auto &frame : multiframe_.frames_) { + if (!frame.IsValid()) { + return Iterator{&frame}; } } - return end(); } From 5e64b19745f89133827b8966bb674ba4fa8d1ad2 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 16:30:55 +0100 Subject: [PATCH 27/51] Replace pull_count_->did_pull_ --- src/query/v2/plan/operator.cpp | 10 +++++----- src/query/v2/plan/operator.lcp | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index db2a1372d..c9fd05609 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -257,8 +257,8 @@ class DistributedCreateNodeCursor : public Cursor { bool Once::OnceCursor::Pull(Frame &, ExecutionContext &context) { SCOPED_PROFILE_OP("Once"); - if (pull_count_ < 1) { - pull_count_++; + if (!did_pull_) { + did_pull_ = true; return true; } return false; @@ -270,14 +270,14 @@ void Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &c auto iterator_for_valid_frame_only = multi_frame.GetValidFramesConsumer(); auto first_it = iterator_for_valid_frame_only.begin(); MG_ASSERT(first_it != iterator_for_valid_frame_only.end()); - if (pull_count_ < 1) { + if (!did_pull_) { auto *memory_resource = multi_frame.GetMemoryResource(); auto &frame = *first_it; frame.MakeValid(); for (auto &value : frame.elems()) { value = TypedValue{memory_resource}; } - pull_count_++; + did_pull_ = true; } } @@ -291,7 +291,7 @@ WITHOUT_SINGLE_INPUT(Once); void Once::OnceCursor::Shutdown() {} -void Once::OnceCursor::Reset() { pull_count_ = 0; } +void Once::OnceCursor::Reset() { did_pull_ = false; } CreateNode::CreateNode(const std::shared_ptr &input, const NodeCreationInfo &node_info) : input_(input ? input : std::make_shared()), node_info_(node_info) {} diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp index 9f8b7d6db..efa0d5df0 100644 --- a/src/query/v2/plan/operator.lcp +++ b/src/query/v2/plan/operator.lcp @@ -341,7 +341,7 @@ and false on every following Pull.") void Reset() override; private: - size_t pull_count_{0}; + bool did_pull_{false}; }; cpp<#) (:serialize (:slk)) From b0b8c0a5c94cbeb84137fd04e68dff714435d936 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 16:49:32 +0100 Subject: [PATCH 28/51] Add noexcept to basic functions --- src/expr/interpret/frame.hpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/expr/interpret/frame.hpp b/src/expr/interpret/frame.hpp index 6a220b6a0..1bc5bba83 100644 --- a/src/expr/interpret/frame.hpp +++ b/src/expr/interpret/frame.hpp @@ -49,9 +49,9 @@ class FrameWithValidity final : public Frame { FrameWithValidity(int64_t size, utils::MemoryResource *memory) : Frame(size, memory), is_valid_(false) {} - bool IsValid() const { return is_valid_; } - void MakeValid() { is_valid_ = true; } - void MakeInvalid() { is_valid_ = false; } + bool IsValid() const noexcept { return is_valid_; } + void MakeValid() noexcept { is_valid_ = true; } + void MakeInvalid() noexcept { is_valid_ = false; } private: bool is_valid_; From 9f9a81455f4c29902faee3d80d45de8f8a405f99 Mon Sep 17 00:00:00 2001 From: jeremy Date: Wed, 30 Nov 2022 16:56:35 +0100 Subject: [PATCH 29/51] Change type size_t->int64_t --- src/query/v2/multiframe.cpp | 2 +- src/query/v2/multiframe.hpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index e6e8685fc..27b0872ed 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -26,7 +26,7 @@ namespace memgraph::query::v2 { // static_assert(std::forward_iterator && // std::equality_comparable); -MultiFrame::MultiFrame(size_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory) +MultiFrame::MultiFrame(int64_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory) : frames_(utils::pmr::vector( number_of_frames, FrameWithValidity(size_of_frame, execution_memory), execution_memory)) { MG_ASSERT(number_of_frames > 0); diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 0da5c1717..5583e80f3 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -30,7 +30,7 @@ class MultiFrame { friend class ValidFramesReader; friend class InvalidFramesPopulator; - MultiFrame(size_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); + MultiFrame(int64_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); ~MultiFrame() = default; MultiFrame(const MultiFrame &other); // copy constructor From e5d892683c3790351674a7c0121aad309a4de660 Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 11:14:54 +0100 Subject: [PATCH 30/51] Keep wraper as ptr instead of ref --- src/query/v2/multiframe.hpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 5583e80f3..7c0f366eb 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -103,7 +103,7 @@ class ValidFramesReader { using reference = const Frame &; Iterator(FrameWithValidity *ptr, ValidFramesReader &iterator_wrapper) - : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } @@ -112,7 +112,7 @@ class ValidFramesReader { Iterator &operator++() { do { ptr_++; - } while (*this != iterator_wrapper_.end() && !this->ptr_->IsValid()); + } while (*this != iterator_wrapper_->end() && !this->ptr_->IsValid()); return *this; } @@ -122,7 +122,7 @@ class ValidFramesReader { private: FrameWithValidity *ptr_; - ValidFramesReader &iterator_wrapper_; + ValidFramesReader *iterator_wrapper_; }; Iterator begin(); @@ -150,7 +150,7 @@ class ValidFramesModifier { using reference = Frame &; Iterator(FrameWithValidity *ptr, ValidFramesModifier &iterator_wrapper) - : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } @@ -159,7 +159,7 @@ class ValidFramesModifier { Iterator &operator++() { do { ptr_++; - } while (*this != iterator_wrapper_.end() && !this->ptr_->IsValid()); + } while (*this != iterator_wrapper_->end() && !this->ptr_->IsValid()); return *this; } @@ -169,7 +169,7 @@ class ValidFramesModifier { private: FrameWithValidity *ptr_; - ValidFramesModifier &iterator_wrapper_; + ValidFramesModifier *iterator_wrapper_; }; Iterator begin(); @@ -197,7 +197,7 @@ class ValidFramesConsumer { using reference = FrameWithValidity &; Iterator(FrameWithValidity *ptr, ValidFramesConsumer &iterator_wrapper) - : ptr_(ptr), iterator_wrapper_(iterator_wrapper) {} + : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } @@ -206,7 +206,7 @@ class ValidFramesConsumer { Iterator &operator++() { do { ptr_++; - } while (*this != iterator_wrapper_.end() && !this->ptr_->IsValid()); + } while (*this != iterator_wrapper_->end() && !this->ptr_->IsValid()); return *this; } @@ -216,7 +216,7 @@ class ValidFramesConsumer { private: FrameWithValidity *ptr_; - ValidFramesConsumer &iterator_wrapper_; + ValidFramesConsumer *iterator_wrapper_; }; Iterator begin(); From 23bfd7f4fca4eb29aaff4c21404c357b7b10b450 Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 13:45:24 +0100 Subject: [PATCH 31/51] Updated OnceCursor --- src/query/v2/multiframe.cpp | 5 +++++ src/query/v2/multiframe.hpp | 7 +++++++ src/query/v2/plan/operator.cpp | 11 ++++------- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 27b0872ed..74ccf8478 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -41,6 +41,11 @@ MultiFrame::MultiFrame(const MultiFrame &other) { // NOLINTNEXTLINE (bugprone-exception-escape) MultiFrame::MultiFrame(MultiFrame &&other) noexcept : frames_(std::move(other.frames_)) {} +FrameWithValidity &MultiFrame::GetFirstFrame() { + MG_ASSERT(!frames_.empty()); + return frames_.front(); +} + void MultiFrame::MakeAllFramesInvalid() noexcept { std::for_each(frames_.begin(), frames_.end(), [](auto &frame) { frame.MakeInvalid(); }); } diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 7c0f366eb..e76f34c5a 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -71,6 +71,13 @@ class MultiFrame { */ InvalidFramesPopulator GetInvalidFramesPopulator(); + /** + * Return the first Frame of the MultiFrame. This is only meant to be used in very specific cases. Please consider + * using the iterators instead. + * The Frame can be valid or invalid. + */ + FrameWithValidity &GetFirstFrame(); + void MakeAllFramesInvalid() noexcept; bool HasValidFrame() const noexcept; diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index c9fd05609..9a000face 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -267,14 +267,11 @@ bool Once::OnceCursor::Pull(Frame &, ExecutionContext &context) { void Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) { SCOPED_PROFILE_OP("OnceMF"); - auto iterator_for_valid_frame_only = multi_frame.GetValidFramesConsumer(); - auto first_it = iterator_for_valid_frame_only.begin(); - MG_ASSERT(first_it != iterator_for_valid_frame_only.end()); if (!did_pull_) { - auto *memory_resource = multi_frame.GetMemoryResource(); - auto &frame = *first_it; - frame.MakeValid(); - for (auto &value : frame.elems()) { + auto &first_frame = multi_frame.GetFirstFrame(); + auto *memory_resource = first_frame.GetMemoryResource(); + first_frame.MakeValid(); + for (auto &value : first_frame.elems()) { value = TypedValue{memory_resource}; } did_pull_ = true; From 54907d2a1aefba1a4fb34ebd26b9227534908a1c Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 1 Dec 2022 14:19:15 +0100 Subject: [PATCH 32/51] Update src/query/v2/multiframe.cpp Co-authored-by: Kostas Kyrimis --- src/query/v2/multiframe.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 27b0872ed..05d2a755b 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -57,7 +57,6 @@ void MultiFrame::DefragmentValidFrames() noexcept { in the range in such a way that the elements that are not to be removed appear in the beginning of the range. Relative order of the elements that remain is preserved and the physical size of the container is unchanged." */ - [[maybe_unused]] const auto it = std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); }); } From 00fd69c17090ca259dfb0c1916b4c34e8ba2482f Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 14:19:41 +0100 Subject: [PATCH 33/51] Add statement to ignroe clang warning --- src/query/v2/multiframe.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 74ccf8478..41ac7c6ed 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -62,8 +62,8 @@ void MultiFrame::DefragmentValidFrames() noexcept { in the range in such a way that the elements that are not to be removed appear in the beginning of the range. Relative order of the elements that remain is preserved and the physical size of the container is unchanged." */ - [[maybe_unused]] const auto it = - std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); }); + // NOLINTNEXTLINE (bugprone-unused-return-value) + std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); }); } ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader{*this}; } From d0c960e90036dcf4a8422441039a3108009010ea Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 1 Dec 2022 14:20:19 +0100 Subject: [PATCH 34/51] Update src/query/v2/multiframe.hpp Co-authored-by: Kostas Kyrimis --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 5583e80f3..5651365a7 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -33,7 +33,7 @@ class MultiFrame { MultiFrame(int64_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory); ~MultiFrame() = default; - MultiFrame(const MultiFrame &other); // copy constructor + MultiFrame(const MultiFrame &other); MultiFrame(MultiFrame &&other) noexcept; // move constructor MultiFrame &operator=(const MultiFrame &other) = delete; MultiFrame &operator=(MultiFrame &&other) noexcept = delete; From ee9ba1a7f8a2f5f6559c05a0673e4f27dba2b92e Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 1 Dec 2022 14:20:26 +0100 Subject: [PATCH 35/51] Update src/query/v2/multiframe.hpp Co-authored-by: Kostas Kyrimis --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 5651365a7..782a1e8fc 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -34,7 +34,7 @@ class MultiFrame { ~MultiFrame() = default; MultiFrame(const MultiFrame &other); - MultiFrame(MultiFrame &&other) noexcept; // move constructor + MultiFrame(MultiFrame &&other) noexcept; MultiFrame &operator=(const MultiFrame &other) = delete; MultiFrame &operator=(MultiFrame &&other) noexcept = delete; From 6c441b80ec3b2f0bbf53b318807fc9b743304561 Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 1 Dec 2022 14:20:57 +0100 Subject: [PATCH 36/51] Update src/query/v2/multiframe.hpp Co-authored-by: Kostas Kyrimis --- src/query/v2/multiframe.hpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 782a1e8fc..f0219c7df 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -90,10 +90,10 @@ class ValidFramesReader { explicit ValidFramesReader(MultiFrame &multiframe); ~ValidFramesReader() = default; - ValidFramesReader(const ValidFramesReader &other) = delete; // copy constructor - ValidFramesReader(ValidFramesReader &&other) noexcept = delete; // move constructor - ValidFramesReader &operator=(const ValidFramesReader &other) = delete; // copy assignment - ValidFramesReader &operator=(ValidFramesReader &&other) noexcept = delete; // move assignment + ValidFramesReader(const ValidFramesReader &other) = delete; + ValidFramesReader(ValidFramesReader &&other) noexcept = delete; + ValidFramesReader &operator=(const ValidFramesReader &other) = delete; + ValidFramesReader &operator=(ValidFramesReader &&other) noexcept = delete; struct Iterator { using iterator_category = std::forward_iterator_tag; From 452722f4f8067831996a2d576eb48ec62b98ad90 Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 1 Dec 2022 14:52:30 +0100 Subject: [PATCH 37/51] Update src/query/v2/multiframe.hpp Co-authored-by: Kostas Kyrimis --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index f0219c7df..852674c8c 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -159,7 +159,7 @@ class ValidFramesModifier { Iterator &operator++() { do { ptr_++; - } while (*this != iterator_wrapper_.end() && !this->ptr_->IsValid()); + } while (*this != iterator_wrapper_.end() && ptr_->IsValid()); return *this; } From 29347c83e79505f51d1d3e7d2b1f2111612b01df Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 14:54:26 +0100 Subject: [PATCH 38/51] Remove unneeded tag --- src/query/v2/multiframe.hpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index e76f34c5a..fae95ccca 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -85,7 +85,6 @@ class MultiFrame { inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); } private: - // NOLINTNEXTLINE (bugprone-exception-escape) void DefragmentValidFrames() noexcept; utils::pmr::vector frames_ = From 8eec8399a3539d282e59b95032b4ed10202cf40f Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 15:04:51 +0100 Subject: [PATCH 39/51] Rmove unneeded "this" --- src/query/v2/multiframe.hpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index a82c4a242..5a0a5af8b 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -118,7 +118,7 @@ class ValidFramesReader { Iterator &operator++() { do { ptr_++; - } while (*this != iterator_wrapper_->end() && !this->ptr_->IsValid()); + } while (*this != iterator_wrapper_->end() && !ptr_->IsValid()); return *this; } @@ -165,7 +165,7 @@ class ValidFramesModifier { Iterator &operator++() { do { ptr_++; - } while (*this != iterator_wrapper_.end() && ptr_->IsValid()); + } while (*this != iterator_wrapper_->end() && ptr_->IsValid()); return *this; } @@ -212,7 +212,7 @@ class ValidFramesConsumer { Iterator &operator++() { do { ptr_++; - } while (*this != iterator_wrapper_->end() && !this->ptr_->IsValid()); + } while (*this != iterator_wrapper_->end() && !ptr_->IsValid()); return *this; } From 5cd0d5137eb266dfc75fa71593faa38cb6ff94fd Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 1 Dec 2022 15:18:50 +0100 Subject: [PATCH 40/51] Update src/query/v2/multiframe.hpp Co-authored-by: Kostas Kyrimis --- src/query/v2/multiframe.hpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 852674c8c..edeb9094d 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -81,8 +81,7 @@ class MultiFrame { // NOLINTNEXTLINE (bugprone-exception-escape) void DefragmentValidFrames() noexcept; - utils::pmr::vector frames_ = - utils::pmr::vector(0, FrameWithValidity{1}, utils::NewDeleteResource()); + utils::pmr::vector frames_; }; class ValidFramesReader { From 4bbf3c95ca8a87834983ce2f2955f6a2e8878eec Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Thu, 1 Dec 2022 15:49:17 +0100 Subject: [PATCH 41/51] Update src/query/v2/multiframe.cpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/query/v2/multiframe.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 904961213..ab47ee19f 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -32,11 +32,7 @@ MultiFrame::MultiFrame(int64_t size_of_frame, size_t number_of_frames, utils::Me MG_ASSERT(number_of_frames > 0); } -MultiFrame::MultiFrame(const MultiFrame &other) { - frames_.reserve(other.frames_.size()); - std::transform(other.frames_.begin(), other.frames_.end(), std::back_inserter(frames_), - [](const auto &other_frame) { return other_frame; }); -} +MultiFrame::MultiFrame(const MultiFrame &other) :frames_{other.frames_} {} // NOLINTNEXTLINE (bugprone-exception-escape) MultiFrame::MultiFrame(MultiFrame &&other) noexcept : frames_(std::move(other.frames_)) {} From db45845619eddbe7470fbad2b148d0f62710c071 Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 15:52:35 +0100 Subject: [PATCH 42/51] format --- src/query/v2/multiframe.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index ab47ee19f..e5aaca883 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -32,7 +32,7 @@ MultiFrame::MultiFrame(int64_t size_of_frame, size_t number_of_frames, utils::Me MG_ASSERT(number_of_frames > 0); } -MultiFrame::MultiFrame(const MultiFrame &other) :frames_{other.frames_} {} +MultiFrame::MultiFrame(const MultiFrame &other) : frames_{other.frames_} {} // NOLINTNEXTLINE (bugprone-exception-escape) MultiFrame::MultiFrame(MultiFrame &&other) noexcept : frames_(std::move(other.frames_)) {} From 13cabcaab5362157f84ca5f0dda0017e9dad1650 Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 16:31:21 +0100 Subject: [PATCH 43/51] Re-implement ValidFramesReader and iterators --- src/query/v2/multiframe.cpp | 22 ++++++++++++++++++---- src/query/v2/multiframe.hpp | 10 +++------- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index e5aaca883..c007baf7c 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -11,6 +11,9 @@ #include "query/v2/multiframe.hpp" +#include +#include + #include "query/v2/bindings/frame.hpp" #include "utils/pmr/vector.hpp" @@ -71,12 +74,23 @@ ValidFramesConsumer MultiFrame::GetValidFramesConsumer() { return ValidFramesCon InvalidFramesPopulator MultiFrame::GetInvalidFramesPopulator() { return InvalidFramesPopulator{*this}; } -ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multiframe) {} +ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multiframe) { + /* + From: https://en.cppreference.com/w/cpp/algorithm/find + Returns an iterator to the first element in the range [first, last) that satisfies specific criteria: + find_if searches for an element for which predicate p returns true + Return value + Iterator to the first element satisfying the condition or last if no such element is found. + + -> this is what we want. We want the "after" last valid frame (weather this is vector::end or and invalid frame). + */ + auto it = std::find_if(multiframe.frames_.begin(), multiframe.frames_.end(), + [](const auto &frame) { return !frame.IsValid(); }); + after_last_valid_frame_ = multiframe_.frames_.data() + std::distance(multiframe.frames_.begin(), it); +} ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator{&multiframe_.frames_[0], *this}; } -ValidFramesReader::Iterator ValidFramesReader::end() { - return Iterator{multiframe_.frames_.data() + multiframe_.frames_.size(), *this}; -} +ValidFramesReader::Iterator ValidFramesReader::end() { return Iterator{after_last_valid_frame_, *this}; } ValidFramesModifier::ValidFramesModifier(MultiFrame &multiframe) : multiframe_(multiframe) {} diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index cc3dbd12a..c1bf9cdf1 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -107,18 +107,14 @@ class ValidFramesReader { using pointer = value_type *; using reference = const Frame &; - Iterator(FrameWithValidity *ptr, ValidFramesReader &iterator_wrapper) - : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} + explicit Iterator(FrameWithValidity *ptr, ValidFramesReader &iterator_wrapper) : ptr_(ptr) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } // Prefix increment Iterator &operator++() { - do { - ptr_++; - } while (*this != iterator_wrapper_->end() && !ptr_->IsValid()); - + ptr_++; return *this; } @@ -127,13 +123,13 @@ class ValidFramesReader { private: FrameWithValidity *ptr_; - ValidFramesReader *iterator_wrapper_; }; Iterator begin(); Iterator end(); private: + FrameWithValidity *after_last_valid_frame_; MultiFrame &multiframe_; }; From d0e1d86df37220af3d6a2114dcd0792e19d700c4 Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 16:57:09 +0100 Subject: [PATCH 44/51] Remove unused param --- jba.txt | 0 src/query/v2/multiframe.hpp | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) create mode 100644 jba.txt diff --git a/jba.txt b/jba.txt new file mode 100644 index 000000000..e69de29bb diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index c1bf9cdf1..b92eebf8d 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -107,7 +107,7 @@ class ValidFramesReader { using pointer = value_type *; using reference = const Frame &; - explicit Iterator(FrameWithValidity *ptr, ValidFramesReader &iterator_wrapper) : ptr_(ptr) {} + explicit Iterator(FrameWithValidity *ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } From be3797e0a1c451597495970d53468ca5d28145de Mon Sep 17 00:00:00 2001 From: jeremy Date: Thu, 1 Dec 2022 17:01:08 +0100 Subject: [PATCH 45/51] Remove unused param --- src/query/v2/multiframe.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index c007baf7c..26cbb7f10 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -89,8 +89,8 @@ ValidFramesReader::ValidFramesReader(MultiFrame &multiframe) : multiframe_(multi after_last_valid_frame_ = multiframe_.frames_.data() + std::distance(multiframe.frames_.begin(), it); } -ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator{&multiframe_.frames_[0], *this}; } -ValidFramesReader::Iterator ValidFramesReader::end() { return Iterator{after_last_valid_frame_, *this}; } +ValidFramesReader::Iterator ValidFramesReader::begin() { return Iterator{&multiframe_.frames_[0]}; } +ValidFramesReader::Iterator ValidFramesReader::end() { return Iterator{after_last_valid_frame_}; } ValidFramesModifier::ValidFramesModifier(MultiFrame &multiframe) : multiframe_(multiframe) {} From 6b8a5fd41dc1c9d511df4e87d80643b976b889cd Mon Sep 17 00:00:00 2001 From: Kostas Kyrimis Date: Fri, 2 Dec 2022 15:11:51 +0200 Subject: [PATCH 46/51] Make all variants of multiframe iterators model ForwardIterator concept properly --- src/query/v2/multiframe.cpp | 13 ++--- src/query/v2/multiframe.hpp | 96 ++++++++++++++++++++++++++----------- 2 files changed, 71 insertions(+), 38 deletions(-) diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp index 26cbb7f10..4829addb2 100644 --- a/src/query/v2/multiframe.cpp +++ b/src/query/v2/multiframe.cpp @@ -19,15 +19,10 @@ namespace memgraph::query::v2 { -// #NoCommit uncomment https://github.com/memgraph/memgraph/pull/676#discussion_r1035704661 -// static_assert(std::forward_iterator && -// std::equality_comparable); -// static_assert(std::forward_iterator && -// std::equality_comparable); -// static_assert(std::forward_iterator && -// std::equality_comparable); -// static_assert(std::forward_iterator && -// std::equality_comparable); +static_assert(std::forward_iterator); +static_assert(std::forward_iterator); +static_assert(std::forward_iterator); +static_assert(std::forward_iterator); MultiFrame::MultiFrame(int64_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory) : frames_(utils::pmr::vector( diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index b92eebf8d..a73ca1f41 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -107,22 +107,31 @@ class ValidFramesReader { using pointer = value_type *; using reference = const Frame &; + Iterator() {} explicit Iterator(FrameWithValidity *ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } - // Prefix increment Iterator &operator++() { ptr_++; return *this; } - friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; - friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment + // temporaries, e.g, (it++)++ + // NOLINTNEXTLINE (cert-dcl21-cpp) + Iterator operator++(int) & { + auto old = *this; + ptr_++; + return old; + } + + friend bool operator==(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ == rhs.ptr_; }; + friend bool operator!=(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ != rhs.ptr_; }; private: - FrameWithValidity *ptr_; + FrameWithValidity *ptr_{nullptr}; }; Iterator begin(); @@ -138,10 +147,10 @@ class ValidFramesModifier { explicit ValidFramesModifier(MultiFrame &multiframe); ~ValidFramesModifier() = default; - ValidFramesModifier(const ValidFramesModifier &other) = delete; // copy constructor - ValidFramesModifier(ValidFramesModifier &&other) noexcept = delete; // move constructor - ValidFramesModifier &operator=(const ValidFramesModifier &other) = delete; // copy assignment - ValidFramesModifier &operator=(ValidFramesModifier &&other) noexcept = delete; // move assignment + ValidFramesModifier(const ValidFramesModifier &other) = delete; + ValidFramesModifier(ValidFramesModifier &&other) noexcept = delete; + ValidFramesModifier &operator=(const ValidFramesModifier &other) = delete; + ValidFramesModifier &operator=(ValidFramesModifier &&other) noexcept = delete; struct Iterator { using iterator_category = std::forward_iterator_tag; @@ -149,6 +158,7 @@ class ValidFramesModifier { using value_type = Frame; using pointer = value_type *; using reference = Frame &; + Iterator() {} Iterator(FrameWithValidity *ptr, ValidFramesModifier &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} @@ -165,12 +175,21 @@ class ValidFramesModifier { return *this; } - friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; - friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment + // temporaries, e.g, (it++)++ + // NOLINTNEXTLINE (cert-dcl21-cpp) + Iterator operator++(int) & { + auto old = *this; + ++*this; + return old; + } + + friend bool operator==(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ == rhs.ptr_; }; + friend bool operator!=(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ != rhs.ptr_; }; private: - FrameWithValidity *ptr_; - ValidFramesModifier *iterator_wrapper_; + FrameWithValidity *ptr_{nullptr}; + ValidFramesModifier *iterator_wrapper_{nullptr}; }; Iterator begin(); @@ -185,10 +204,10 @@ class ValidFramesConsumer { explicit ValidFramesConsumer(MultiFrame &multiframe); ~ValidFramesConsumer() noexcept; - ValidFramesConsumer(const ValidFramesConsumer &other) = delete; // copy constructor - ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete; // move constructor - ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete; // copy assignment - ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete; // move assignment + ValidFramesConsumer(const ValidFramesConsumer &other) = delete; + ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete; + ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete; + ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete; struct Iterator { using iterator_category = std::forward_iterator_tag; @@ -197,13 +216,14 @@ class ValidFramesConsumer { using pointer = value_type *; using reference = FrameWithValidity &; + Iterator() {} + Iterator(FrameWithValidity *ptr, ValidFramesConsumer &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } - // Prefix increment Iterator &operator++() { do { ptr_++; @@ -212,12 +232,21 @@ class ValidFramesConsumer { return *this; } - friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; - friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment + // temporaries, e.g, (it++)++ + // NOLINTNEXTLINE (cert-dcl21-cpp) + Iterator operator++(int) & { + auto old = *this; + ++*this; + return old; + } + + friend bool operator==(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ == rhs.ptr_; }; + friend bool operator!=(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ != rhs.ptr_; }; private: - FrameWithValidity *ptr_; - ValidFramesConsumer *iterator_wrapper_; + FrameWithValidity *ptr_{nullptr}; + ValidFramesConsumer *iterator_wrapper_{nullptr}; }; Iterator begin(); @@ -232,10 +261,10 @@ class InvalidFramesPopulator { explicit InvalidFramesPopulator(MultiFrame &multiframe); ~InvalidFramesPopulator() = default; - InvalidFramesPopulator(const InvalidFramesPopulator &other) = delete; // copy constructor - InvalidFramesPopulator(InvalidFramesPopulator &&other) noexcept = delete; // move constructor - InvalidFramesPopulator &operator=(const InvalidFramesPopulator &other) = delete; // copy assignment - InvalidFramesPopulator &operator=(InvalidFramesPopulator &&other) noexcept = delete; // move assignment + InvalidFramesPopulator(const InvalidFramesPopulator &other) = delete; + InvalidFramesPopulator(InvalidFramesPopulator &&other) noexcept = delete; + InvalidFramesPopulator &operator=(const InvalidFramesPopulator &other) = delete; + InvalidFramesPopulator &operator=(InvalidFramesPopulator &&other) noexcept = delete; struct Iterator { using iterator_category = std::forward_iterator_tag; @@ -244,23 +273,32 @@ class InvalidFramesPopulator { using pointer = value_type *; using reference = FrameWithValidity &; + Iterator() {} explicit Iterator(FrameWithValidity *ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } pointer operator->() { return ptr_; } - // Prefix increment Iterator &operator++() { ptr_->MakeValid(); ptr_++; return *this; } - friend bool operator==(const Iterator &a, const Iterator &b) { return a.ptr_ == b.ptr_; }; - friend bool operator!=(const Iterator &a, const Iterator &b) { return a.ptr_ != b.ptr_; }; + // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment + // temporaries, e.g, (it++)++ + // NOLINTNEXTLINE (cert-dcl21-cpp) + Iterator operator++(int) & { + auto old = *this; + ++ptr_; + return old; + } + + friend bool operator==(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ == rhs.ptr_; }; + friend bool operator!=(const Iterator &lhs, const Iterator &rhs) { return lhs.ptr_ != rhs.ptr_; }; private: - FrameWithValidity *ptr_; + FrameWithValidity *ptr_{nullptr}; }; Iterator begin(); From 68ae729b07ba4c8818b47278f876eed2db043af4 Mon Sep 17 00:00:00 2001 From: Jeremy B <97525434+42jeremy@users.noreply.github.com> Date: Mon, 5 Dec 2022 10:34:58 +0100 Subject: [PATCH 47/51] Update src/query/v2/multiframe.hpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/query/v2/multiframe.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index a73ca1f41..304a71cdb 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -107,7 +107,7 @@ class ValidFramesReader { using pointer = value_type *; using reference = const Frame &; - Iterator() {} + Iterator() = default; explicit Iterator(FrameWithValidity *ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } From c7c0234889cf807bd17c69cf4c28f4c8e7ffce0a Mon Sep 17 00:00:00 2001 From: jeremy Date: Mon, 5 Dec 2022 10:38:01 +0100 Subject: [PATCH 48/51] Add default constructor to iterators --- src/query/v2/multiframe.hpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index a73ca1f41..2c57c8853 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -107,7 +107,7 @@ class ValidFramesReader { using pointer = value_type *; using reference = const Frame &; - Iterator() {} + Iterator() = default; explicit Iterator(FrameWithValidity *ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } @@ -158,8 +158,8 @@ class ValidFramesModifier { using value_type = Frame; using pointer = value_type *; using reference = Frame &; - Iterator() {} + Iterator() = default; Iterator(FrameWithValidity *ptr, ValidFramesModifier &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} @@ -216,8 +216,7 @@ class ValidFramesConsumer { using pointer = value_type *; using reference = FrameWithValidity &; - Iterator() {} - + Iterator() = default; Iterator(FrameWithValidity *ptr, ValidFramesConsumer &iterator_wrapper) : ptr_(ptr), iterator_wrapper_(&iterator_wrapper) {} @@ -273,7 +272,7 @@ class InvalidFramesPopulator { using pointer = value_type *; using reference = FrameWithValidity &; - Iterator() {} + Iterator() = default; explicit Iterator(FrameWithValidity *ptr) : ptr_(ptr) {} reference operator*() const { return *ptr_; } From f4428af210306d7edf32eb8bd2c728e09c619fb3 Mon Sep 17 00:00:00 2001 From: jeremy Date: Mon, 5 Dec 2022 11:05:47 +0100 Subject: [PATCH 49/51] Remove reference function on operator++() & --- src/query/v2/multiframe.hpp | 24 ++++++++---------------- 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp index 2c57c8853..0b6896422 100644 --- a/src/query/v2/multiframe.hpp +++ b/src/query/v2/multiframe.hpp @@ -118,10 +118,8 @@ class ValidFramesReader { return *this; } - // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment - // temporaries, e.g, (it++)++ - // NOLINTNEXTLINE (cert-dcl21-cpp) - Iterator operator++(int) & { + // NOLINTNEXTLINE(cert-dcl21-cpp) + Iterator operator++(int) { auto old = *this; ptr_++; return old; @@ -175,10 +173,8 @@ class ValidFramesModifier { return *this; } - // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment - // temporaries, e.g, (it++)++ - // NOLINTNEXTLINE (cert-dcl21-cpp) - Iterator operator++(int) & { + // NOLINTNEXTLINE(cert-dcl21-cpp) + Iterator operator++(int) { auto old = *this; ++*this; return old; @@ -231,10 +227,8 @@ class ValidFramesConsumer { return *this; } - // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment - // temporaries, e.g, (it++)++ - // NOLINTNEXTLINE (cert-dcl21-cpp) - Iterator operator++(int) & { + // NOLINTNEXTLINE(cert-dcl21-cpp) + Iterator operator++(int) { auto old = *this; ++*this; return old; @@ -284,10 +278,8 @@ class InvalidFramesPopulator { return *this; } - // clang-tidy warning is wrong here, because we & qualify the function, meaning that you can't post increment - // temporaries, e.g, (it++)++ - // NOLINTNEXTLINE (cert-dcl21-cpp) - Iterator operator++(int) & { + // NOLINTNEXTLINE(cert-dcl21-cpp) + Iterator operator++(int) { auto old = *this; ++ptr_; return old; From 7f9eceadb333476d28801580296a92411219b960 Mon Sep 17 00:00:00 2001 From: jeremy Date: Mon, 5 Dec 2022 11:19:47 +0100 Subject: [PATCH 50/51] Remove un-needed frame modification in Once This is not needed and would be incorrect with the optional --- src/query/v2/plan/operator.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp index 9a000face..eeb5cd6b4 100644 --- a/src/query/v2/plan/operator.cpp +++ b/src/query/v2/plan/operator.cpp @@ -269,11 +269,7 @@ void Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &c if (!did_pull_) { auto &first_frame = multi_frame.GetFirstFrame(); - auto *memory_resource = first_frame.GetMemoryResource(); first_frame.MakeValid(); - for (auto &value : first_frame.elems()) { - value = TypedValue{memory_resource}; - } did_pull_ = true; } } From ec73ee666c02c32a7853f7c60d284b18917921ed Mon Sep 17 00:00:00 2001 From: Kostas Kyrimis Date: Mon, 5 Dec 2022 13:02:33 +0200 Subject: [PATCH 51/51] Remove unused jba.txt --- jba.txt | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 jba.txt diff --git a/jba.txt b/jba.txt deleted file mode 100644 index e69de29bb..000000000