From cfba24692368b217a7ad14f3e0b83c83226cfc52 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 2 Oct 2020 13:48:27 +0100 Subject: [PATCH 001/168] Add EventsExecutor --- rclcpp/CMakeLists.txt | 1 + .../rclcpp/executors/events_executor.hpp | 153 +++++++++++++ .../static_executor_entities_collector.hpp | 60 +++++ .../include/rclcpp/executors/timers_heap.hpp | 209 ++++++++++++++++++ .../rclcpp/experimental/executable_list.hpp | 16 ++ rclcpp/src/rclcpp/executable_list.cpp | 48 ++++ .../src/rclcpp/executors/events_executor.cpp | 188 ++++++++++++++++ .../static_executor_entities_collector.cpp | 89 ++++++++ 8 files changed, 764 insertions(+) create mode 100644 rclcpp/include/rclcpp/executors/events_executor.hpp create mode 100644 rclcpp/include/rclcpp/executors/timers_heap.hpp create mode 100644 rclcpp/src/rclcpp/executors/events_executor.cpp diff --git a/rclcpp/CMakeLists.txt b/rclcpp/CMakeLists.txt index 26d7e69362..15cb37c729 100644 --- a/rclcpp/CMakeLists.txt +++ b/rclcpp/CMakeLists.txt @@ -50,6 +50,7 @@ set(${PROJECT_NAME}_SRCS src/rclcpp/executor.cpp src/rclcpp/executors.cpp src/rclcpp/expand_topic_or_service_name.cpp + src/rclcpp/executors/events_executor.cpp src/rclcpp/executors/multi_threaded_executor.cpp src/rclcpp/executors/single_threaded_executor.cpp src/rclcpp/executors/static_executor_entities_collector.cpp diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp new file mode 100644 index 0000000000..c4fa6f9093 --- /dev/null +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -0,0 +1,153 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ +#define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ + +#include + +#include "rclcpp/executor.hpp" +#include "rclcpp/executors/static_executor_entities_collector.hpp" +#include "rclcpp/executors/timers_heap.hpp" +#include "rclcpp/experimental/executable_list.hpp" +#include "rclcpp/node.hpp" + +#include "rcutils/event_types.h" + +namespace rclcpp +{ +namespace executors +{ + +/// Events executor implementation +/** + * Add description + * + * To run this executor: + * rclcpp::executors::EventsExecutor executor; + * executor.add_node(node); + * executor.spin(); + * executor.remove_node(node); + */ +class EventsExecutor : public rclcpp::Executor +{ +public: + RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutor) + + /// Default constructor. See the default constructor for Executor. + RCLCPP_PUBLIC + explicit EventsExecutor( + const rclcpp::ExecutorOptions & options = rclcpp::ExecutorOptions()); + + /// Default destrcutor. + RCLCPP_PUBLIC + virtual ~EventsExecutor(); + + /// Events executor implementation of spin. + /** + * This function will block until work comes in, execute it, and keep blocking. + * It will only be interrupted by a CTRL-C (managed by the global signal handler). + * \throws std::runtime_error when spin() called while already spinning + */ + RCLCPP_PUBLIC + void + spin() override; + + /// Add a node to the executor. + /** + * \sa rclcpp::Executor::add_node + */ + RCLCPP_PUBLIC + void + add_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + bool notify = true) override; + + /// Convenience function which takes Node and forwards NodeBaseInterface. + /** + * \sa rclcpp::EventsExecutor::add_node + */ + RCLCPP_PUBLIC + void + add_node(std::shared_ptr node_ptr, bool notify = true) override; + + /// Remove a node from the executor. + /** + * \sa rclcpp::Executor::remove_node + */ + RCLCPP_PUBLIC + void + remove_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + bool notify = true) override; + + /// Convenience function which takes Node and forwards NodeBaseInterface. + /** + * \sa rclcpp::Executor::remove_node + */ + RCLCPP_PUBLIC + void + remove_node(std::shared_ptr node_ptr, bool notify = true) override; + +protected: + /// Execute timers when ready + RCLCPP_PUBLIC + void + execute_timers(); + + /// Execute events in the queue until is empty + RCLCPP_PUBLIC + void + execute_events(); + +private: + RCLCPP_DISABLE_COPY(EventsExecutor) + + StaticExecutorEntitiesCollector::SharedPtr entities_collector_; + + // Executor callback: Push new events into the queue and trigger cv. + // This function is called by the DDS entities when an event happened, + // like a subscription receiving a message. + static void + push_event(void * executor_ptr, EventQ event) + { + auto this_executor = static_cast(executor_ptr); + + { + std::unique_lock lock(this_executor->event_queue_mutex_); + + this_executor->event_queue.push(event); + } + // Notify that the event queue has some events in it. + this_executor->event_queue_cv.notify_one(); + } + + // Event queue members + std::queue event_queue; + std::mutex event_queue_mutex_; + std::condition_variable event_queue_cv; + + // Executable list mutex + std::mutex exec_list_mutex_; + + // Timers heap members + // TODO: Finish this to not hardcode that magic number + TimersHeap<30> timers; + std::mutex m_; +}; + +} // namespace executors +} // namespace rclcpp + +#endif // RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index b00d29c774..04fda2fa45 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -67,6 +67,21 @@ class StaticExecutorEntitiesCollector final rclcpp::memory_strategy::MemoryStrategy::SharedPtr & memory_strategy, rcl_guard_condition_t * executor_guard_condition); + /// Initialize StaticExecutorEntitiesCollector for EventsExecutor + /** + * \param executor_guard_condition executor's guard condition + * \param executor_context executor's pointer + * \param executor_callback executor's callback to place event in queue + * \param exec_list_mutex mutex to protect executable list + */ + RCLCPP_PUBLIC + void + init_events_executor( + rcl_guard_condition_t * executor_guard_condition, + void * executor_context, + Event_callback executor_callback, + std::mutex * exec_list_mutex); + RCLCPP_PUBLIC void execute() override; @@ -266,6 +281,42 @@ class StaticExecutorEntitiesCollector final rclcpp::Waitable::SharedPtr get_waitable(size_t i) {return exec_list_.waitable[i];} + /** Return a Subscription Sharedptr by handle + * \param[in] handle The handle of the Subscription + * \return a Subscription shared pointer + * \throws ???? if the Subscription is not found. + */ + RCLCPP_PUBLIC + rclcpp::SubscriptionBase::SharedPtr + get_subscription_by_handle(void * handle); + + /** Return a Service Sharedptr by handle + * \param[in] handle The handle of the Service + * \return a Service shared pointer + * \throws ???? if the Service is not found. + */ + RCLCPP_PUBLIC + rclcpp::ServiceBase::SharedPtr + get_service_by_handle(void * handle); + + /** Return a Client Sharedptr by handle + * \param[in] handle The handle of the Client + * \return a Client shared pointer + * \throws ???? if the Client is not found. + */ + RCLCPP_PUBLIC + rclcpp::ClientBase::SharedPtr + get_client_by_handle(void * handle); + + /** Return a Waitable Sharedptr by handle + * \param[in] handle The handle of the Waitable + * \return a Waitable shared pointer + * \throws ???? if the Waitable is not found. + */ + RCLCPP_PUBLIC + rclcpp::Waitable::SharedPtr + get_waitable_by_handle(void * handle); + private: /// Function to reallocate space for entities in the wait set. /** @@ -324,6 +375,15 @@ class StaticExecutorEntitiesCollector final /// Executable list: timers, subscribers, clients, services and waitables rclcpp::experimental::ExecutableList exec_list_; + + /// Context (associated executor) + void * executor_context_ = nullptr; + + /// Event callback: push new events to queue + Event_callback executor_callback_; + + /// Mutex to protect the executable list + std::mutex * exec_list_mutex_; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/timers_heap.hpp b/rclcpp/include/rclcpp/executors/timers_heap.hpp new file mode 100644 index 0000000000..89632d6f8e --- /dev/null +++ b/rclcpp/include/rclcpp/executors/timers_heap.hpp @@ -0,0 +1,209 @@ +#include +#include + +#include +#include + +using namespace std::chrono_literals; + +namespace rclcpp +{ +namespace executors +{ + +template +struct TimersHeap +{ +public: + /** + * @brief Construct a new Timers Heap object + */ + TimersHeap() + { + clock = rclcpp::Clock(RCL_ROS_TIME); + size = 0; + } + + /** + * @brief Adds a new TimerBase to the heap + * @param timer the timer to be added + * @return int 0 if success, -1 if the heap is already full + */ + inline int add_timer(rclcpp::TimerBase::SharedPtr timer) + { + if (size == MAX_SIZE) { + // The heap is full, can't push + return -1; + } + + // Create a TimerInternal object to store the timer + timers_storage[size] = TimerInternal(timer, clock); + TimerInternalPtr t = &(timers_storage[size]); + + // Push a TimerInternalPtr into the heap + push(t); + return 0; + } + + /** + * @brief Get the time before the first timer in the heap expires + * + * @return std::chrono::nanoseconds to wait, 0 if the timer is already expired + */ + inline std::chrono::nanoseconds get_head_timeout() + { + auto min_timeout = std::chrono::nanoseconds::max(); + if (peek(&head) == 0) { + min_timeout = std::chrono::nanoseconds(head->expire_time - clock.now().nanoseconds()); + } + + if (min_timeout < 0ns) { + min_timeout = 0ns; + } + return min_timeout; + } + + /** + * @brief Executes all the ready timers in the heap + * These timers are refreshed and added back to the heap + * NOTE: may block indefinitely if the time for processing callbacks is longer than the timers period + */ + inline void execute_ready_timers() + { + while (peek(&head) == 0 && head->timer->is_ready()) { + head->timer->execute_callback(); + + head->refresh(clock); + remove_at(0); + push(head); + } + } + +private: + struct TimerInternal + { + inline TimerInternal() + { + timer = nullptr; + expire_time = INT64_MAX; + } + + inline TimerInternal(rclcpp::TimerBase::SharedPtr t, rclcpp::Clock& clock) + { + timer = t; + refresh(clock); + } + + inline void refresh(rclcpp::Clock& clock) + { + expire_time = clock.now().nanoseconds() + timer->time_until_trigger().count(); + } + + rclcpp::TimerBase::SharedPtr timer; + int64_t expire_time; + }; + + using TimerInternalPtr = TimerInternal*; + + inline void push(TimerInternalPtr x) + { + size_t i = size++; + a[i] = x; + while (i && (x->expire_time < a[(i-1)/2]->expire_time)) { + a[i] = a[(i-1)/2]; + a[(i-1)/2] = x; + i = (i-1)/2; + } + } + + inline void remove_at(size_t i) + { + TimerInternalPtr y = a[--size]; + a[i] = y; + + // Heapify upwards. + while (i > 0) { + size_t parent = (i-1)/2; + if (y->expire_time < a[parent]->expire_time) { + a[i] = a[parent]; + a[parent] = y; + i = parent; + } else { + break; + } + } + + // Heapify downwards + while (2*i + 1 < size) { + size_t hi = i; + size_t left = 2*i+1; + size_t right = left + 1; + if (y->expire_time > a[left]->expire_time) { + hi = left; + } + if (right < size && (a[hi]->expire_time > a[right]->expire_time)) { + hi = right; + } + if (hi != i) { + a[i] = a[hi]; + a[hi] = y; + i = hi; + } else { + break; + } + } + } + + inline int pop(TimerInternalPtr x) + { + if (size == 0) { + // The heap is empty, can't pop + return -1; + } + + x = a[0]; + remove_at(0); + return 0; + } + + inline int peek(TimerInternalPtr* x) + { + if (size == 0) { + // The heap is empty, can't peek + return -1; + } + + *x = a[0]; + return 0; + } + + inline int remove(TimerInternalPtr x) + { + size_t i; + for (i = 0; i < size; ++i) { + if (x == a[i]) { + break; + } + } + if (i == size) { + return -1; + } + + remove_at(i); + return 0; + } + + // Array to keep ownership of the timers + std::array timers_storage; + // Array of pointers to stored timers used to implement the priority queue + std::array a; + // Helper to access first element in the heap + TimerInternalPtr head; + // Current number of elements in the heap + size_t size; + // Clock to update expiration times and generate timeouts + rclcpp::Clock clock; +}; + +} +} diff --git a/rclcpp/include/rclcpp/experimental/executable_list.hpp b/rclcpp/include/rclcpp/experimental/executable_list.hpp index e1c70db3de..f209810c86 100644 --- a/rclcpp/include/rclcpp/experimental/executable_list.hpp +++ b/rclcpp/include/rclcpp/experimental/executable_list.hpp @@ -64,6 +64,22 @@ class ExecutableList final void add_waitable(rclcpp::Waitable::SharedPtr waitable); + RCLCPP_PUBLIC + rclcpp::SubscriptionBase::SharedPtr + get_subscription(void * subscription_handle); + + RCLCPP_PUBLIC + rclcpp::ServiceBase::SharedPtr + get_service(void * service_handle); + + RCLCPP_PUBLIC + rclcpp::ClientBase::SharedPtr + get_client(void * client_handle); + + RCLCPP_PUBLIC + rclcpp::Waitable::SharedPtr + get_waitable(void * waitable_handle); + // Vector containing the SubscriptionBase of all the subscriptions added to the executor. std::vector subscription; // Contains the count of added subscriptions diff --git a/rclcpp/src/rclcpp/executable_list.cpp b/rclcpp/src/rclcpp/executable_list.cpp index 07edbc9586..af187acf3d 100644 --- a/rclcpp/src/rclcpp/executable_list.cpp +++ b/rclcpp/src/rclcpp/executable_list.cpp @@ -82,3 +82,51 @@ ExecutableList::add_waitable(rclcpp::Waitable::SharedPtr waitable) this->waitable.push_back(std::move(waitable)); this->number_of_waitables++; } + +rclcpp::SubscriptionBase::SharedPtr +ExecutableList::get_subscription(void * subscription_handler) +{ + for(const auto& sub : subscription) { + if (sub.get() == subscription_handler) { + return sub; + } + } + throw std::runtime_error("Couldn't get subscription!"); + return nullptr; +} + +rclcpp::ServiceBase::SharedPtr +ExecutableList::get_service(void * service_handler) +{ + for(const auto& srv : service) { + if (srv.get() == service_handler) { + return srv; + } + } + throw std::runtime_error("Couldn't get service!"); + return nullptr; +} + +rclcpp::ClientBase::SharedPtr +ExecutableList::get_client(void * client_handler) +{ + for(const auto& cli : client) { + if (cli.get() == client_handler) { + return cli; + } + } + throw std::runtime_error("Couldn't get client!"); + return nullptr; +} + +rclcpp::Waitable::SharedPtr +ExecutableList::get_waitable(void * waitable_handle) +{ + for(const auto& w : waitable) { + if (w.get() == waitable_handle) { + return w; + } + } + throw std::runtime_error("Couldn't get waitable!"); + return nullptr; +} \ No newline at end of file diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp new file mode 100644 index 0000000000..2b55e755f1 --- /dev/null +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -0,0 +1,188 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "rclcpp/executors/events_executor.hpp" + +#include +#include + +#include "rclcpp/scope_exit.hpp" + +using rclcpp::executors::EventsExecutor; +using rclcpp::experimental::ExecutableList; + +EventsExecutor::EventsExecutor( + const rclcpp::ExecutorOptions & options) +: rclcpp::Executor(options) +{ + entities_collector_ = std::make_shared(); +} + +EventsExecutor::~EventsExecutor() {} + +void +EventsExecutor::spin() +{ + if (spinning.exchange(true)) { + throw std::runtime_error("spin() called while already spinning"); + } + RCLCPP_SCOPE_EXIT(this->spinning.store(false); ); + + // Init entities collector + entities_collector_->init_events_executor( + &interrupt_guard_condition_, + this, + &EventsExecutor::push_event, + &exec_list_mutex_); + + std::thread t_exec_timers(&EventsExecutor::execute_timers, this); + pthread_setname_np(t_exec_timers.native_handle(), "Timers"); + + while(spinning.load()) + { + execute_events(); + } + + t_exec_timers.join(); +} + +void +EventsExecutor::add_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) +{ + bool is_new_node = entities_collector_->add_node(node_ptr); + if (is_new_node && notify) { + // Interrupt waiting to handle new node + if (rcl_trigger_guard_condition(&interrupt_guard_condition_) != RCL_RET_OK) { + throw std::runtime_error(rcl_get_error_string().str); + } + } + + // Add timers to timers heap + for (auto & weak_group : node_ptr->get_callback_groups()) { + auto group = weak_group.lock(); + if (!group || !group->can_be_taken_from().load()) { + continue; + } + group->find_timer_ptrs_if( + [this](const rclcpp::TimerBase::SharedPtr & timer) { + if (timer) { + timers.add_timer(timer); + } + return false; + }); + } +} + +void +EventsExecutor::add_node(std::shared_ptr node_ptr, bool notify) +{ + this->add_node(node_ptr->get_node_base_interface(), notify); +} + +void +EventsExecutor::remove_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) +{ + bool node_removed = entities_collector_->remove_node(node_ptr); + + if (notify) { + // If the node was matched and removed, interrupt waiting + if (node_removed) { + if (rcl_trigger_guard_condition(&interrupt_guard_condition_) != RCL_RET_OK) { + throw std::runtime_error(rcl_get_error_string().str); + } + } + } + + std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); + has_executor.store(false); + + // Remove the timers from the timers heap here? +} + +void +EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) +{ + this->remove_node(node_ptr->get_node_base_interface(), notify); +} + +void +EventsExecutor::execute_timers() +{ + while (rclcpp::ok(this->context_) && spinning.load()) + { + std::this_thread::sleep_for(timers.get_head_timeout()); + timers.execute_ready_timers(); + } +} + +void +EventsExecutor::execute_events() +{ + // When condition variable is notified, check this predicate to proceed + auto predicate = [this]() { return !event_queue.empty(); }; + + // Local event queue + std::queue local_event_queue; + + // Scope block for the mutex + { + std::unique_lock lock(event_queue_mutex_); + // We wait here until something has been pushed to the event queue + event_queue_cv.wait(lock, predicate); + + // We got an event! Swap queues and execute events + swap(local_event_queue, event_queue); + } + + // Mutex to protect the executable list from being + // cleared while we still have events to process + std::unique_lock lock(exec_list_mutex_); + + // Execute events + do { + EventQ event = local_event_queue.front(); + + local_event_queue.pop(); + + switch(event.type) + { + case SUBSCRIPTION_EVENT: + { + execute_subscription(std::move(entities_collector_->get_subscription_by_handle(event.entity))); + break; + } + + case SERVICE_EVENT: + { + execute_service(std::move(entities_collector_->get_service_by_handle(event.entity))); + break; + } + + case CLIENT_EVENT: + { + execute_client(std::move(entities_collector_->get_client_by_handle(event.entity))); + break; + } + + case GUARD_CONDITION_EVENT: + { + entities_collector_->get_waitable_by_handle(event.entity)->execute(); + break; + } + + } + } while (!local_event_queue.empty()); +} diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index 0b81c10069..091844a454 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -58,6 +58,37 @@ StaticExecutorEntitiesCollector::~StaticExecutorEntitiesCollector() weak_nodes_to_guard_conditions_.clear(); } +void +StaticExecutorEntitiesCollector::init_events_executor( + rcl_guard_condition_t * executor_guard_condition, + void * executor_context, + Event_callback executor_callback, + std::mutex * exec_list_mutex) +{ + // Empty initialize executable list + exec_list_ = rclcpp::experimental::ExecutableList(); + + // Add executor's guard condition + // memory_strategy_->add_guard_condition(executor_guard_condition); + + // Set context (associated executor) + executor_context_ = executor_context; + + // Set executor callback to push events into the event queue + executor_callback_ = executor_callback; + + // Init event handle : The handler is the waitabe pointer (this) + // So if any guard condition attached to this waitable is triggered, + // we execute the waitable + // event_handle_ = {executor_context_, static_cast(this), executor_callback_}; + + // Init executable list mutex + exec_list_mutex_ = exec_list_mutex; + + // Fill executable list + fill_executable_list(); +} + void StaticExecutorEntitiesCollector::init( rcl_wait_set_t * p_wait_set, @@ -139,6 +170,10 @@ StaticExecutorEntitiesCollector::fill_memory_strategy() void StaticExecutorEntitiesCollector::fill_executable_list() { + // Mutex to avoid clearing the executable list if we are + // in the middle of events processing in the events queue + std::unique_lock lk(*exec_list_mutex_); + exec_list_.clear(); add_callback_groups_from_nodes_associated_to_executor(); fill_executable_list_from_map(weak_groups_associated_with_executor_to_nodes_); @@ -167,6 +202,16 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { exec_list_.add_subscription(subscription); + + rcl_ret_t ret = rcl_set_subscription_callback( + executor_context_, + executor_callback_, + subscription.get(), + subscription->get_subscription_handle().get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set subscription callback")); + } } return false; }); @@ -174,6 +219,16 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { exec_list_.add_service(service); + + rcl_ret_t ret = rcl_set_service_callback( + executor_context_, + executor_callback_, + service.get(), + service->get_service_handle().get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set service callback")); + } } return false; }); @@ -181,6 +236,16 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { exec_list_.add_client(client); + + rcl_ret_t ret = rcl_set_client_callback( + executor_context_, + executor_callback_, + client.get(), + client->get_client_handle().get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set client callback")); + } } return false; }); @@ -498,3 +563,27 @@ StaticExecutorEntitiesCollector::get_automatically_added_callback_groups_from_no } return groups; } + +rclcpp::SubscriptionBase::SharedPtr +StaticExecutorEntitiesCollector::get_subscription_by_handle(void * handle) +{ + return exec_list_.get_subscription(handle); +} + +rclcpp::ServiceBase::SharedPtr +StaticExecutorEntitiesCollector::get_service_by_handle(void * handle) +{ + return exec_list_.get_service(handle); +} + +rclcpp::ClientBase::SharedPtr +StaticExecutorEntitiesCollector::get_client_by_handle(void * handle) +{ + return exec_list_.get_client(handle); +} + +rclcpp::Waitable::SharedPtr +StaticExecutorEntitiesCollector::get_waitable_by_handle(void * handle) +{ + return exec_list_.get_waitable(handle); +} From e773a0bf6885218ab1148883d5a9875b843ada89 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 2 Oct 2020 16:30:26 +0100 Subject: [PATCH 002/168] Handle intra-process communication --- .../rclcpp/executors/events_executor.hpp | 1 + .../static_executor_entities_collector.hpp | 8 ++++- .../subscription_intra_process_base.hpp | 7 ++++ rclcpp/include/rclcpp/waitable.hpp | 8 +++++ .../src/rclcpp/executors/events_executor.cpp | 1 - .../static_executor_entities_collector.cpp | 35 ++++++++++++++----- .../subscription_intra_process_base.cpp | 17 +++++++++ rclcpp/src/rclcpp/waitable.cpp | 11 ++++++ 8 files changed, 77 insertions(+), 11 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index c4fa6f9093..64a98f6306 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -143,6 +143,7 @@ class EventsExecutor : public rclcpp::Executor // Timers heap members // TODO: Finish this to not hardcode that magic number + // Also the timers heap vector could be part of the executable list TimersHeap<30> timers; std::mutex m_; }; diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index 04fda2fa45..6053df6229 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -77,7 +77,6 @@ class StaticExecutorEntitiesCollector final RCLCPP_PUBLIC void init_events_executor( - rcl_guard_condition_t * executor_guard_condition, void * executor_context, Event_callback executor_callback, std::mutex * exec_list_mutex); @@ -103,6 +102,13 @@ class StaticExecutorEntitiesCollector final bool add_to_wait_set(rcl_wait_set_t * wait_set) override; + RCLCPP_PUBLIC + void + set_guard_condition_callback( + void * executor_context, + Event_callback executor_callback, + void * waitable_handle) const override; + RCLCPP_PUBLIC size_t get_number_of_ready_guard_conditions() override; diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 7afd68abef..f41ff9c57e 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -70,6 +70,13 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable rmw_qos_profile_t get_actual_qos() const; + RCLCPP_PUBLIC + void + set_guard_condition_callback( + void * executor_context, + Event_callback executor_callback, + void * waitable_handle) const override; + protected: std::recursive_mutex reentrant_mutex_; rcl_guard_condition_t gc_; diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 64504f3950..5b67925ed9 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -165,6 +165,14 @@ class Waitable bool exchange_in_use_by_wait_set_state(bool in_use_state); + RCLCPP_PUBLIC + virtual + void + set_guard_condition_callback( + void * executor_context, + Event_callback executor_callback, + void * waitable_handle) const; + private: std::atomic in_use_by_wait_set_{false}; }; // class Waitable diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 2b55e755f1..0e4b9db8bb 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -41,7 +41,6 @@ EventsExecutor::spin() // Init entities collector entities_collector_->init_events_executor( - &interrupt_guard_condition_, this, &EventsExecutor::push_event, &exec_list_mutex_); diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index 091844a454..0a67bd3340 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -60,7 +60,6 @@ StaticExecutorEntitiesCollector::~StaticExecutorEntitiesCollector() void StaticExecutorEntitiesCollector::init_events_executor( - rcl_guard_condition_t * executor_guard_condition, void * executor_context, Event_callback executor_callback, std::mutex * exec_list_mutex) @@ -68,20 +67,12 @@ StaticExecutorEntitiesCollector::init_events_executor( // Empty initialize executable list exec_list_ = rclcpp::experimental::ExecutableList(); - // Add executor's guard condition - // memory_strategy_->add_guard_condition(executor_guard_condition); - // Set context (associated executor) executor_context_ = executor_context; // Set executor callback to push events into the event queue executor_callback_ = executor_callback; - // Init event handle : The handler is the waitabe pointer (this) - // So if any guard condition attached to this waitable is triggered, - // we execute the waitable - // event_handle_ = {executor_context_, static_cast(this), executor_callback_}; - // Init executable list mutex exec_list_mutex_ = exec_list_mutex; @@ -253,12 +244,38 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { exec_list_.add_waitable(waitable); + + waitable->set_guard_condition_callback( + executor_context_, + executor_callback_, + waitable.get()); } return false; }); } } +void +StaticExecutorEntitiesCollector::set_guard_condition_callback( + void * executor_context, + Event_callback executor_callback, + void * waitable_handle) const +{ + // Set waitable guard conditions' callback (one for each registered node) + for (const auto & pair : weak_nodes_to_guard_conditions_) { + auto & gc = pair.second; + rcl_ret_t ret = rcl_set_guard_condition_callback( + executor_context, + executor_callback, + waitable_handle, + gc); + + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } + } +} + void StaticExecutorEntitiesCollector::prepare_wait_set() { diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 3b951f34de..8e1d12daf8 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -36,3 +36,20 @@ SubscriptionIntraProcessBase::get_actual_qos() const { return qos_profile_; } + +void +SubscriptionIntraProcessBase::set_guard_condition_callback( + void * executor_context, + Event_callback executor_callback, + void * waitable_handle) const +{ + rcl_ret_t ret = rcl_set_guard_condition_callback( + executor_context, + executor_callback, + waitable_handle, + &gc_); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } +} \ No newline at end of file diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index b76c7215e0..a41f01e959 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -57,3 +57,14 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) { return in_use_by_wait_set_.exchange(in_use_state); } + +void +Waitable::set_guard_condition_callback( + void * executor_context, + Event_callback executor_callback, + void * waitable_handle) const +{ + (void)executor_context; + (void)executor_callback; + (void)waitable_handle; +} From 09c555374564fb9caf5bf8d0f2d0cc5f06157141 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 2 Oct 2020 18:33:58 +0100 Subject: [PATCH 003/168] Move apis - Add constness --- rclcpp/include/rclcpp/client.hpp | 4 ++ .../rclcpp/executors/events_executor.hpp | 6 ++- .../static_executor_entities_collector.hpp | 11 ++-- .../rclcpp/experimental/executable_list.hpp | 8 +-- .../subscription_intra_process_base.hpp | 3 +- rclcpp/include/rclcpp/service.hpp | 4 ++ rclcpp/include/rclcpp/subscription_base.hpp | 4 ++ rclcpp/include/rclcpp/waitable.hpp | 3 +- rclcpp/src/rclcpp/client.cpp | 16 ++++++ rclcpp/src/rclcpp/executable_list.cpp | 8 +-- .../static_executor_entities_collector.cpp | 54 ++++--------------- rclcpp/src/rclcpp/service.cpp | 16 ++++++ rclcpp/src/rclcpp/subscription_base.cpp | 16 ++++++ .../subscription_intra_process_base.cpp | 7 ++- rclcpp/src/rclcpp/waitable.cpp | 4 +- 15 files changed, 94 insertions(+), 70 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index bd2d326012..ab6ccf91db 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -150,6 +150,10 @@ class ClientBase bool exchange_in_use_by_wait_set_state(bool in_use_state); + RCLCPP_PUBLIC + void + set_callback(const void * executor_context, Event_callback executor_callback) const; + protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 64a98f6306..849def128c 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -120,9 +120,11 @@ class EventsExecutor : public rclcpp::Executor // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. static void - push_event(void * executor_ptr, EventQ event) + push_event(const void * executor_ptr, EventQ event) { - auto this_executor = static_cast(executor_ptr); + // Cast executor_ptr to this + auto this_executor = const_cast( + static_cast(executor_ptr)); { std::unique_lock lock(this_executor->event_queue_mutex_); diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index 6053df6229..640ffd1521 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -106,8 +106,7 @@ class StaticExecutorEntitiesCollector final void set_guard_condition_callback( void * executor_context, - Event_callback executor_callback, - void * waitable_handle) const override; + Event_callback executor_callback) const override; RCLCPP_PUBLIC size_t @@ -294,7 +293,7 @@ class StaticExecutorEntitiesCollector final */ RCLCPP_PUBLIC rclcpp::SubscriptionBase::SharedPtr - get_subscription_by_handle(void * handle); + get_subscription_by_handle(const void * handle); /** Return a Service Sharedptr by handle * \param[in] handle The handle of the Service @@ -303,7 +302,7 @@ class StaticExecutorEntitiesCollector final */ RCLCPP_PUBLIC rclcpp::ServiceBase::SharedPtr - get_service_by_handle(void * handle); + get_service_by_handle(const void * handle); /** Return a Client Sharedptr by handle * \param[in] handle The handle of the Client @@ -312,7 +311,7 @@ class StaticExecutorEntitiesCollector final */ RCLCPP_PUBLIC rclcpp::ClientBase::SharedPtr - get_client_by_handle(void * handle); + get_client_by_handle(const void * handle); /** Return a Waitable Sharedptr by handle * \param[in] handle The handle of the Waitable @@ -321,7 +320,7 @@ class StaticExecutorEntitiesCollector final */ RCLCPP_PUBLIC rclcpp::Waitable::SharedPtr - get_waitable_by_handle(void * handle); + get_waitable_by_handle(const void * handle); private: /// Function to reallocate space for entities in the wait set. diff --git a/rclcpp/include/rclcpp/experimental/executable_list.hpp b/rclcpp/include/rclcpp/experimental/executable_list.hpp index f209810c86..16608a9fdf 100644 --- a/rclcpp/include/rclcpp/experimental/executable_list.hpp +++ b/rclcpp/include/rclcpp/experimental/executable_list.hpp @@ -66,19 +66,19 @@ class ExecutableList final RCLCPP_PUBLIC rclcpp::SubscriptionBase::SharedPtr - get_subscription(void * subscription_handle); + get_subscription(const void * subscription_handle); RCLCPP_PUBLIC rclcpp::ServiceBase::SharedPtr - get_service(void * service_handle); + get_service(const void * service_handle); RCLCPP_PUBLIC rclcpp::ClientBase::SharedPtr - get_client(void * client_handle); + get_client(const void * client_handle); RCLCPP_PUBLIC rclcpp::Waitable::SharedPtr - get_waitable(void * waitable_handle); + get_waitable(const void * waitable_handle); // Vector containing the SubscriptionBase of all the subscriptions added to the executor. std::vector subscription; diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index f41ff9c57e..e75fe5d65c 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -74,8 +74,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable void set_guard_condition_callback( void * executor_context, - Event_callback executor_callback, - void * waitable_handle) const override; + Event_callback executor_callback) const override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 5c7411892d..fc0ebe6f8e 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -121,6 +121,10 @@ class ServiceBase bool exchange_in_use_by_wait_set_state(bool in_use_state); + RCLCPP_PUBLIC + void + set_callback(const void * executor_context, Event_callback executor_callback) const; + protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 28b92ffb13..8e82f430d2 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -263,6 +263,10 @@ class SubscriptionBase : public std::enable_shared_from_this bool exchange_in_use_by_wait_set_state(void * pointer_to_subscription_part, bool in_use_state); + RCLCPP_PUBLIC + void + set_callback(const void * executor_context, Event_callback executor_callback) const; + protected: template void diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 5b67925ed9..e39e01b294 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -170,8 +170,7 @@ class Waitable void set_guard_condition_callback( void * executor_context, - Event_callback executor_callback, - void * waitable_handle) const; + Event_callback executor_callback) const; private: std::atomic in_use_by_wait_set_{false}; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 6a1d3934d4..adae1e250a 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -198,3 +198,19 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) { return in_use_by_wait_set_.exchange(in_use_state); } + +void +ClientBase::set_callback( + const void * executor_context, + Event_callback executor_callback) const +{ + rcl_ret_t ret = rcl_client_set_callback( + executor_context, + executor_callback, + this, + client_handle_.get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set client callback")); + } +} diff --git a/rclcpp/src/rclcpp/executable_list.cpp b/rclcpp/src/rclcpp/executable_list.cpp index af187acf3d..32c694c7e8 100644 --- a/rclcpp/src/rclcpp/executable_list.cpp +++ b/rclcpp/src/rclcpp/executable_list.cpp @@ -84,7 +84,7 @@ ExecutableList::add_waitable(rclcpp::Waitable::SharedPtr waitable) } rclcpp::SubscriptionBase::SharedPtr -ExecutableList::get_subscription(void * subscription_handler) +ExecutableList::get_subscription(const void * subscription_handler) { for(const auto& sub : subscription) { if (sub.get() == subscription_handler) { @@ -96,7 +96,7 @@ ExecutableList::get_subscription(void * subscription_handler) } rclcpp::ServiceBase::SharedPtr -ExecutableList::get_service(void * service_handler) +ExecutableList::get_service(const void * service_handler) { for(const auto& srv : service) { if (srv.get() == service_handler) { @@ -108,7 +108,7 @@ ExecutableList::get_service(void * service_handler) } rclcpp::ClientBase::SharedPtr -ExecutableList::get_client(void * client_handler) +ExecutableList::get_client(const void * client_handler) { for(const auto& cli : client) { if (cli.get() == client_handler) { @@ -120,7 +120,7 @@ ExecutableList::get_client(void * client_handler) } rclcpp::Waitable::SharedPtr -ExecutableList::get_waitable(void * waitable_handle) +ExecutableList::get_waitable(const void * waitable_handle) { for(const auto& w : waitable) { if (w.get() == waitable_handle) { diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index 0a67bd3340..535d7be545 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -193,16 +193,7 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { exec_list_.add_subscription(subscription); - - rcl_ret_t ret = rcl_set_subscription_callback( - executor_context_, - executor_callback_, - subscription.get(), - subscription->get_subscription_handle().get()); - - if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set subscription callback")); - } + subscription->set_callback(executor_context_, executor_callback_); } return false; }); @@ -210,16 +201,7 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { exec_list_.add_service(service); - - rcl_ret_t ret = rcl_set_service_callback( - executor_context_, - executor_callback_, - service.get(), - service->get_service_handle().get()); - - if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set service callback")); - } + service->set_callback(executor_context_, executor_callback_); } return false; }); @@ -227,16 +209,7 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { exec_list_.add_client(client); - - rcl_ret_t ret = rcl_set_client_callback( - executor_context_, - executor_callback_, - client.get(), - client->get_client_handle().get()); - - if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set client callback")); - } + client->set_callback(executor_context_, executor_callback_); } return false; }); @@ -244,11 +217,7 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { exec_list_.add_waitable(waitable); - - waitable->set_guard_condition_callback( - executor_context_, - executor_callback_, - waitable.get()); + waitable->set_guard_condition_callback(executor_context_, executor_callback_); } return false; }); @@ -258,16 +227,15 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( void StaticExecutorEntitiesCollector::set_guard_condition_callback( void * executor_context, - Event_callback executor_callback, - void * waitable_handle) const + Event_callback executor_callback) const { // Set waitable guard conditions' callback (one for each registered node) for (const auto & pair : weak_nodes_to_guard_conditions_) { auto & gc = pair.second; - rcl_ret_t ret = rcl_set_guard_condition_callback( + rcl_ret_t ret = rcl_guard_condition_set_callback( executor_context, executor_callback, - waitable_handle, + this, gc); if (ret != RCL_RET_OK) { @@ -582,25 +550,25 @@ StaticExecutorEntitiesCollector::get_automatically_added_callback_groups_from_no } rclcpp::SubscriptionBase::SharedPtr -StaticExecutorEntitiesCollector::get_subscription_by_handle(void * handle) +StaticExecutorEntitiesCollector::get_subscription_by_handle(const void * handle) { return exec_list_.get_subscription(handle); } rclcpp::ServiceBase::SharedPtr -StaticExecutorEntitiesCollector::get_service_by_handle(void * handle) +StaticExecutorEntitiesCollector::get_service_by_handle(const void * handle) { return exec_list_.get_service(handle); } rclcpp::ClientBase::SharedPtr -StaticExecutorEntitiesCollector::get_client_by_handle(void * handle) +StaticExecutorEntitiesCollector::get_client_by_handle(const void * handle) { return exec_list_.get_client(handle); } rclcpp::Waitable::SharedPtr -StaticExecutorEntitiesCollector::get_waitable_by_handle(void * handle) +StaticExecutorEntitiesCollector::get_waitable_by_handle(const void * handle) { return exec_list_.get_waitable(handle); } diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 827062bdd3..3f1302c875 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -84,3 +84,19 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) { return in_use_by_wait_set_.exchange(in_use_state); } + +void +ServiceBase::set_callback( + const void * executor_context, + Event_callback executor_callback) const +{ + rcl_ret_t ret = rcl_service_set_callback( + executor_context, + executor_callback, + this, + service_handle_.get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set service callback")); + } +} diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 824d0f2f85..d9c72812a3 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -288,3 +288,19 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( } throw std::runtime_error("given pointer_to_subscription_part does not match any part"); } + +void +SubscriptionBase::set_callback( + const void * executor_context, + Event_callback executor_callback) const +{ + rcl_ret_t ret = rcl_subscription_set_callback( + executor_context, + executor_callback, + this, + subscription_handle_.get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set subscription callback")); + } +} diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 8e1d12daf8..529f5bb8de 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -40,13 +40,12 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_guard_condition_callback( void * executor_context, - Event_callback executor_callback, - void * waitable_handle) const + Event_callback executor_callback) const { - rcl_ret_t ret = rcl_set_guard_condition_callback( + rcl_ret_t ret = rcl_guard_condition_set_callback( executor_context, executor_callback, - waitable_handle, + this, &gc_); if (RCL_RET_OK != ret) { diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index a41f01e959..75c5744868 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -61,10 +61,8 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_guard_condition_callback( void * executor_context, - Event_callback executor_callback, - void * waitable_handle) const + Event_callback executor_callback) const { (void)executor_context; (void)executor_callback; - (void)waitable_handle; } From e7ddba8ed018996c323fdc3989e17a0c9048ac14 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 11:19:22 +0100 Subject: [PATCH 004/168] Create raw pointer version of Executor::execute_ --- rclcpp/include/rclcpp/executor.hpp | 8 ++ .../rclcpp/executors/events_executor.hpp | 19 ++- rclcpp/src/rclcpp/executor.cpp | 3 +- .../src/rclcpp/executors/events_executor.cpp | 114 +++++++++++++++++- 4 files changed, 135 insertions(+), 9 deletions(-) diff --git a/rclcpp/include/rclcpp/executor.hpp b/rclcpp/include/rclcpp/executor.hpp index 4af227192b..f16d1e00b4 100644 --- a/rclcpp/include/rclcpp/executor.hpp +++ b/rclcpp/include/rclcpp/executor.hpp @@ -420,6 +420,14 @@ class Executor void execute_any_executable(AnyExecutable & any_exec); + RCLCPP_PUBLIC + static void + take_and_do_error_handling( + const char * action_description, + const char * topic_or_service_name, + std::function take_action, + std::function handle_action); + RCLCPP_PUBLIC static void execute_subscription( diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 849def128c..528a81f236 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -100,6 +100,18 @@ class EventsExecutor : public rclcpp::Executor void remove_node(std::shared_ptr node_ptr, bool notify = true) override; + RCLCPP_PUBLIC + void + execute_subscription(rclcpp::SubscriptionBase* subscription); + + RCLCPP_PUBLIC + void + execute_service(rclcpp::ServiceBase* service); + + RCLCPP_PUBLIC + void + execute_client(rclcpp::ClientBase* client); + protected: /// Execute timers when ready RCLCPP_PUBLIC @@ -122,10 +134,11 @@ class EventsExecutor : public rclcpp::Executor static void push_event(const void * executor_ptr, EventQ event) { - // Cast executor_ptr to this - auto this_executor = const_cast( - static_cast(executor_ptr)); + // Cast executor_ptr to this (need to remove constness) + auto this_executor = const_cast( + static_cast(executor_ptr)); + // Event queue mutex scope { std::unique_lock lock(this_executor->event_queue_mutex_); diff --git a/rclcpp/src/rclcpp/executor.cpp b/rclcpp/src/rclcpp/executor.cpp index 659a3034d7..3204187b10 100644 --- a/rclcpp/src/rclcpp/executor.cpp +++ b/rclcpp/src/rclcpp/executor.cpp @@ -518,9 +518,8 @@ Executor::execute_any_executable(AnyExecutable & any_exec) } } -static void -take_and_do_error_handling( +Executor::take_and_do_error_handling( const char * action_description, const char * topic_or_service_name, std::function take_action, diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 0e4b9db8bb..7fc9d3b2e6 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -160,28 +160,134 @@ EventsExecutor::execute_events() { case SUBSCRIPTION_EVENT: { - execute_subscription(std::move(entities_collector_->get_subscription_by_handle(event.entity))); + //execute_subscription(std::move(entities_collector_->get_subscription_by_handle(event.entity))); + auto subscription = const_cast( + static_cast(event.entity)); + execute_subscription(subscription); break; } case SERVICE_EVENT: { - execute_service(std::move(entities_collector_->get_service_by_handle(event.entity))); + //execute_service(std::move(entities_collector_->get_service_by_handle(event.entity))); + auto service = const_cast( + static_cast(event.entity)); + execute_service(service); break; } case CLIENT_EVENT: { - execute_client(std::move(entities_collector_->get_client_by_handle(event.entity))); + //execute_client(std::move(entities_collector_->get_client_by_handle(event.entity))); + auto client = const_cast( + static_cast(event.entity)); + execute_client(client); break; } case GUARD_CONDITION_EVENT: { - entities_collector_->get_waitable_by_handle(event.entity)->execute(); + //entities_collector_->get_waitable_by_handle(event.entity)->execute(); + auto waitable = const_cast( + static_cast(event.entity)); + waitable->execute(); break; } } } while (!local_event_queue.empty()); } + + +// Raw pointer versions of Executor::execute_ +void +EventsExecutor::execute_subscription(rclcpp::SubscriptionBase* subscription) +{ + rclcpp::MessageInfo message_info; + message_info.get_rmw_message_info().from_intra_process = false; + + if (subscription->is_serialized()) { + // This is the case where a copy of the serialized message is taken from + // the middleware via inter-process communication. + std::shared_ptr serialized_msg = subscription->create_serialized_message(); + take_and_do_error_handling( + "taking a serialized message from topic", + subscription->get_topic_name(), + [&]() {return subscription->take_serialized(*serialized_msg.get(), message_info);}, + [&]() + { + auto void_serialized_msg = std::static_pointer_cast(serialized_msg); + subscription->handle_message(void_serialized_msg, message_info); + }); + subscription->return_serialized_message(serialized_msg); + } else if (subscription->can_loan_messages()) { + // This is the case where a loaned message is taken from the middleware via + // inter-process communication, given to the user for their callback, + // and then returned. + void * loaned_msg = nullptr; + // TODO(wjwwood): refactor this into methods on subscription when LoanedMessage + // is extened to support subscriptions as well. + take_and_do_error_handling( + "taking a loaned message from topic", + subscription->get_topic_name(), + [&]() + { + rcl_ret_t ret = rcl_take_loaned_message( + subscription->get_subscription_handle().get(), + &loaned_msg, + &message_info.get_rmw_message_info(), + nullptr); + if (RCL_RET_SUBSCRIPTION_TAKE_FAILED == ret) { + return false; + } else if (RCL_RET_OK != ret) { + rclcpp::exceptions::throw_from_rcl_error(ret); + } + return true; + }, + [&]() {subscription->handle_loaned_message(loaned_msg, message_info);}); + rcl_ret_t ret = rcl_return_loaned_message_from_subscription( + subscription->get_subscription_handle().get(), + loaned_msg); + if (RCL_RET_OK != ret) { + RCLCPP_ERROR( + rclcpp::get_logger("rclcpp"), + "rcl_return_loaned_message_from_subscription() failed for subscription on topic '%s': %s", + subscription->get_topic_name(), rcl_get_error_string().str); + } + loaned_msg = nullptr; + } else { + // This case is taking a copy of the message data from the middleware via + // inter-process communication. + std::shared_ptr message = subscription->create_message(); + take_and_do_error_handling( + "taking a message from topic", + subscription->get_topic_name(), + [&]() {return subscription->take_type_erased(message.get(), message_info);}, + [&]() {subscription->handle_message(message, message_info);}); + subscription->return_message(message); + } +} + +void +EventsExecutor::execute_service(rclcpp::ServiceBase* service) +{ + auto request_header = service->create_request_header(); + std::shared_ptr request = service->create_request(); + take_and_do_error_handling( + "taking a service server request from service", + service->get_service_name(), + [&]() {return service->take_type_erased_request(request.get(), *request_header);}, + [&]() {service->handle_request(request_header, request);}); +} + +void +EventsExecutor::execute_client(rclcpp::ClientBase* client) +{ + auto request_header = client->create_request_header(); + std::shared_ptr response = client->create_response(); + take_and_do_error_handling( + "taking a service client response from service", + client->get_service_name(), + [&]() {return client->take_type_erased_response(response.get(), *request_header);}, + [&]() {client->handle_response(request_header, response);}); +} From 0696147f7758bd5524f0575ecf8c7472a4469728 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 12:39:06 +0100 Subject: [PATCH 005/168] Remove exec mutex. Add comments --- .../static_executor_entities_collector.hpp | 8 +--- .../src/rclcpp/executors/events_executor.cpp | 46 ++++++++++++++++--- .../static_executor_entities_collector.cpp | 28 +++++++---- 3 files changed, 59 insertions(+), 23 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index 640ffd1521..140b42e36f 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -78,8 +78,7 @@ class StaticExecutorEntitiesCollector final void init_events_executor( void * executor_context, - Event_callback executor_callback, - std::mutex * exec_list_mutex); + Event_callback executor_callback); RCLCPP_PUBLIC void @@ -385,10 +384,7 @@ class StaticExecutorEntitiesCollector final void * executor_context_ = nullptr; /// Event callback: push new events to queue - Event_callback executor_callback_; - - /// Mutex to protect the executable list - std::mutex * exec_list_mutex_; + Event_callback executor_callback_ = nullptr; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 7fc9d3b2e6..514c13d443 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -40,10 +40,7 @@ EventsExecutor::spin() RCLCPP_SCOPE_EXIT(this->spinning.store(false); ); // Init entities collector - entities_collector_->init_events_executor( - this, - &EventsExecutor::push_event, - &exec_list_mutex_); + entities_collector_->init_events_executor(this, &EventsExecutor::push_event); std::thread t_exec_timers(&EventsExecutor::execute_timers, this); pthread_setname_np(t_exec_timers.native_handle(), "Timers"); @@ -61,19 +58,20 @@ EventsExecutor::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { bool is_new_node = entities_collector_->add_node(node_ptr); + if (is_new_node && notify) { - // Interrupt waiting to handle new node + // Interrupt waiting to handle new node: Define what to do with this. if (rcl_trigger_guard_condition(&interrupt_guard_condition_) != RCL_RET_OK) { throw std::runtime_error(rcl_get_error_string().str); } } - // Add timers to timers heap for (auto & weak_group : node_ptr->get_callback_groups()) { auto group = weak_group.lock(); if (!group || !group->can_be_taken_from().load()) { continue; } + // Add timers to timers heap group->find_timer_ptrs_if( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { @@ -81,7 +79,41 @@ EventsExecutor::add_node( } return false; }); + // Set the callbacks to all the entities + group->find_subscription_ptrs_if( + [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { + if (subscription) { + subscription->set_callback(this, &EventsExecutor::push_event); + } + return false; + }); + group->find_service_ptrs_if( + [this](const rclcpp::ServiceBase::SharedPtr & service) { + if (service) { + service->set_callback(this, &EventsExecutor::push_event); + } + return false; + }); + group->find_client_ptrs_if( + [this](const rclcpp::ClientBase::SharedPtr & client) { + if (client) { + client->set_callback(this, &EventsExecutor::push_event); + } + return false; + }); + group->find_waitable_ptrs_if( + [this](const rclcpp::Waitable::SharedPtr & waitable) { + if (waitable) { + waitable->set_guard_condition_callback(this, &EventsExecutor::push_event); + } + return false; + }); } + + // We should add the node's guard condition to the entities collector waitable, + // so if a new entitity is added to the node while spinning, its guard condition is triggered + // so the entitites collector (waitable) gets an event on the queue, which should + // set that new entity' callback } void @@ -108,7 +140,7 @@ EventsExecutor::remove_node( std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); has_executor.store(false); - // Remove the timers from the timers heap here? + // Remove the timers from the timers heap here and unset entities callback } void diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index 535d7be545..ccc338336d 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -61,10 +61,10 @@ StaticExecutorEntitiesCollector::~StaticExecutorEntitiesCollector() void StaticExecutorEntitiesCollector::init_events_executor( void * executor_context, - Event_callback executor_callback, - std::mutex * exec_list_mutex) + Event_callback executor_callback) { // Empty initialize executable list + // Why do I need it for the new EventsExecutor approach? exec_list_ = rclcpp::experimental::ExecutableList(); // Set context (associated executor) @@ -73,11 +73,12 @@ StaticExecutorEntitiesCollector::init_events_executor( // Set executor callback to push events into the event queue executor_callback_ = executor_callback; - // Init executable list mutex - exec_list_mutex_ = exec_list_mutex; + // Now that we have all nodes registered we can set the nodes + // guard condition callback + set_guard_condition_callback(executor_context_, executor_callback_); - // Fill executable list - fill_executable_list(); + // If we are already spinning this won't be called, but we have to set + // add the new node's guard condition callback } void @@ -102,6 +103,17 @@ StaticExecutorEntitiesCollector::init( execute(); } +// The purpose of "execute" is handling the situation of a new entity added to +// a node, while the executor is already spinning. +// With the new approach, "execute" should only take care of setting that +// entitiy's callback. +// If a entity is removed from a node, should we unset its callback? +// Maybe worth it create a new EventsExecutorEntitiesCollector?? +// because now we'd be doing unnecessary thinks like: +// fill_memory_strategy, prepare_wait_set,fill_executable_list +// Now we only use fill_executable_list because it can set the callbacks +// to entities, but it's setting the ones already set so we shoud do it +// more efficently void StaticExecutorEntitiesCollector::execute() { @@ -161,10 +173,6 @@ StaticExecutorEntitiesCollector::fill_memory_strategy() void StaticExecutorEntitiesCollector::fill_executable_list() { - // Mutex to avoid clearing the executable list if we are - // in the middle of events processing in the events queue - std::unique_lock lk(*exec_list_mutex_); - exec_list_.clear(); add_callback_groups_from_nodes_associated_to_executor(); fill_executable_list_from_map(weak_groups_associated_with_executor_to_nodes_); From f2a7c29098a7fa3025f560f92f963faacb7912e3 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 12:47:30 +0100 Subject: [PATCH 006/168] remove not needed APIs --- .../static_executor_entities_collector.hpp | 36 -------------- .../rclcpp/experimental/executable_list.hpp | 16 ------- rclcpp/src/rclcpp/executable_list.cpp | 48 ------------------- .../static_executor_entities_collector.cpp | 24 ---------- 4 files changed, 124 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index 140b42e36f..234f4f6b23 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -285,42 +285,6 @@ class StaticExecutorEntitiesCollector final rclcpp::Waitable::SharedPtr get_waitable(size_t i) {return exec_list_.waitable[i];} - /** Return a Subscription Sharedptr by handle - * \param[in] handle The handle of the Subscription - * \return a Subscription shared pointer - * \throws ???? if the Subscription is not found. - */ - RCLCPP_PUBLIC - rclcpp::SubscriptionBase::SharedPtr - get_subscription_by_handle(const void * handle); - - /** Return a Service Sharedptr by handle - * \param[in] handle The handle of the Service - * \return a Service shared pointer - * \throws ???? if the Service is not found. - */ - RCLCPP_PUBLIC - rclcpp::ServiceBase::SharedPtr - get_service_by_handle(const void * handle); - - /** Return a Client Sharedptr by handle - * \param[in] handle The handle of the Client - * \return a Client shared pointer - * \throws ???? if the Client is not found. - */ - RCLCPP_PUBLIC - rclcpp::ClientBase::SharedPtr - get_client_by_handle(const void * handle); - - /** Return a Waitable Sharedptr by handle - * \param[in] handle The handle of the Waitable - * \return a Waitable shared pointer - * \throws ???? if the Waitable is not found. - */ - RCLCPP_PUBLIC - rclcpp::Waitable::SharedPtr - get_waitable_by_handle(const void * handle); - private: /// Function to reallocate space for entities in the wait set. /** diff --git a/rclcpp/include/rclcpp/experimental/executable_list.hpp b/rclcpp/include/rclcpp/experimental/executable_list.hpp index 16608a9fdf..e1c70db3de 100644 --- a/rclcpp/include/rclcpp/experimental/executable_list.hpp +++ b/rclcpp/include/rclcpp/experimental/executable_list.hpp @@ -64,22 +64,6 @@ class ExecutableList final void add_waitable(rclcpp::Waitable::SharedPtr waitable); - RCLCPP_PUBLIC - rclcpp::SubscriptionBase::SharedPtr - get_subscription(const void * subscription_handle); - - RCLCPP_PUBLIC - rclcpp::ServiceBase::SharedPtr - get_service(const void * service_handle); - - RCLCPP_PUBLIC - rclcpp::ClientBase::SharedPtr - get_client(const void * client_handle); - - RCLCPP_PUBLIC - rclcpp::Waitable::SharedPtr - get_waitable(const void * waitable_handle); - // Vector containing the SubscriptionBase of all the subscriptions added to the executor. std::vector subscription; // Contains the count of added subscriptions diff --git a/rclcpp/src/rclcpp/executable_list.cpp b/rclcpp/src/rclcpp/executable_list.cpp index 32c694c7e8..07edbc9586 100644 --- a/rclcpp/src/rclcpp/executable_list.cpp +++ b/rclcpp/src/rclcpp/executable_list.cpp @@ -82,51 +82,3 @@ ExecutableList::add_waitable(rclcpp::Waitable::SharedPtr waitable) this->waitable.push_back(std::move(waitable)); this->number_of_waitables++; } - -rclcpp::SubscriptionBase::SharedPtr -ExecutableList::get_subscription(const void * subscription_handler) -{ - for(const auto& sub : subscription) { - if (sub.get() == subscription_handler) { - return sub; - } - } - throw std::runtime_error("Couldn't get subscription!"); - return nullptr; -} - -rclcpp::ServiceBase::SharedPtr -ExecutableList::get_service(const void * service_handler) -{ - for(const auto& srv : service) { - if (srv.get() == service_handler) { - return srv; - } - } - throw std::runtime_error("Couldn't get service!"); - return nullptr; -} - -rclcpp::ClientBase::SharedPtr -ExecutableList::get_client(const void * client_handler) -{ - for(const auto& cli : client) { - if (cli.get() == client_handler) { - return cli; - } - } - throw std::runtime_error("Couldn't get client!"); - return nullptr; -} - -rclcpp::Waitable::SharedPtr -ExecutableList::get_waitable(const void * waitable_handle) -{ - for(const auto& w : waitable) { - if (w.get() == waitable_handle) { - return w; - } - } - throw std::runtime_error("Couldn't get waitable!"); - return nullptr; -} \ No newline at end of file diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index ccc338336d..423d54f0bd 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -556,27 +556,3 @@ StaticExecutorEntitiesCollector::get_automatically_added_callback_groups_from_no } return groups; } - -rclcpp::SubscriptionBase::SharedPtr -StaticExecutorEntitiesCollector::get_subscription_by_handle(const void * handle) -{ - return exec_list_.get_subscription(handle); -} - -rclcpp::ServiceBase::SharedPtr -StaticExecutorEntitiesCollector::get_service_by_handle(const void * handle) -{ - return exec_list_.get_service(handle); -} - -rclcpp::ClientBase::SharedPtr -StaticExecutorEntitiesCollector::get_client_by_handle(const void * handle) -{ - return exec_list_.get_client(handle); -} - -rclcpp::Waitable::SharedPtr -StaticExecutorEntitiesCollector::get_waitable_by_handle(const void * handle) -{ - return exec_list_.get_waitable(handle); -} From 6de9fd74ab286551ad1866dddd8ddb609a7e9ffa Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 13:12:08 +0100 Subject: [PATCH 007/168] Change how to add node gc's to entities collector --- .../static_executor_entities_collector.hpp | 5 +-- .../src/rclcpp/executors/events_executor.cpp | 8 ++--- .../static_executor_entities_collector.cpp | 32 +++++++------------ 3 files changed, 17 insertions(+), 28 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index 234f4f6b23..2110fe5268 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -103,9 +103,10 @@ class StaticExecutorEntitiesCollector final RCLCPP_PUBLIC void - set_guard_condition_callback( + add_node_gc( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, void * executor_context, - Event_callback executor_callback) const override; + Event_callback executor_callback) const; RCLCPP_PUBLIC size_t diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 514c13d443..074ece3022 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -57,14 +57,10 @@ void EventsExecutor::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { + (void) notify; bool is_new_node = entities_collector_->add_node(node_ptr); - if (is_new_node && notify) { - // Interrupt waiting to handle new node: Define what to do with this. - if (rcl_trigger_guard_condition(&interrupt_guard_condition_) != RCL_RET_OK) { - throw std::runtime_error(rcl_get_error_string().str); - } - } + entities_collector_->add_node_gc(node_ptr, this, &EventsExecutor::push_event); for (auto & weak_group : node_ptr->get_callback_groups()) { auto group = weak_group.lock(); diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index 423d54f0bd..939bedd778 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -72,13 +72,6 @@ StaticExecutorEntitiesCollector::init_events_executor( // Set executor callback to push events into the event queue executor_callback_ = executor_callback; - - // Now that we have all nodes registered we can set the nodes - // guard condition callback - set_guard_condition_callback(executor_context_, executor_callback_); - - // If we are already spinning this won't be called, but we have to set - // add the new node's guard condition callback } void @@ -232,23 +225,22 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( } } + +// Do also remove_node_gc void -StaticExecutorEntitiesCollector::set_guard_condition_callback( +StaticExecutorEntitiesCollector::add_node_gc( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, void * executor_context, Event_callback executor_callback) const { - // Set waitable guard conditions' callback (one for each registered node) - for (const auto & pair : weak_nodes_to_guard_conditions_) { - auto & gc = pair.second; - rcl_ret_t ret = rcl_guard_condition_set_callback( - executor_context, - executor_callback, - this, - gc); - - if (ret != RCL_RET_OK) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } + rcl_ret_t ret = rcl_guard_condition_set_callback( + executor_context, + executor_callback, + this, + node_ptr->get_notify_guard_condition()); + + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); } } From 60351b0d7cac3569c32baa4e538beb3647168fbb Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 13:31:19 +0100 Subject: [PATCH 008/168] Remove comments --- .../static_executor_entities_collector.hpp | 11 +---------- rclcpp/src/rclcpp/executors/events_executor.cpp | 15 ++++----------- .../static_executor_entities_collector.cpp | 6 +++--- 3 files changed, 8 insertions(+), 24 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index 2110fe5268..a1475888c2 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -67,18 +67,9 @@ class StaticExecutorEntitiesCollector final rclcpp::memory_strategy::MemoryStrategy::SharedPtr & memory_strategy, rcl_guard_condition_t * executor_guard_condition); - /// Initialize StaticExecutorEntitiesCollector for EventsExecutor - /** - * \param executor_guard_condition executor's guard condition - * \param executor_context executor's pointer - * \param executor_callback executor's callback to place event in queue - * \param exec_list_mutex mutex to protect executable list - */ RCLCPP_PUBLIC void - init_events_executor( - void * executor_context, - Event_callback executor_callback); + set_callback(void * executor_context, Event_callback executor_callback); RCLCPP_PUBLIC void diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 074ece3022..5198826444 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -40,7 +40,7 @@ EventsExecutor::spin() RCLCPP_SCOPE_EXIT(this->spinning.store(false); ); // Init entities collector - entities_collector_->init_events_executor(this, &EventsExecutor::push_event); + entities_collector_->set_callback(this, &EventsExecutor::push_event); std::thread t_exec_timers(&EventsExecutor::execute_timers, this); pthread_setname_np(t_exec_timers.native_handle(), "Timers"); @@ -58,7 +58,9 @@ EventsExecutor::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { (void) notify; - bool is_new_node = entities_collector_->add_node(node_ptr); + // entities_collector_->add_node(node_ptr); + + std::unique_lock lock(event_queue_mutex_); entities_collector_->add_node_gc(node_ptr, this, &EventsExecutor::push_event); @@ -105,11 +107,6 @@ EventsExecutor::add_node( return false; }); } - - // We should add the node's guard condition to the entities collector waitable, - // so if a new entitity is added to the node while spinning, its guard condition is triggered - // so the entitites collector (waitable) gets an event on the queue, which should - // set that new entity' callback } void @@ -188,7 +185,6 @@ EventsExecutor::execute_events() { case SUBSCRIPTION_EVENT: { - //execute_subscription(std::move(entities_collector_->get_subscription_by_handle(event.entity))); auto subscription = const_cast( static_cast(event.entity)); execute_subscription(subscription); @@ -197,7 +193,6 @@ EventsExecutor::execute_events() case SERVICE_EVENT: { - //execute_service(std::move(entities_collector_->get_service_by_handle(event.entity))); auto service = const_cast( static_cast(event.entity)); execute_service(service); @@ -206,7 +201,6 @@ EventsExecutor::execute_events() case CLIENT_EVENT: { - //execute_client(std::move(entities_collector_->get_client_by_handle(event.entity))); auto client = const_cast( static_cast(event.entity)); execute_client(client); @@ -215,7 +209,6 @@ EventsExecutor::execute_events() case GUARD_CONDITION_EVENT: { - //entities_collector_->get_waitable_by_handle(event.entity)->execute(); auto waitable = const_cast( static_cast(event.entity)); waitable->execute(); diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index 939bedd778..9678ec7b89 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -59,7 +59,7 @@ StaticExecutorEntitiesCollector::~StaticExecutorEntitiesCollector() } void -StaticExecutorEntitiesCollector::init_events_executor( +StaticExecutorEntitiesCollector::set_callback( void * executor_context, Event_callback executor_callback) { @@ -111,11 +111,11 @@ void StaticExecutorEntitiesCollector::execute() { // Fill memory strategy with entities coming from weak_nodes_ - fill_memory_strategy(); + //fill_memory_strategy(); // Fill exec_list_ with entities coming from weak_nodes_ (same as memory strategy) fill_executable_list(); // Resize the wait_set_ based on memory_strategy handles (rcl_wait_set_resize) - prepare_wait_set(); + //prepare_wait_set(); } void From 7da26aca13fabed09ca873010fbe68fd1f754c83 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 14:12:04 +0100 Subject: [PATCH 009/168] Use EventsExecutorEntitiesCollector --- rclcpp/CMakeLists.txt | 1 + .../rclcpp/executors/events_executor.hpp | 4 +- .../events_executor_entities_collector.hpp | 109 ++++++++++++ .../src/rclcpp/executors/events_executor.cpp | 5 +- .../events_executor_entities_collector.cpp | 164 ++++++++++++++++++ 5 files changed, 278 insertions(+), 5 deletions(-) create mode 100644 rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp create mode 100644 rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp diff --git a/rclcpp/CMakeLists.txt b/rclcpp/CMakeLists.txt index 15cb37c729..f236e68511 100644 --- a/rclcpp/CMakeLists.txt +++ b/rclcpp/CMakeLists.txt @@ -51,6 +51,7 @@ set(${PROJECT_NAME}_SRCS src/rclcpp/executors.cpp src/rclcpp/expand_topic_or_service_name.cpp src/rclcpp/executors/events_executor.cpp + src/rclcpp/executors/events_executor_entities_collector.cpp src/rclcpp/executors/multi_threaded_executor.cpp src/rclcpp/executors/single_threaded_executor.cpp src/rclcpp/executors/static_executor_entities_collector.cpp diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 528a81f236..affa508624 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -18,7 +18,7 @@ #include #include "rclcpp/executor.hpp" -#include "rclcpp/executors/static_executor_entities_collector.hpp" +#include "rclcpp/executors/events_executor_entities_collector.hpp" #include "rclcpp/executors/timers_heap.hpp" #include "rclcpp/experimental/executable_list.hpp" #include "rclcpp/node.hpp" @@ -126,7 +126,7 @@ class EventsExecutor : public rclcpp::Executor private: RCLCPP_DISABLE_COPY(EventsExecutor) - StaticExecutorEntitiesCollector::SharedPtr entities_collector_; + EventsExecutorEntitiesCollector::SharedPtr entities_collector_; // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp new file mode 100644 index 0000000000..fdca27ba28 --- /dev/null +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -0,0 +1,109 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ +#define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ + +#include +#include +#include +#include +#include + +#include "rcl/guard_condition.h" +#include "rcl/wait.h" + +#include "rclcpp/experimental/executable_list.hpp" +#include "rclcpp/macros.hpp" +#include "rclcpp/memory_strategy.hpp" +#include "rclcpp/visibility_control.hpp" +#include "rclcpp/waitable.hpp" + +namespace rclcpp +{ +namespace executors +{ +typedef std::map> WeakCallbackGroupsToNodesMap; + +class EventsExecutorEntitiesCollector final + : public rclcpp::Waitable, + public std::enable_shared_from_this +{ +public: + RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorEntitiesCollector) + + // Constructor + RCLCPP_PUBLIC + EventsExecutorEntitiesCollector() = default; + + // Destructor + RCLCPP_PUBLIC + ~EventsExecutorEntitiesCollector(); + + RCLCPP_PUBLIC + void + set_callback(void * executor_context, Event_callback executor_callback); + + RCLCPP_PUBLIC + void + execute() override; + + RCLCPP_PUBLIC + void + add_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + void * executor_context, + Event_callback executor_callback); + + RCLCPP_PUBLIC + bool + remove_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr); + + RCLCPP_PUBLIC + bool + is_ready(rcl_wait_set_t * wait_set) override + { + (void)wait_set; + return false; + } + + RCLCPP_PUBLIC + bool + add_to_wait_set(rcl_wait_set_t * wait_set) override + { + (void)wait_set; + return true; + } + +private: + void + set_entities_callbacks(); + + /// List of weak nodes registered in the static executor + std::list weak_nodes_; + + /// Context (associated executor) + void * executor_context_ = nullptr; + + /// Event callback: push new events to queue + Event_callback executor_callback_ = nullptr; +}; + +} // namespace executors +} // namespace rclcpp + +#endif // RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 5198826444..a2474bfc79 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -26,7 +26,7 @@ EventsExecutor::EventsExecutor( const rclcpp::ExecutorOptions & options) : rclcpp::Executor(options) { - entities_collector_ = std::make_shared(); + entities_collector_ = std::make_shared(); } EventsExecutor::~EventsExecutor() {} @@ -58,11 +58,10 @@ EventsExecutor::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { (void) notify; - // entities_collector_->add_node(node_ptr); std::unique_lock lock(event_queue_mutex_); - entities_collector_->add_node_gc(node_ptr, this, &EventsExecutor::push_event); + entities_collector_->add_node(node_ptr, this, &EventsExecutor::push_event); for (auto & weak_group : node_ptr->get_callback_groups()) { auto group = weak_group.lock(); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp new file mode 100644 index 0000000000..9bb9f9a877 --- /dev/null +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -0,0 +1,164 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "rclcpp/executors/events_executor_entities_collector.hpp" + +#include +#include +#include +#include +#include +#include + +#include "rclcpp/memory_strategy.hpp" +#include "rclcpp/executors/events_executor.hpp" + +using rclcpp::executors::EventsExecutorEntitiesCollector; + +EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() +{ + // Disassociate all nodes + for (const auto & weak_node : weak_nodes_) { + auto node = weak_node.lock(); + if (node) { + std::atomic_bool & has_executor = node->get_associated_with_executor_atomic(); + has_executor.store(false); + } + } + weak_nodes_.clear(); +} + +void +EventsExecutorEntitiesCollector::set_callback( + void * executor_context, Event_callback executor_callback) +{ + executor_context_ = executor_context; + executor_callback_ = executor_callback; +} + +// The purpose of "execute" is handling the situation of a new entity added to +// a node, while the executor is already spinning. +// With the new approach, "execute" should only take care of setting that +// entitiy's callback. +// If a entity is removed from a node, we should unset its callback +void +EventsExecutorEntitiesCollector::execute() +{ + // Fill exec_list_ with entities coming from weak_nodes_ (same as memory strategy) + set_entities_callbacks(); +} + +void +EventsExecutorEntitiesCollector::set_entities_callbacks() +{ + for (auto & weak_node : weak_nodes_) { + auto node = weak_node.lock(); + if (!node) { + continue; + } + // Check in all the callback groups + for (auto & weak_group : node->get_callback_groups()) { + auto group = weak_group.lock(); + if (!group || !group->can_be_taken_from().load()) { + continue; + } + + group->find_timer_ptrs_if( + [this](const rclcpp::TimerBase::SharedPtr & timer) { + if (timer) { + // Add timer!! + } + return false; + }); + group->find_subscription_ptrs_if( + [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { + if (subscription) { + subscription->set_callback(executor_context_, executor_callback_); + } + return false; + }); + group->find_service_ptrs_if( + [this](const rclcpp::ServiceBase::SharedPtr & service) { + if (service) { + service->set_callback(executor_context_, executor_callback_); + } + return false; + }); + group->find_client_ptrs_if( + [this](const rclcpp::ClientBase::SharedPtr & client) { + if (client) { + client->set_callback(executor_context_, executor_callback_); + } + return false; + }); + group->find_waitable_ptrs_if( + [this](const rclcpp::Waitable::SharedPtr & waitable) { + if (waitable) { + waitable->set_guard_condition_callback(executor_context_, executor_callback_); + } + return false; + }); + } + } +} + +void +EventsExecutorEntitiesCollector::add_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + void * executor_context, + Event_callback executor_callback) +{ + // If the node already has an executor + std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); + + if (has_executor.exchange(true)) { + throw std::runtime_error("Node has already been added to an executor."); + } + + weak_nodes_.push_back(node_ptr); + + // Set node's guard condition callback, so if a new entitiy is added while + // spinning we can set its callback. + rcl_ret_t ret = rcl_guard_condition_set_callback( + executor_context, + executor_callback, + this, + node_ptr->get_notify_guard_condition()); + + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } +} + +// Here we should unset the node's guard condition callback. +bool +EventsExecutorEntitiesCollector::remove_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) +{ + auto node_it = weak_nodes_.begin(); + + while (node_it != weak_nodes_.end()) { + bool matched = (node_it->lock() == node_ptr); + if (matched) { + // Find and remove node and unset its guard condition callback (TODO) + // rcl_ret_t ret = rcl_guard_condition_unset_callback( + // node_ptr->get_notify_guard_condition()); + weak_nodes_.erase(node_it); + return true; + } else { + ++node_it; + } + } + return false; +} From 216941715eac42bf4dadef790aeda8918acb0783 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 14:13:26 +0100 Subject: [PATCH 010/168] restore original StaticExecutorEntitiesCollector --- .../static_executor_entities_collector.hpp | 17 ------ .../static_executor_entities_collector.cpp | 54 +------------------ 2 files changed, 2 insertions(+), 69 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index a1475888c2..b00d29c774 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -67,10 +67,6 @@ class StaticExecutorEntitiesCollector final rclcpp::memory_strategy::MemoryStrategy::SharedPtr & memory_strategy, rcl_guard_condition_t * executor_guard_condition); - RCLCPP_PUBLIC - void - set_callback(void * executor_context, Event_callback executor_callback); - RCLCPP_PUBLIC void execute() override; @@ -92,13 +88,6 @@ class StaticExecutorEntitiesCollector final bool add_to_wait_set(rcl_wait_set_t * wait_set) override; - RCLCPP_PUBLIC - void - add_node_gc( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, - void * executor_context, - Event_callback executor_callback) const; - RCLCPP_PUBLIC size_t get_number_of_ready_guard_conditions() override; @@ -335,12 +324,6 @@ class StaticExecutorEntitiesCollector final /// Executable list: timers, subscribers, clients, services and waitables rclcpp::experimental::ExecutableList exec_list_; - - /// Context (associated executor) - void * executor_context_ = nullptr; - - /// Event callback: push new events to queue - Event_callback executor_callback_ = nullptr; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index 9678ec7b89..0b81c10069 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -58,22 +58,6 @@ StaticExecutorEntitiesCollector::~StaticExecutorEntitiesCollector() weak_nodes_to_guard_conditions_.clear(); } -void -StaticExecutorEntitiesCollector::set_callback( - void * executor_context, - Event_callback executor_callback) -{ - // Empty initialize executable list - // Why do I need it for the new EventsExecutor approach? - exec_list_ = rclcpp::experimental::ExecutableList(); - - // Set context (associated executor) - executor_context_ = executor_context; - - // Set executor callback to push events into the event queue - executor_callback_ = executor_callback; -} - void StaticExecutorEntitiesCollector::init( rcl_wait_set_t * p_wait_set, @@ -96,26 +80,15 @@ StaticExecutorEntitiesCollector::init( execute(); } -// The purpose of "execute" is handling the situation of a new entity added to -// a node, while the executor is already spinning. -// With the new approach, "execute" should only take care of setting that -// entitiy's callback. -// If a entity is removed from a node, should we unset its callback? -// Maybe worth it create a new EventsExecutorEntitiesCollector?? -// because now we'd be doing unnecessary thinks like: -// fill_memory_strategy, prepare_wait_set,fill_executable_list -// Now we only use fill_executable_list because it can set the callbacks -// to entities, but it's setting the ones already set so we shoud do it -// more efficently void StaticExecutorEntitiesCollector::execute() { // Fill memory strategy with entities coming from weak_nodes_ - //fill_memory_strategy(); + fill_memory_strategy(); // Fill exec_list_ with entities coming from weak_nodes_ (same as memory strategy) fill_executable_list(); // Resize the wait_set_ based on memory_strategy handles (rcl_wait_set_resize) - //prepare_wait_set(); + prepare_wait_set(); } void @@ -194,7 +167,6 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { exec_list_.add_subscription(subscription); - subscription->set_callback(executor_context_, executor_callback_); } return false; }); @@ -202,7 +174,6 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { exec_list_.add_service(service); - service->set_callback(executor_context_, executor_callback_); } return false; }); @@ -210,7 +181,6 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { exec_list_.add_client(client); - client->set_callback(executor_context_, executor_callback_); } return false; }); @@ -218,32 +188,12 @@ StaticExecutorEntitiesCollector::fill_executable_list_from_map( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { exec_list_.add_waitable(waitable); - waitable->set_guard_condition_callback(executor_context_, executor_callback_); } return false; }); } } - -// Do also remove_node_gc -void -StaticExecutorEntitiesCollector::add_node_gc( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, - void * executor_context, - Event_callback executor_callback) const -{ - rcl_ret_t ret = rcl_guard_condition_set_callback( - executor_context, - executor_callback, - this, - node_ptr->get_notify_guard_condition()); - - if (ret != RCL_RET_OK) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } -} - void StaticExecutorEntitiesCollector::prepare_wait_set() { From ca15daa583a357fc8a1c11ea31857d4887d0a0c0 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 15:20:48 +0100 Subject: [PATCH 011/168] remove includes --- .../include/rclcpp/executors/events_executor.hpp | 4 ---- .../events_executor_entities_collector.hpp | 14 +------------- rclcpp/src/rclcpp/executors/events_executor.cpp | 6 +----- .../events_executor_entities_collector.cpp | 11 +---------- 4 files changed, 3 insertions(+), 32 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index affa508624..093c5b4c82 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -153,14 +153,10 @@ class EventsExecutor : public rclcpp::Executor std::mutex event_queue_mutex_; std::condition_variable event_queue_cv; - // Executable list mutex - std::mutex exec_list_mutex_; - // Timers heap members // TODO: Finish this to not hardcode that magic number // Also the timers heap vector could be part of the executable list TimersHeap<30> timers; - std::mutex m_; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index fdca27ba28..5ae50dc227 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -15,28 +15,16 @@ #ifndef RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ -#include #include #include -#include -#include -#include "rcl/guard_condition.h" -#include "rcl/wait.h" - -#include "rclcpp/experimental/executable_list.hpp" -#include "rclcpp/macros.hpp" -#include "rclcpp/memory_strategy.hpp" -#include "rclcpp/visibility_control.hpp" +#include "rclcpp/node_interfaces/node_base_interface.hpp" #include "rclcpp/waitable.hpp" namespace rclcpp { namespace executors { -typedef std::map> WeakCallbackGroupsToNodesMap; class EventsExecutorEntitiesCollector final : public rclcpp::Waitable, diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index a2474bfc79..043356723a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -132,7 +132,7 @@ EventsExecutor::remove_node( std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); has_executor.store(false); - // Remove the timers from the timers heap here and unset entities callback + // Todo: Remove the timers from the timers heap here and unset entities callback } void @@ -170,10 +170,6 @@ EventsExecutor::execute_events() swap(local_event_queue, event_queue); } - // Mutex to protect the executable list from being - // cleared while we still have events to process - std::unique_lock lock(exec_list_mutex_); - // Execute events do { EventQ event = local_event_queue.front(); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 9bb9f9a877..99442bad53 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -12,17 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "rclcpp/executors/events_executor_entities_collector.hpp" - -#include -#include -#include -#include -#include -#include - -#include "rclcpp/memory_strategy.hpp" #include "rclcpp/executors/events_executor.hpp" +#include "rclcpp/executors/events_executor_entities_collector.hpp" using rclcpp::executors::EventsExecutorEntitiesCollector; From 645c93b21faf26c5f82fd8c87e6fdc3fd168f08c Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 16:04:53 +0100 Subject: [PATCH 012/168] Handle timers from EventsExecutorEntitiesCollector --- .../events_executor_entities_collector.hpp | 12 +++++++++- .../include/rclcpp/executors/timers_heap.hpp | 5 +++++ .../src/rclcpp/executors/events_executor.cpp | 22 ++++++++++++++----- .../events_executor_entities_collector.cpp | 14 +++++++----- 4 files changed, 42 insertions(+), 11 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 5ae50dc227..ebad82e2b7 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -33,6 +33,9 @@ class EventsExecutorEntitiesCollector final public: RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorEntitiesCollector) + using PushTimer = std::function; + using ClearTimers = std::function; + // Constructor RCLCPP_PUBLIC EventsExecutorEntitiesCollector() = default; @@ -43,7 +46,11 @@ class EventsExecutorEntitiesCollector final RCLCPP_PUBLIC void - set_callback(void * executor_context, Event_callback executor_callback); + set_callbacks( + void * executor_context, + Event_callback executor_callback, + PushTimer push_timer, + ClearTimers clear_timers); RCLCPP_PUBLIC void @@ -89,6 +96,9 @@ class EventsExecutorEntitiesCollector final /// Event callback: push new events to queue Event_callback executor_callback_ = nullptr; + + PushTimer push_timer_ = nullptr; + ClearTimers clear_timers_ = nullptr; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/timers_heap.hpp b/rclcpp/include/rclcpp/executors/timers_heap.hpp index 89632d6f8e..3b0e07814a 100644 --- a/rclcpp/include/rclcpp/executors/timers_heap.hpp +++ b/rclcpp/include/rclcpp/executors/timers_heap.hpp @@ -79,6 +79,11 @@ struct TimersHeap } } + inline void clear() + { + // Todo: Implement clear all timers. + } + private: struct TimerInternal { diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 043356723a..475af861fe 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -37,10 +37,22 @@ EventsExecutor::spin() if (spinning.exchange(true)) { throw std::runtime_error("spin() called while already spinning"); } - RCLCPP_SCOPE_EXIT(this->spinning.store(false); ); + RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + + auto push_timer = [this](const rclcpp::TimerBase::SharedPtr & t) { + timers.add_timer(t); + }; + + auto clear_timers = [this]() { + timers.clear(); + }; // Init entities collector - entities_collector_->set_callback(this, &EventsExecutor::push_event); + entities_collector_->set_callbacks( + this, + &EventsExecutor::push_event, + push_timer, + clear_timers); std::thread t_exec_timers(&EventsExecutor::execute_timers, this); pthread_setname_np(t_exec_timers.native_handle(), "Timers"); @@ -72,9 +84,9 @@ EventsExecutor::add_node( group->find_timer_ptrs_if( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { - timers.add_timer(timer); - } - return false; + timers.add_timer(timer); + } + return false; }); // Set the callbacks to all the entities group->find_subscription_ptrs_if( diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 99442bad53..22f7a92070 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -31,11 +31,16 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() } void -EventsExecutorEntitiesCollector::set_callback( - void * executor_context, Event_callback executor_callback) +EventsExecutorEntitiesCollector::set_callbacks( + void * executor_context, + Event_callback executor_callback, + PushTimer push_timer, + ClearTimers clear_timers) { executor_context_ = executor_context; executor_callback_ = executor_callback; + push_timer_ = push_timer; + clear_timers_ = clear_timers; } // The purpose of "execute" is handling the situation of a new entity added to @@ -46,7 +51,7 @@ EventsExecutorEntitiesCollector::set_callback( void EventsExecutorEntitiesCollector::execute() { - // Fill exec_list_ with entities coming from weak_nodes_ (same as memory strategy) + clear_timers_(); set_entities_callbacks(); } @@ -64,11 +69,10 @@ EventsExecutorEntitiesCollector::set_entities_callbacks() if (!group || !group->can_be_taken_from().load()) { continue; } - group->find_timer_ptrs_if( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { - // Add timer!! + push_timer_(timer); } return false; }); From 16e876bcc955a901f1f2a6a403c04ee3a42f1ea8 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 16:55:07 +0100 Subject: [PATCH 013/168] Not ready for rebuild stuff --- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 -- .../rclcpp/executors/events_executor_entities_collector.cpp | 4 ++-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 475af861fe..d5b266fe01 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -71,8 +71,6 @@ EventsExecutor::add_node( { (void) notify; - std::unique_lock lock(event_queue_mutex_); - entities_collector_->add_node(node_ptr, this, &EventsExecutor::push_event); for (auto & weak_group : node_ptr->get_callback_groups()) { diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 22f7a92070..806e259151 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -51,8 +51,8 @@ EventsExecutorEntitiesCollector::set_callbacks( void EventsExecutorEntitiesCollector::execute() { - clear_timers_(); - set_entities_callbacks(); + // clear_timers_(); + // set_entities_callbacks(); } void From db4d2099da07338706c491782e6c12de8d585b91 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 17:24:51 +0100 Subject: [PATCH 014/168] Some cleaning --- .../rclcpp/executors/events_executor.hpp | 2 -- .../events_executor_entities_collector.hpp | 22 +++++++++---------- .../src/rclcpp/executors/events_executor.cpp | 14 ++++-------- .../events_executor_entities_collector.cpp | 4 ++-- 4 files changed, 16 insertions(+), 26 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 093c5b4c82..1b8044426b 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -20,7 +20,6 @@ #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" #include "rclcpp/executors/timers_heap.hpp" -#include "rclcpp/experimental/executable_list.hpp" #include "rclcpp/node.hpp" #include "rcutils/event_types.h" @@ -155,7 +154,6 @@ class EventsExecutor : public rclcpp::Executor // Timers heap members // TODO: Finish this to not hardcode that magic number - // Also the timers heap vector could be part of the executable list TimersHeap<30> timers; }; diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index ebad82e2b7..a332b16e5a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -16,7 +16,6 @@ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ #include -#include #include "rclcpp/node_interfaces/node_base_interface.hpp" #include "rclcpp/waitable.hpp" @@ -26,15 +25,13 @@ namespace rclcpp namespace executors { -class EventsExecutorEntitiesCollector final - : public rclcpp::Waitable, - public std::enable_shared_from_this +class EventsExecutorEntitiesCollector final : public rclcpp::Waitable { public: RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorEntitiesCollector) - using PushTimer = std::function; - using ClearTimers = std::function; + using PushTimerFn = std::function; + using ClearTimersFn = std::function; // Constructor RCLCPP_PUBLIC @@ -49,8 +46,8 @@ class EventsExecutorEntitiesCollector final set_callbacks( void * executor_context, Event_callback executor_callback, - PushTimer push_timer, - ClearTimers clear_timers); + PushTimerFn push_timer, + ClearTimersFn clear_timers); RCLCPP_PUBLIC void @@ -88,17 +85,18 @@ class EventsExecutorEntitiesCollector final void set_entities_callbacks(); - /// List of weak nodes registered in the static executor + /// List of weak nodes registered in the events executor std::list weak_nodes_; /// Context (associated executor) void * executor_context_ = nullptr; - /// Event callback: push new events to queue + /// Events callback Event_callback executor_callback_ = nullptr; - PushTimer push_timer_ = nullptr; - ClearTimers clear_timers_ = nullptr; + /// Function pointers to push and clear timers from the timers heap + PushTimerFn push_timer_ = nullptr; + ClearTimersFn clear_timers_ = nullptr; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index d5b266fe01..674823882e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -14,13 +14,7 @@ #include "rclcpp/executors/events_executor.hpp" -#include -#include - -#include "rclcpp/scope_exit.hpp" - using rclcpp::executors::EventsExecutor; -using rclcpp::experimental::ExecutableList; EventsExecutor::EventsExecutor( const rclcpp::ExecutorOptions & options) @@ -39,11 +33,11 @@ EventsExecutor::spin() } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - auto push_timer = [this](const rclcpp::TimerBase::SharedPtr & t) { + auto push_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { timers.add_timer(t); }; - auto clear_timers = [this]() { + auto clear_timers_function = [this]() { timers.clear(); }; @@ -51,8 +45,8 @@ EventsExecutor::spin() entities_collector_->set_callbacks( this, &EventsExecutor::push_event, - push_timer, - clear_timers); + push_timer_function, + clear_timers_function); std::thread t_exec_timers(&EventsExecutor::execute_timers, this); pthread_setname_np(t_exec_timers.native_handle(), "Timers"); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 806e259151..cf0eb969a8 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -34,8 +34,8 @@ void EventsExecutorEntitiesCollector::set_callbacks( void * executor_context, Event_callback executor_callback, - PushTimer push_timer, - ClearTimers clear_timers) + PushTimerFn push_timer, + ClearTimersFn clear_timers) { executor_context_ = executor_context; executor_callback_ = executor_callback; From 428522b8bf8a3881bacd4d2a0720f13c5f4d0263 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 5 Oct 2020 17:41:03 +0100 Subject: [PATCH 015/168] Some reorder --- .../events_executor_entities_collector.cpp | 102 +++++++++--------- 1 file changed, 52 insertions(+), 50 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index cf0eb969a8..0aab73e28e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -48,13 +48,65 @@ EventsExecutorEntitiesCollector::set_callbacks( // With the new approach, "execute" should only take care of setting that // entitiy's callback. // If a entity is removed from a node, we should unset its callback +// Todo: We're still not ready for this. void EventsExecutorEntitiesCollector::execute() { + // std::cout << "EventsExecutorEntitiesCollector::execute()" << std::endl; // clear_timers_(); // set_entities_callbacks(); } +void +EventsExecutorEntitiesCollector::add_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + void * executor_context, + Event_callback executor_callback) +{ + // If the node already has an executor + std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); + + if (has_executor.exchange(true)) { + throw std::runtime_error("Node has already been added to an executor."); + } + + weak_nodes_.push_back(node_ptr); + + // Set node's guard condition callback, so if a new entitiy is added while + // spinning we can set its callback. + rcl_ret_t ret = rcl_guard_condition_set_callback( + executor_context, + executor_callback, + this, + node_ptr->get_notify_guard_condition()); + + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } +} + +// Here we should unset the node's guard condition callback. +bool +EventsExecutorEntitiesCollector::remove_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) +{ + auto node_it = weak_nodes_.begin(); + + while (node_it != weak_nodes_.end()) { + bool matched = (node_it->lock() == node_ptr); + if (matched) { + // Find and remove node and unset its guard condition callback (TODO) + // rcl_ret_t ret = rcl_guard_condition_unset_callback( + // node_ptr->get_notify_guard_condition()); + weak_nodes_.erase(node_it); + return true; + } else { + ++node_it; + } + } + return false; +} + void EventsExecutorEntitiesCollector::set_entities_callbacks() { @@ -107,53 +159,3 @@ EventsExecutorEntitiesCollector::set_entities_callbacks() } } } - -void -EventsExecutorEntitiesCollector::add_node( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, - void * executor_context, - Event_callback executor_callback) -{ - // If the node already has an executor - std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); - - if (has_executor.exchange(true)) { - throw std::runtime_error("Node has already been added to an executor."); - } - - weak_nodes_.push_back(node_ptr); - - // Set node's guard condition callback, so if a new entitiy is added while - // spinning we can set its callback. - rcl_ret_t ret = rcl_guard_condition_set_callback( - executor_context, - executor_callback, - this, - node_ptr->get_notify_guard_condition()); - - if (ret != RCL_RET_OK) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } -} - -// Here we should unset the node's guard condition callback. -bool -EventsExecutorEntitiesCollector::remove_node( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) -{ - auto node_it = weak_nodes_.begin(); - - while (node_it != weak_nodes_.end()) { - bool matched = (node_it->lock() == node_ptr); - if (matched) { - // Find and remove node and unset its guard condition callback (TODO) - // rcl_ret_t ret = rcl_guard_condition_unset_callback( - // node_ptr->get_notify_guard_condition()); - weak_nodes_.erase(node_it); - return true; - } else { - ++node_it; - } - } - return false; -} From d9039cc0873a581d09c08d5818763925e8fd6aaa Mon Sep 17 00:00:00 2001 From: Mauro Date: Tue, 6 Oct 2020 10:35:46 +0100 Subject: [PATCH 016/168] Use or discard previous events: Guard conditions --- .../rclcpp/executors/events_executor_entities_collector.cpp | 3 ++- rclcpp/src/rclcpp/subscription_intra_process_base.cpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 0aab73e28e..0a3c9d0c5b 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -78,7 +78,8 @@ EventsExecutorEntitiesCollector::add_node( executor_context, executor_callback, this, - node_ptr->get_notify_guard_condition()); + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); if (ret != RCL_RET_OK) { throw std::runtime_error(std::string("Couldn't set guard condition callback")); diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 529f5bb8de..5bb7ad1ebf 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -46,7 +46,8 @@ SubscriptionIntraProcessBase::set_guard_condition_callback( executor_context, executor_callback, this, - &gc_); + &gc_, + true /*Use previous events*/); if (RCL_RET_OK != ret) { throw std::runtime_error(std::string("Couldn't set guard condition callback")); From f0c990660a12423cdbfe0205729aa126c1287be9 Mon Sep 17 00:00:00 2001 From: Mauro Date: Tue, 6 Oct 2020 11:53:26 +0100 Subject: [PATCH 017/168] Implement spin some --- .../rclcpp/executors/events_executor.hpp | 18 +++++- .../src/rclcpp/executors/events_executor.cpp | 56 ++++++++++++++----- .../events_executor_entities_collector.cpp | 3 +- 3 files changed, 62 insertions(+), 15 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 1b8044426b..6f56c58664 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -16,6 +16,7 @@ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ #include +#include #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" @@ -63,6 +64,17 @@ class EventsExecutor : public rclcpp::Executor void spin() override; + /// Events executor implementation of spin some + /** + * executor.provide_callbacks(); + * while(condition) { + * executor.spin_some(); + * } + */ + RCLCPP_PUBLIC + void + spin_some(std::chrono::nanoseconds max_duration) override; + /// Add a node to the executor. /** * \sa rclcpp::Executor::add_node @@ -111,11 +123,15 @@ class EventsExecutor : public rclcpp::Executor void execute_client(rclcpp::ClientBase* client); + RCLCPP_PUBLIC + void + provide_callbacks(); + protected: /// Execute timers when ready RCLCPP_PUBLIC void - execute_timers(); + spin_timers(bool spin_once); /// Execute events in the queue until is empty RCLCPP_PUBLIC diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 674823882e..af5ba277bb 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -33,6 +33,43 @@ EventsExecutor::spin() } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + // Provide callbacks to entities collector + provide_callbacks(); + + std::thread t_spin_timers(&EventsExecutor::spin_timers, this, false); + pthread_setname_np(t_spin_timers.native_handle(), "Timers"); + + while (rclcpp::ok(context_) && spinning.load()) + { + execute_events(); + } + + t_spin_timers.join(); +} + +// Before calling spin_some +void +EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) +{ + (void)max_duration; + + // Check, are we already spinning? + if (spinning.exchange(true)) { + throw std::runtime_error("spin_some() called while already spinning"); + } + RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + + std::thread t_spin_timers(&EventsExecutor::spin_timers, this, true); + + // Execute events and leave + execute_events(); + + t_spin_timers.join(); +} + +void +EventsExecutor::provide_callbacks() +{ auto push_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { timers.add_timer(t); }; @@ -41,22 +78,12 @@ EventsExecutor::spin() timers.clear(); }; - // Init entities collector + // Set entities collector callbacks entities_collector_->set_callbacks( this, &EventsExecutor::push_event, push_timer_function, clear_timers_function); - - std::thread t_exec_timers(&EventsExecutor::execute_timers, this); - pthread_setname_np(t_exec_timers.native_handle(), "Timers"); - - while(spinning.load()) - { - execute_events(); - } - - t_exec_timers.join(); } void @@ -146,12 +173,15 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) } void -EventsExecutor::execute_timers() +EventsExecutor::spin_timers(bool spin_once) { - while (rclcpp::ok(this->context_) && spinning.load()) + while (rclcpp::ok(context_) && spinning.load()) { std::this_thread::sleep_for(timers.get_head_timeout()); timers.execute_ready_timers(); + if (spin_once) { + break; + } } } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 0a3c9d0c5b..3dc7dbdf14 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -37,6 +37,8 @@ EventsExecutorEntitiesCollector::set_callbacks( PushTimerFn push_timer, ClearTimersFn clear_timers) { + // These callbacks are used whenever a new entity is added/removed + // to/from a new node executor_context_ = executor_context; executor_callback_ = executor_callback; push_timer_ = push_timer; @@ -52,7 +54,6 @@ EventsExecutorEntitiesCollector::set_callbacks( void EventsExecutorEntitiesCollector::execute() { - // std::cout << "EventsExecutorEntitiesCollector::execute()" << std::endl; // clear_timers_(); // set_entities_callbacks(); } From b6f092295d15ab9b99e9b3ee18e31a9352d883ed Mon Sep 17 00:00:00 2001 From: Mauro Date: Tue, 6 Oct 2020 12:59:15 +0100 Subject: [PATCH 018/168] Remove node from executor support --- .../events_executor_entities_collector.hpp | 14 ++-- .../include/rclcpp/executors/timers_heap.hpp | 8 +- .../src/rclcpp/executors/events_executor.cpp | 25 +++--- .../events_executor_entities_collector.cpp | 78 +++++++++++++++---- 4 files changed, 90 insertions(+), 35 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index a332b16e5a..48b66b2cfa 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -30,7 +30,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable public: RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorEntitiesCollector) - using PushTimerFn = std::function; + using TimerFn = std::function; using ClearTimersFn = std::function; // Constructor @@ -46,8 +46,9 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable set_callbacks( void * executor_context, Event_callback executor_callback, - PushTimerFn push_timer, - ClearTimersFn clear_timers); + TimerFn push_timer, + TimerFn clear_timer, + ClearTimersFn clear_all_timers); RCLCPP_PUBLIC void @@ -61,7 +62,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable Event_callback executor_callback); RCLCPP_PUBLIC - bool + void remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr); @@ -95,8 +96,9 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable Event_callback executor_callback_ = nullptr; /// Function pointers to push and clear timers from the timers heap - PushTimerFn push_timer_ = nullptr; - ClearTimersFn clear_timers_ = nullptr; + TimerFn push_timer_ = nullptr; + TimerFn clear_timer_ = nullptr; + ClearTimersFn clear_all_timers_ = nullptr; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/timers_heap.hpp b/rclcpp/include/rclcpp/executors/timers_heap.hpp index 3b0e07814a..4fb4417e15 100644 --- a/rclcpp/include/rclcpp/executors/timers_heap.hpp +++ b/rclcpp/include/rclcpp/executors/timers_heap.hpp @@ -79,11 +79,17 @@ struct TimersHeap } } - inline void clear() + inline void clear_all() { // Todo: Implement clear all timers. } + inline void remove_timer(rclcpp::TimerBase::SharedPtr timer) + { + // Todo: Implement + (void)timer; + } + private: struct TimerInternal { diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index af5ba277bb..ff764ac761 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -74,8 +74,12 @@ EventsExecutor::provide_callbacks() timers.add_timer(t); }; - auto clear_timers_function = [this]() { - timers.clear(); + auto clear_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { + timers.remove_timer(t); + }; + + auto clear_all_timers_function = [this]() { + timers.clear_all(); }; // Set entities collector callbacks @@ -83,7 +87,8 @@ EventsExecutor::provide_callbacks() this, &EventsExecutor::push_event, push_timer_function, - clear_timers_function); + clear_timer_function, + clear_all_timers_function); } void @@ -149,21 +154,11 @@ void EventsExecutor::remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { - bool node_removed = entities_collector_->remove_node(node_ptr); - - if (notify) { - // If the node was matched and removed, interrupt waiting - if (node_removed) { - if (rcl_trigger_guard_condition(&interrupt_guard_condition_) != RCL_RET_OK) { - throw std::runtime_error(rcl_get_error_string().str); - } - } - } + (void)notify; + entities_collector_->remove_node(node_ptr); std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); has_executor.store(false); - - // Todo: Remove the timers from the timers heap here and unset entities callback } void diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 3dc7dbdf14..5fb54c1f6a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -34,15 +34,18 @@ void EventsExecutorEntitiesCollector::set_callbacks( void * executor_context, Event_callback executor_callback, - PushTimerFn push_timer, - ClearTimersFn clear_timers) + TimerFn push_timer, + TimerFn clear_timer, + ClearTimersFn clear_all_timers) { - // These callbacks are used whenever a new entity is added/removed - // to/from a new node + // These callbacks are used when: + // 1. A new entity is added/removed to/from a new node + // 2. A node is removed from the executor executor_context_ = executor_context; executor_callback_ = executor_callback; push_timer_ = push_timer; - clear_timers_ = clear_timers; + clear_timer_ = clear_timer; + clear_all_timers_ = clear_all_timers; } // The purpose of "execute" is handling the situation of a new entity added to @@ -54,7 +57,7 @@ EventsExecutorEntitiesCollector::set_callbacks( void EventsExecutorEntitiesCollector::execute() { - // clear_timers_(); + // clear_all_timers_(); // set_entities_callbacks(); } @@ -87,8 +90,8 @@ EventsExecutorEntitiesCollector::add_node( } } -// Here we should unset the node's guard condition callback. -bool +// Here we unset the node entities callback. +void EventsExecutorEntitiesCollector::remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { @@ -97,16 +100,65 @@ EventsExecutorEntitiesCollector::remove_node( while (node_it != weak_nodes_.end()) { bool matched = (node_it->lock() == node_ptr); if (matched) { - // Find and remove node and unset its guard condition callback (TODO) - // rcl_ret_t ret = rcl_guard_condition_unset_callback( - // node_ptr->get_notify_guard_condition()); + // Node found: unset its entities callbacks + rcl_ret_t ret = rcl_guard_condition_set_callback( + nullptr, nullptr, nullptr, + node_ptr->get_notify_guard_condition(), + false); + + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } + + // Unset entities callbacks + for (auto & weak_group : node_ptr->get_callback_groups()) { + auto group = weak_group.lock(); + if (!group || !group->can_be_taken_from().load()) { + continue; + } + group->find_timer_ptrs_if( + [this](const rclcpp::TimerBase::SharedPtr & timer) { + if (timer) { + clear_timer_(timer); + } + return false; + }); + group->find_subscription_ptrs_if( + [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { + if (subscription) { + subscription->set_callback(nullptr, nullptr); + } + return false; + }); + group->find_service_ptrs_if( + [this](const rclcpp::ServiceBase::SharedPtr & service) { + if (service) { + service->set_callback(nullptr, nullptr); + } + return false; + }); + group->find_client_ptrs_if( + [this](const rclcpp::ClientBase::SharedPtr & client) { + if (client) { + client->set_callback(nullptr, nullptr); + } + return false; + }); + group->find_waitable_ptrs_if( + [this](const rclcpp::Waitable::SharedPtr & waitable) { + if (waitable) { + waitable->set_guard_condition_callback(nullptr, nullptr); + } + return false; + }); + } + weak_nodes_.erase(node_it); - return true; + return; } else { ++node_it; } } - return false; } void From dd29fa0e040be85da81d64fe5e60d38816ce3016 Mon Sep 17 00:00:00 2001 From: Mauro Date: Tue, 6 Oct 2020 17:17:37 +0100 Subject: [PATCH 019/168] move execute entities to executor.cpp --- rclcpp/include/rclcpp/executor.hpp | 20 ++-- .../rclcpp/executors/events_executor.hpp | 12 --- rclcpp/src/rclcpp/executor.cpp | 24 ++++- .../src/rclcpp/executors/events_executor.cpp | 100 +----------------- 4 files changed, 36 insertions(+), 120 deletions(-) diff --git a/rclcpp/include/rclcpp/executor.hpp b/rclcpp/include/rclcpp/executor.hpp index f16d1e00b4..628e05b58e 100644 --- a/rclcpp/include/rclcpp/executor.hpp +++ b/rclcpp/include/rclcpp/executor.hpp @@ -420,14 +420,6 @@ class Executor void execute_any_executable(AnyExecutable & any_exec); - RCLCPP_PUBLIC - static void - take_and_do_error_handling( - const char * action_description, - const char * topic_or_service_name, - std::function take_action, - std::function handle_action); - RCLCPP_PUBLIC static void execute_subscription( @@ -445,6 +437,18 @@ class Executor static void execute_client(rclcpp::ClientBase::SharedPtr client); + RCLCPP_PUBLIC + static void + execute_subscription(rclcpp::SubscriptionBase* subscription); + + RCLCPP_PUBLIC + static void + execute_service(rclcpp::ServiceBase* service); + + RCLCPP_PUBLIC + static void + execute_client(rclcpp::ClientBase* client); + /** * \throws std::runtime_error if the wait set can be cleared */ diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 6f56c58664..6012837125 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -111,18 +111,6 @@ class EventsExecutor : public rclcpp::Executor void remove_node(std::shared_ptr node_ptr, bool notify = true) override; - RCLCPP_PUBLIC - void - execute_subscription(rclcpp::SubscriptionBase* subscription); - - RCLCPP_PUBLIC - void - execute_service(rclcpp::ServiceBase* service); - - RCLCPP_PUBLIC - void - execute_client(rclcpp::ClientBase* client); - RCLCPP_PUBLIC void provide_callbacks(); diff --git a/rclcpp/src/rclcpp/executor.cpp b/rclcpp/src/rclcpp/executor.cpp index 3204187b10..12225e3cef 100644 --- a/rclcpp/src/rclcpp/executor.cpp +++ b/rclcpp/src/rclcpp/executor.cpp @@ -518,8 +518,9 @@ Executor::execute_any_executable(AnyExecutable & any_exec) } } +static void -Executor::take_and_do_error_handling( +take_and_do_error_handling( const char * action_description, const char * topic_or_service_name, std::function take_action, @@ -555,6 +556,12 @@ Executor::take_and_do_error_handling( void Executor::execute_subscription(rclcpp::SubscriptionBase::SharedPtr subscription) +{ + execute_subscription(subscription.get()); +} + +void +Executor::execute_subscription(rclcpp::SubscriptionBase* subscription) { rclcpp::MessageInfo message_info; message_info.get_rmw_message_info().from_intra_process = false; @@ -629,6 +636,12 @@ Executor::execute_timer(rclcpp::TimerBase::SharedPtr timer) void Executor::execute_service(rclcpp::ServiceBase::SharedPtr service) +{ + execute_service(service.get()); +} + +void +Executor::execute_service(rclcpp::ServiceBase* service) { auto request_header = service->create_request_header(); std::shared_ptr request = service->create_request(); @@ -640,8 +653,13 @@ Executor::execute_service(rclcpp::ServiceBase::SharedPtr service) } void -Executor::execute_client( - rclcpp::ClientBase::SharedPtr client) +Executor::execute_client(rclcpp::ClientBase::SharedPtr client) +{ + execute_client(client.get()); +} + +void +Executor::execute_client(rclcpp::ClientBase* client) { auto request_header = client->create_request_header(); std::shared_ptr response = client->create_response(); diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index ff764ac761..c252b39edc 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -211,7 +211,7 @@ EventsExecutor::execute_events() { auto subscription = const_cast( static_cast(event.entity)); - execute_subscription(subscription); + //execute_subscription(subscription); break; } @@ -219,7 +219,7 @@ EventsExecutor::execute_events() { auto service = const_cast( static_cast(event.entity)); - execute_service(service); + //execute_service(service); break; } @@ -227,7 +227,7 @@ EventsExecutor::execute_events() { auto client = const_cast( static_cast(event.entity)); - execute_client(client); + //execute_client(client); break; } @@ -242,97 +242,3 @@ EventsExecutor::execute_events() } } while (!local_event_queue.empty()); } - - -// Raw pointer versions of Executor::execute_ -void -EventsExecutor::execute_subscription(rclcpp::SubscriptionBase* subscription) -{ - rclcpp::MessageInfo message_info; - message_info.get_rmw_message_info().from_intra_process = false; - - if (subscription->is_serialized()) { - // This is the case where a copy of the serialized message is taken from - // the middleware via inter-process communication. - std::shared_ptr serialized_msg = subscription->create_serialized_message(); - take_and_do_error_handling( - "taking a serialized message from topic", - subscription->get_topic_name(), - [&]() {return subscription->take_serialized(*serialized_msg.get(), message_info);}, - [&]() - { - auto void_serialized_msg = std::static_pointer_cast(serialized_msg); - subscription->handle_message(void_serialized_msg, message_info); - }); - subscription->return_serialized_message(serialized_msg); - } else if (subscription->can_loan_messages()) { - // This is the case where a loaned message is taken from the middleware via - // inter-process communication, given to the user for their callback, - // and then returned. - void * loaned_msg = nullptr; - // TODO(wjwwood): refactor this into methods on subscription when LoanedMessage - // is extened to support subscriptions as well. - take_and_do_error_handling( - "taking a loaned message from topic", - subscription->get_topic_name(), - [&]() - { - rcl_ret_t ret = rcl_take_loaned_message( - subscription->get_subscription_handle().get(), - &loaned_msg, - &message_info.get_rmw_message_info(), - nullptr); - if (RCL_RET_SUBSCRIPTION_TAKE_FAILED == ret) { - return false; - } else if (RCL_RET_OK != ret) { - rclcpp::exceptions::throw_from_rcl_error(ret); - } - return true; - }, - [&]() {subscription->handle_loaned_message(loaned_msg, message_info);}); - rcl_ret_t ret = rcl_return_loaned_message_from_subscription( - subscription->get_subscription_handle().get(), - loaned_msg); - if (RCL_RET_OK != ret) { - RCLCPP_ERROR( - rclcpp::get_logger("rclcpp"), - "rcl_return_loaned_message_from_subscription() failed for subscription on topic '%s': %s", - subscription->get_topic_name(), rcl_get_error_string().str); - } - loaned_msg = nullptr; - } else { - // This case is taking a copy of the message data from the middleware via - // inter-process communication. - std::shared_ptr message = subscription->create_message(); - take_and_do_error_handling( - "taking a message from topic", - subscription->get_topic_name(), - [&]() {return subscription->take_type_erased(message.get(), message_info);}, - [&]() {subscription->handle_message(message, message_info);}); - subscription->return_message(message); - } -} - -void -EventsExecutor::execute_service(rclcpp::ServiceBase* service) -{ - auto request_header = service->create_request_header(); - std::shared_ptr request = service->create_request(); - take_and_do_error_handling( - "taking a service server request from service", - service->get_service_name(), - [&]() {return service->take_type_erased_request(request.get(), *request_header);}, - [&]() {service->handle_request(request_header, request);}); -} - -void -EventsExecutor::execute_client(rclcpp::ClientBase* client) -{ - auto request_header = client->create_request_header(); - std::shared_ptr response = client->create_response(); - take_and_do_error_handling( - "taking a service client response from service", - client->get_service_name(), - [&]() {return client->take_type_erased_response(response.get(), *request_header);}, - [&]() {client->handle_response(request_header, response);}); -} From 8e55fc795c82055056ee240864a43cfe2c1487da Mon Sep 17 00:00:00 2001 From: Mauro Date: Tue, 6 Oct 2020 17:52:45 +0100 Subject: [PATCH 020/168] Move apis --- .../events_executor_entities_collector.hpp | 4 +--- .../src/rclcpp/executors/events_executor.cpp | 23 +++++++++++++++---- .../events_executor_entities_collector.cpp | 17 +------------- 3 files changed, 21 insertions(+), 23 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 48b66b2cfa..46b183222b 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -57,9 +57,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable RCLCPP_PUBLIC void add_node( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, - void * executor_context, - Event_callback executor_callback); + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr); RCLCPP_PUBLIC void diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index c252b39edc..73997589a6 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -97,8 +97,23 @@ EventsExecutor::add_node( { (void) notify; - entities_collector_->add_node(node_ptr, this, &EventsExecutor::push_event); + // Add node to entities collector + entities_collector_->add_node(node_ptr); + + // Set node's guard condition callback, so if a new entitiy is added while + // spinning we can set its callback. + rcl_ret_t ret = rcl_guard_condition_set_callback( + this, + &EventsExecutor::push_event, + entities_collector_.get(), + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } + // Get nodes entities, and assign their callbaks for (auto & weak_group : node_ptr->get_callback_groups()) { auto group = weak_group.lock(); if (!group || !group->can_be_taken_from().load()) { @@ -211,7 +226,7 @@ EventsExecutor::execute_events() { auto subscription = const_cast( static_cast(event.entity)); - //execute_subscription(subscription); + execute_subscription(subscription); break; } @@ -219,7 +234,7 @@ EventsExecutor::execute_events() { auto service = const_cast( static_cast(event.entity)); - //execute_service(service); + execute_service(service); break; } @@ -227,7 +242,7 @@ EventsExecutor::execute_events() { auto client = const_cast( static_cast(event.entity)); - //execute_client(client); + execute_client(client); break; } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 5fb54c1f6a..2f283b1215 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -63,9 +63,7 @@ EventsExecutorEntitiesCollector::execute() void EventsExecutorEntitiesCollector::add_node( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, - void * executor_context, - Event_callback executor_callback) + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { // If the node already has an executor std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); @@ -75,19 +73,6 @@ EventsExecutorEntitiesCollector::add_node( } weak_nodes_.push_back(node_ptr); - - // Set node's guard condition callback, so if a new entitiy is added while - // spinning we can set its callback. - rcl_ret_t ret = rcl_guard_condition_set_callback( - executor_context, - executor_callback, - this, - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } } // Here we unset the node entities callback. From a8bd37d3311fc2995d6649da192b36c0dbca334f Mon Sep 17 00:00:00 2001 From: Mauro Date: Wed, 7 Oct 2020 10:11:47 +0100 Subject: [PATCH 021/168] init entities collector at executor construction --- .../rclcpp/executors/events_executor.hpp | 4 -- .../events_executor_entities_collector.hpp | 2 +- .../src/rclcpp/executors/events_executor.cpp | 48 ++++++++----------- .../events_executor_entities_collector.cpp | 2 +- 4 files changed, 23 insertions(+), 33 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 6012837125..5f16cec79a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -111,10 +111,6 @@ class EventsExecutor : public rclcpp::Executor void remove_node(std::shared_ptr node_ptr, bool notify = true) override; - RCLCPP_PUBLIC - void - provide_callbacks(); - protected: /// Execute timers when ready RCLCPP_PUBLIC diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 46b183222b..96540f74ce 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -43,7 +43,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable RCLCPP_PUBLIC void - set_callbacks( + init( void * executor_context, Event_callback executor_callback, TimerFn push_timer, diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 73997589a6..bd1d7ae8ae 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -21,6 +21,27 @@ EventsExecutor::EventsExecutor( : rclcpp::Executor(options) { entities_collector_ = std::make_shared(); + + auto push_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { + timers.add_timer(t); + }; + + auto clear_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { + timers.remove_timer(t); + }; + + auto clear_all_timers_function = [this]() { + timers.clear_all(); + }; + + // Set entities collector callbacks + entities_collector_->init( + this, + &EventsExecutor::push_event, + push_timer_function, + clear_timer_function, + clear_all_timers_function); + } EventsExecutor::~EventsExecutor() {} @@ -33,9 +54,6 @@ EventsExecutor::spin() } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - // Provide callbacks to entities collector - provide_callbacks(); - std::thread t_spin_timers(&EventsExecutor::spin_timers, this, false); pthread_setname_np(t_spin_timers.native_handle(), "Timers"); @@ -67,30 +85,6 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) t_spin_timers.join(); } -void -EventsExecutor::provide_callbacks() -{ - auto push_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { - timers.add_timer(t); - }; - - auto clear_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { - timers.remove_timer(t); - }; - - auto clear_all_timers_function = [this]() { - timers.clear_all(); - }; - - // Set entities collector callbacks - entities_collector_->set_callbacks( - this, - &EventsExecutor::push_event, - push_timer_function, - clear_timer_function, - clear_all_timers_function); -} - void EventsExecutor::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 2f283b1215..fb0ffb633e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -31,7 +31,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() } void -EventsExecutorEntitiesCollector::set_callbacks( +EventsExecutorEntitiesCollector::init( void * executor_context, Event_callback executor_callback, TimerFn push_timer, From 4c03213fee85afcd3028a6185209d77d4c3bd148 Mon Sep 17 00:00:00 2001 From: Mauro Date: Wed, 7 Oct 2020 13:24:21 +0100 Subject: [PATCH 022/168] reimplement timers heap -> timers queue --- .../rclcpp/executors/events_executor.hpp | 2 +- .../include/rclcpp/executors/timers_heap.hpp | 193 ++++-------------- .../events_executor_entities_collector.cpp | 4 +- 3 files changed, 46 insertions(+), 153 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 5f16cec79a..0e74ededfe 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -154,7 +154,7 @@ class EventsExecutor : public rclcpp::Executor // Timers heap members // TODO: Finish this to not hardcode that magic number - TimersHeap<30> timers; + TimersHeap timers; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/timers_heap.hpp b/rclcpp/include/rclcpp/executors/timers_heap.hpp index 4fb4417e15..d56f952621 100644 --- a/rclcpp/include/rclcpp/executors/timers_heap.hpp +++ b/rclcpp/include/rclcpp/executors/timers_heap.hpp @@ -11,209 +11,102 @@ namespace rclcpp namespace executors { -template struct TimersHeap { public: /** * @brief Construct a new Timers Heap object */ - TimersHeap() - { - clock = rclcpp::Clock(RCL_ROS_TIME); - size = 0; - } + TimersHeap() = default; /** - * @brief Adds a new TimerBase to the heap + * @brief Adds a new TimerBase to the queue * @param timer the timer to be added - * @return int 0 if success, -1 if the heap is already full + * @return int 0 if success, -1 if the queue is already full */ - inline int add_timer(rclcpp::TimerBase::SharedPtr timer) + inline void add_timer(rclcpp::TimerBase::SharedPtr timer) { - if (size == MAX_SIZE) { - // The heap is full, can't push - return -1; - } - - // Create a TimerInternal object to store the timer - timers_storage[size] = TimerInternal(timer, clock); - TimerInternalPtr t = &(timers_storage[size]); - - // Push a TimerInternalPtr into the heap - push(t); - return 0; + timers_queue.emplace_back(std::move(timer)); + reorder_queue(); } /** - * @brief Get the time before the first timer in the heap expires + * @brief Get the time before the first timer in the queue expires * - * @return std::chrono::nanoseconds to wait, 0 if the timer is already expired + * @return std::chrono::nanoseconds to wait, can return a negative number + * if the timer is already expired */ inline std::chrono::nanoseconds get_head_timeout() { auto min_timeout = std::chrono::nanoseconds::max(); - if (peek(&head) == 0) { - min_timeout = std::chrono::nanoseconds(head->expire_time - clock.now().nanoseconds()); - } - if (min_timeout < 0ns) { - min_timeout = 0ns; + auto head = peek(); + + if (head != nullptr) { + min_timeout = head->time_until_trigger(); } + return min_timeout; } /** - * @brief Executes all the ready timers in the heap - * These timers are refreshed and added back to the heap - * NOTE: may block indefinitely if the time for processing callbacks is longer than the timers period + * @brief Executes all the ready timers in the queue + * These timers are refreshed and added back to the queue */ inline void execute_ready_timers() { - while (peek(&head) == 0 && head->timer->is_ready()) { - head->timer->execute_callback(); + auto head = peek(); - head->refresh(clock); - remove_at(0); - push(head); + while (head != nullptr && head->is_ready()) { + head->execute_callback(); + head = peek(); } + + reorder_queue(); } inline void clear_all() { - // Todo: Implement clear all timers. + timers_queue.clear(); } inline void remove_timer(rclcpp::TimerBase::SharedPtr timer) { - // Todo: Implement - (void)timer; - } - -private: - struct TimerInternal - { - inline TimerInternal() - { - timer = nullptr; - expire_time = INT64_MAX; - } - - inline TimerInternal(rclcpp::TimerBase::SharedPtr t, rclcpp::Clock& clock) - { - timer = t; - refresh(clock); - } - - inline void refresh(rclcpp::Clock& clock) - { - expire_time = clock.now().nanoseconds() + timer->time_until_trigger().count(); - } - - rclcpp::TimerBase::SharedPtr timer; - int64_t expire_time; - }; - - using TimerInternalPtr = TimerInternal*; - - inline void push(TimerInternalPtr x) - { - size_t i = size++; - a[i] = x; - while (i && (x->expire_time < a[(i-1)/2]->expire_time)) { - a[i] = a[(i-1)/2]; - a[(i-1)/2] = x; - i = (i-1)/2; - } - } - - inline void remove_at(size_t i) - { - TimerInternalPtr y = a[--size]; - a[i] = y; - - // Heapify upwards. - while (i > 0) { - size_t parent = (i-1)/2; - if (y->expire_time < a[parent]->expire_time) { - a[i] = a[parent]; - a[parent] = y; - i = parent; - } else { - break; - } - } - - // Heapify downwards - while (2*i + 1 < size) { - size_t hi = i; - size_t left = 2*i+1; - size_t right = left + 1; - if (y->expire_time > a[left]->expire_time) { - hi = left; - } - if (right < size && (a[hi]->expire_time > a[right]->expire_time)) { - hi = right; - } - if (hi != i) { - a[i] = a[hi]; - a[hi] = y; - i = hi; - } else { + for (auto it = timers_queue.begin(); it != timers_queue.end(); ++it) { + if((*it).get() == timer.get()) { + timers_queue.erase(it); break; } } } - inline int pop(TimerInternalPtr x) - { - if (size == 0) { - // The heap is empty, can't pop - return -1; - } - - x = a[0]; - remove_at(0); - return 0; - } +private: - inline int peek(TimerInternalPtr* x) + struct timer_less_than_comparison { - if (size == 0) { - // The heap is empty, can't peek - return -1; + inline bool operator() ( + const rclcpp::TimerBase::SharedPtr& timer1, + const rclcpp::TimerBase::SharedPtr& timer2) + { + return (timer1->time_until_trigger() < timer2->time_until_trigger()); } + }; - *x = a[0]; - return 0; + inline void reorder_queue() + { + std::sort(timers_queue.begin(), timers_queue.end(), timer_less_than_comparison()); } - inline int remove(TimerInternalPtr x) + inline rclcpp::TimerBase::SharedPtr peek() { - size_t i; - for (i = 0; i < size; ++i) { - if (x == a[i]) { - break; - } - } - if (i == size) { - return -1; + if (timers_queue.empty()) { + return nullptr; } - - remove_at(i); - return 0; + return timers_queue.front(); } - // Array to keep ownership of the timers - std::array timers_storage; - // Array of pointers to stored timers used to implement the priority queue - std::array a; - // Helper to access first element in the heap - TimerInternalPtr head; - // Current number of elements in the heap - size_t size; - // Clock to update expiration times and generate timeouts - rclcpp::Clock clock; + // Ordered queue of timers + std::vector timers_queue; }; } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index fb0ffb633e..92eee91674 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -57,8 +57,8 @@ EventsExecutorEntitiesCollector::init( void EventsExecutorEntitiesCollector::execute() { - // clear_all_timers_(); - // set_entities_callbacks(); + clear_all_timers_(); + set_entities_callbacks(); } void From aa4f73600798f37f92e27e125ce252c712feec86 Mon Sep 17 00:00:00 2001 From: Mauro Date: Wed, 7 Oct 2020 13:46:46 +0100 Subject: [PATCH 023/168] rename timers queue --- rclcpp/include/rclcpp/executors/events_executor.hpp | 4 ++-- .../rclcpp/executors/{timers_heap.hpp => timers_queue.hpp} | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) rename rclcpp/include/rclcpp/executors/{timers_heap.hpp => timers_queue.hpp} (98%) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 0e74ededfe..3700a81834 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -20,7 +20,7 @@ #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" -#include "rclcpp/executors/timers_heap.hpp" +#include "rclcpp/executors/timers_queue.hpp" #include "rclcpp/node.hpp" #include "rcutils/event_types.h" @@ -154,7 +154,7 @@ class EventsExecutor : public rclcpp::Executor // Timers heap members // TODO: Finish this to not hardcode that magic number - TimersHeap timers; + TimersQueue timers; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/timers_heap.hpp b/rclcpp/include/rclcpp/executors/timers_queue.hpp similarity index 98% rename from rclcpp/include/rclcpp/executors/timers_heap.hpp rename to rclcpp/include/rclcpp/executors/timers_queue.hpp index d56f952621..50ed85234f 100644 --- a/rclcpp/include/rclcpp/executors/timers_heap.hpp +++ b/rclcpp/include/rclcpp/executors/timers_queue.hpp @@ -11,13 +11,13 @@ namespace rclcpp namespace executors { -struct TimersHeap +struct TimersQueue { public: /** * @brief Construct a new Timers Heap object */ - TimersHeap() = default; + TimersQueue() = default; /** * @brief Adds a new TimerBase to the queue From f630da9e9047b1512c596c3612b39738d4942c08 Mon Sep 17 00:00:00 2001 From: Mauro Date: Wed, 7 Oct 2020 13:55:12 +0100 Subject: [PATCH 024/168] update timers queue --- .../include/rclcpp/executors/timers_queue.hpp | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_queue.hpp b/rclcpp/include/rclcpp/executors/timers_queue.hpp index 50ed85234f..b758649d16 100644 --- a/rclcpp/include/rclcpp/executors/timers_queue.hpp +++ b/rclcpp/include/rclcpp/executors/timers_queue.hpp @@ -40,7 +40,7 @@ struct TimersQueue { auto min_timeout = std::chrono::nanoseconds::max(); - auto head = peek(); + auto head = timers_queue.front(); if (head != nullptr) { min_timeout = head->time_until_trigger(); @@ -51,17 +51,15 @@ struct TimersQueue /** * @brief Executes all the ready timers in the queue - * These timers are refreshed and added back to the queue */ inline void execute_ready_timers() { - auto head = peek(); - - while (head != nullptr && head->is_ready()) { - head->execute_callback(); - head = peek(); + for (const auto &timer : timers_queue) { + if (!timer->is_ready()) { + break; + } + timer->execute_callback(); } - reorder_queue(); } @@ -97,14 +95,6 @@ struct TimersQueue std::sort(timers_queue.begin(), timers_queue.end(), timer_less_than_comparison()); } - inline rclcpp::TimerBase::SharedPtr peek() - { - if (timers_queue.empty()) { - return nullptr; - } - return timers_queue.front(); - } - // Ordered queue of timers std::vector timers_queue; }; From 45e58855d9d69ee0902a788a00e6e799690feba5 Mon Sep 17 00:00:00 2001 From: Mauro Date: Thu, 8 Oct 2020 12:06:03 +0100 Subject: [PATCH 025/168] Move code --- .../rclcpp/executors/events_executor.hpp | 3 +-- .../src/rclcpp/executors/events_executor.cpp | 26 +++++++++---------- .../events_executor_entities_collector.cpp | 1 - 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 3700a81834..66fffe1ba0 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -152,8 +152,7 @@ class EventsExecutor : public rclcpp::Executor std::mutex event_queue_mutex_; std::condition_variable event_queue_cv; - // Timers heap members - // TODO: Finish this to not hardcode that magic number + // Timers queue manager TimersQueue timers; }; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index bd1d7ae8ae..f046966bbe 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -94,19 +94,6 @@ EventsExecutor::add_node( // Add node to entities collector entities_collector_->add_node(node_ptr); - // Set node's guard condition callback, so if a new entitiy is added while - // spinning we can set its callback. - rcl_ret_t ret = rcl_guard_condition_set_callback( - this, - &EventsExecutor::push_event, - entities_collector_.get(), - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } - // Get nodes entities, and assign their callbaks for (auto & weak_group : node_ptr->get_callback_groups()) { auto group = weak_group.lock(); @@ -151,6 +138,19 @@ EventsExecutor::add_node( return false; }); } + + // Set node's guard condition callback, so if new entities are added while + // spinning we can set their callback. + rcl_ret_t ret = rcl_guard_condition_set_callback( + this, + &EventsExecutor::push_event, + entities_collector_.get(), + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } } void diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 92eee91674..1379cdb75e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -53,7 +53,6 @@ EventsExecutorEntitiesCollector::init( // With the new approach, "execute" should only take care of setting that // entitiy's callback. // If a entity is removed from a node, we should unset its callback -// Todo: We're still not ready for this. void EventsExecutorEntitiesCollector::execute() { From 8bcd14f368c115acd989a2faf9f5e8d4ee8f04fb Mon Sep 17 00:00:00 2001 From: Mauro Date: Thu, 8 Oct 2020 12:09:23 +0100 Subject: [PATCH 026/168] Use lambdas directly --- .../src/rclcpp/executors/events_executor.cpp | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index f046966bbe..dba0f7a31e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -22,25 +22,19 @@ EventsExecutor::EventsExecutor( { entities_collector_ = std::make_shared(); - auto push_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { - timers.add_timer(t); - }; - - auto clear_timer_function = [this](const rclcpp::TimerBase::SharedPtr & t) { - timers.remove_timer(t); - }; - - auto clear_all_timers_function = [this]() { - timers.clear_all(); - }; - // Set entities collector callbacks entities_collector_->init( this, &EventsExecutor::push_event, - push_timer_function, - clear_timer_function, - clear_all_timers_function); + [this](const rclcpp::TimerBase::SharedPtr & t) { + timers.add_timer(t); + }, + [this](const rclcpp::TimerBase::SharedPtr & t) { + timers.remove_timer(t); + }, + [this]() { + timers.clear_all(); + }); } From 33f1566e02ed0fd9c633e5c7fd4fbb71171d0cce Mon Sep 17 00:00:00 2001 From: Mauro Date: Thu, 8 Oct 2020 15:24:26 +0100 Subject: [PATCH 027/168] Handle default event types --- rclcpp/src/rclcpp/executors/events_executor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index dba0f7a31e..0ec9564b4d 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -48,6 +48,7 @@ EventsExecutor::spin() } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + // Start timers thread std::thread t_spin_timers(&EventsExecutor::spin_timers, this, false); pthread_setname_np(t_spin_timers.native_handle(), "Timers"); @@ -71,6 +72,7 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + // Start timers thread std::thread t_spin_timers(&EventsExecutor::spin_timers, this, true); // Execute events and leave @@ -242,6 +244,7 @@ EventsExecutor::execute_events() break; } + default: break; } } while (!local_event_queue.empty()); } From a5e0d76be9a65ee72cf3f839ff0c8397e769d7a7 Mon Sep 17 00:00:00 2001 From: Mauro Date: Thu, 8 Oct 2020 16:29:08 +0100 Subject: [PATCH 028/168] Fix spinsome --- rclcpp/src/rclcpp/executors/events_executor.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 0ec9564b4d..f386057faf 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -76,7 +76,10 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) std::thread t_spin_timers(&EventsExecutor::spin_timers, this, true); // Execute events and leave - execute_events(); + if (rclcpp::ok(context_) && spinning.load()) + { + execute_events(); + } t_spin_timers.join(); } From a34bc251d1ab07ccd4f07f10b1231437e26c9b95 Mon Sep 17 00:00:00 2001 From: Mauro Date: Thu, 8 Oct 2020 16:59:40 +0100 Subject: [PATCH 029/168] Check rclcpp::ok before execute timers --- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index f386057faf..bb2aa87b66 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -180,11 +180,11 @@ EventsExecutor::spin_timers(bool spin_once) { while (rclcpp::ok(context_) && spinning.load()) { - std::this_thread::sleep_for(timers.get_head_timeout()); timers.execute_ready_timers(); if (spin_once) { break; } + std::this_thread::sleep_for(timers.get_head_timeout()); } } From d67207cb7ef094678473ef972258e9dfb38dd24f Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 9 Oct 2020 12:34:39 +0100 Subject: [PATCH 030/168] Restore timers heap --- .../rclcpp/executors/events_executor.hpp | 6 +- .../include/rclcpp/executors/timers_heap.hpp | 225 ++++++++++++++++++ 2 files changed, 228 insertions(+), 3 deletions(-) create mode 100644 rclcpp/include/rclcpp/executors/timers_heap.hpp diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 66fffe1ba0..32ec15b3d2 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -20,7 +20,7 @@ #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" -#include "rclcpp/executors/timers_queue.hpp" +#include "rclcpp/executors/timers_heap.hpp" #include "rclcpp/node.hpp" #include "rcutils/event_types.h" @@ -152,8 +152,8 @@ class EventsExecutor : public rclcpp::Executor std::mutex event_queue_mutex_; std::condition_variable event_queue_cv; - // Timers queue manager - TimersQueue timers; + // Timers heap manager + TimersHeap timers; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/timers_heap.hpp b/rclcpp/include/rclcpp/executors/timers_heap.hpp new file mode 100644 index 0000000000..af4520cc21 --- /dev/null +++ b/rclcpp/include/rclcpp/executors/timers_heap.hpp @@ -0,0 +1,225 @@ +#include +#include + +#include +#include + +using namespace std::chrono_literals; + +namespace rclcpp +{ +namespace executors +{ + +struct TimersHeap +{ +public: + /** + * @brief Construct a new Timers Heap object + */ + TimersHeap() + { + clock = rclcpp::Clock(RCL_ROS_TIME); + size = 0; + } + + /** + * @brief Adds a new TimerBase to the heap + * @param timer the timer to be added + * @return int 0 if success, -1 if the heap is already full + */ + inline int add_timer(rclcpp::TimerBase::SharedPtr timer) + { + // Add timer to vector and order by expiration time + timers_storage.emplace_back(TimerInternal(timer, clock)); + std::sort(timers_storage.begin(), timers_storage.end()); + + // Clear heap as the pointers likely become invalid after the above emplace_back. + heap.clear(); + for (auto& t : timers_storage) { + heap.push_back(&t); + } + + size = heap.size(); + + return 0; + } + + /** + * @brief Get the time before the first timer in the heap expires + * + * @return std::chrono::nanoseconds to wait, 0 if the timer is already expired + */ + inline std::chrono::nanoseconds get_head_timeout() + { + auto min_timeout = std::chrono::nanoseconds::max(); + if (peek(&head) == 0) { + min_timeout = std::chrono::nanoseconds(head->expire_time - clock.now().nanoseconds()); + } + + if (min_timeout < 0ns) { + min_timeout = 0ns; + } + return min_timeout; + } + + /** + * @brief Executes all the ready timers in the heap + * These timers are refreshed and added back to the heap + * NOTE: may block indefinitely if the time for processing callbacks is longer than the timers period + */ + inline void execute_ready_timers() + { + while (peek(&head) == 0 && head->timer->is_ready()) { + head->timer->execute_callback(); + + head->refresh(clock); + remove_at(0); + push(head); + } + } + + inline void clear_all() + { + // Todo: Implement clear all timers. + } + + inline void remove_timer(rclcpp::TimerBase::SharedPtr timer) + { + // Todo: Implement + (void)timer; + } + +private: + struct TimerInternal + { + inline TimerInternal() + { + timer = nullptr; + expire_time = INT64_MAX; + } + + inline TimerInternal(rclcpp::TimerBase::SharedPtr t, rclcpp::Clock& clock) + { + timer = t; + refresh(clock); + } + + inline void refresh(rclcpp::Clock& clock) + { + expire_time = clock.now().nanoseconds() + timer->time_until_trigger().count(); + } + + bool operator < (const TimerInternal& t) const + { + return (timer->time_until_trigger() < t.timer->time_until_trigger()); + } + + rclcpp::TimerBase::SharedPtr timer; + int64_t expire_time; + }; + + using TimerInternalPtr = TimerInternal*; + + inline void push(TimerInternalPtr x) + { + size_t i = size++; + heap[i] = x; + while (i && (x->expire_time < heap[(i-1)/2]->expire_time)) { + heap[i] = heap[(i-1)/2]; + heap[(i-1)/2] = x; + i = (i-1)/2; + } + } + + inline void remove_at(size_t i) + { + TimerInternalPtr y = heap[--size]; + heap[i] = y; + + // Heapify upwards. + while (i > 0) { + size_t parent = (i-1)/2; + if (y->expire_time < heap[parent]->expire_time) { + heap[i] = heap[parent]; + heap[parent] = y; + i = parent; + } else { + break; + } + } + + // Heapify downwards + while (2*i + 1 < size) { + size_t hi = i; + size_t left = 2*i+1; + size_t right = left + 1; + if (y->expire_time > heap[left]->expire_time) { + hi = left; + } + if (right < size && (heap[hi]->expire_time > heap[right]->expire_time)) { + hi = right; + } + if (hi != i) { + heap[i] = heap[hi]; + heap[hi] = y; + i = hi; + } else { + break; + } + } + } + + inline int pop(TimerInternalPtr x) + { + if (size == 0) { + // The heap is empty, can't pop + return -1; + } + + x = heap[0]; + remove_at(0); + return 0; + } + + inline int peek(TimerInternalPtr* x) + { + if (size == 0) { + // The heap is empty, can't peek + return -1; + } + + *x = heap[0]; + return 0; + } + + inline int remove(TimerInternalPtr x) + { + size_t i; + for (i = 0; i < size; ++i) { + if (x == heap[i]) { + break; + } + } + if (i == size) { + return -1; + } + + remove_at(i); + return 0; + } + + // Vector to keep ownership of the timers + std::vector timers_storage; + // Vector of pointers to stored timers used to implement the priority queue + std::vector heap; + // Helper to access first element in the heap + TimerInternalPtr head; + // Current number of elements in the heap + size_t size; + // Clock to update expiration times and generate timeouts + rclcpp::Clock clock; +}; + +} +} From c9d395e323ce0fd00fe9c6b506761b1186345bd4 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 9 Oct 2020 15:18:29 +0100 Subject: [PATCH 031/168] Set interrupt guard condition callback --- rclcpp/src/rclcpp/executors/events_executor.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index bb2aa87b66..193c740795 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -36,6 +36,17 @@ EventsExecutor::EventsExecutor( timers.clear_all(); }); + // Set interrupt guard condition callback + rcl_ret_t ret = rcl_guard_condition_set_callback( + this, + &EventsExecutor::push_event, + entities_collector_.get(), + &interrupt_guard_condition_, + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set interrupt guard condition callback"); + } } EventsExecutor::~EventsExecutor() {} @@ -148,7 +159,7 @@ EventsExecutor::add_node( false /* Discard previous events */); if (ret != RCL_RET_OK) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); + throw std::runtime_error("Couldn't set node guard condition callback"); } } From 10f4074bca9a0b636652f26e97fc702c85bca31c Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Fri, 9 Oct 2020 14:56:26 +0100 Subject: [PATCH 032/168] create TimersManager class, implement spin() variants for EventsExecutor and fix unit tests Signed-off-by: Alberto Soragna --- rclcpp/CMakeLists.txt | 1 + rclcpp/include/rclcpp/executor.hpp | 2 +- rclcpp/include/rclcpp/executors.hpp | 1 + .../rclcpp/executors/events_executor.hpp | 43 ++- .../include/rclcpp/executors/timers_heap.hpp | 225 ------------- .../rclcpp/executors/timers_manager.hpp | 279 ++++++++++++++++ .../include/rclcpp/executors/timers_queue.hpp | 103 ------ .../src/rclcpp/executors/events_executor.cpp | 300 +++++++++++++----- .../src/rclcpp/executors/timers_manager.cpp | 170 ++++++++++ .../subscription_intra_process_base.cpp | 14 +- rclcpp/src/rclcpp/waitable.cpp | 4 + rclcpp/test/CMakeLists.txt | 2 +- .../test/rclcpp/executors/test_executors.cpp | 50 ++- 13 files changed, 751 insertions(+), 443 deletions(-) delete mode 100644 rclcpp/include/rclcpp/executors/timers_heap.hpp create mode 100644 rclcpp/include/rclcpp/executors/timers_manager.hpp delete mode 100644 rclcpp/include/rclcpp/executors/timers_queue.hpp create mode 100644 rclcpp/src/rclcpp/executors/timers_manager.cpp diff --git a/rclcpp/CMakeLists.txt b/rclcpp/CMakeLists.txt index f236e68511..309f667819 100644 --- a/rclcpp/CMakeLists.txt +++ b/rclcpp/CMakeLists.txt @@ -56,6 +56,7 @@ set(${PROJECT_NAME}_SRCS src/rclcpp/executors/single_threaded_executor.cpp src/rclcpp/executors/static_executor_entities_collector.cpp src/rclcpp/executors/static_single_threaded_executor.cpp + src/rclcpp/executors/timers_manager.cpp src/rclcpp/future_return_code.cpp src/rclcpp/graph_listener.cpp src/rclcpp/guard_condition.cpp diff --git a/rclcpp/include/rclcpp/executor.hpp b/rclcpp/include/rclcpp/executor.hpp index 628e05b58e..5936d31826 100644 --- a/rclcpp/include/rclcpp/executor.hpp +++ b/rclcpp/include/rclcpp/executor.hpp @@ -552,7 +552,7 @@ class Executor RCLCPP_DISABLE_COPY(Executor) RCLCPP_PUBLIC - void + virtual void spin_once_impl(std::chrono::nanoseconds timeout); typedef std::map #include +#include "rclcpp/executors/events_executor.hpp" #include "rclcpp/executors/multi_threaded_executor.hpp" #include "rclcpp/executors/single_threaded_executor.hpp" #include "rclcpp/executors/static_single_threaded_executor.hpp" diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 32ec15b3d2..b077eb727f 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -15,16 +15,17 @@ #ifndef RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ -#include #include +#include +#include + +#include "rcutils/event_types.h" #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" -#include "rclcpp/executors/timers_heap.hpp" +#include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node.hpp" -#include "rcutils/event_types.h" - namespace rclcpp { namespace executors @@ -46,6 +47,9 @@ class EventsExecutor : public rclcpp::Executor RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutor) /// Default constructor. See the default constructor for Executor. + /** + * \param[in] options Options used to configure the executor. + */ RCLCPP_PUBLIC explicit EventsExecutor( const rclcpp::ExecutorOptions & options = rclcpp::ExecutorOptions()); @@ -75,6 +79,10 @@ class EventsExecutor : public rclcpp::Executor void spin_some(std::chrono::nanoseconds max_duration) override; + RCLCPP_PUBLIC + void + spin_all(std::chrono::nanoseconds max_duration) override; + /// Add a node to the executor. /** * \sa rclcpp::Executor::add_node @@ -112,21 +120,30 @@ class EventsExecutor : public rclcpp::Executor remove_node(std::shared_ptr node_ptr, bool notify = true) override; protected: - /// Execute timers when ready + /// Waits for events and then executes them RCLCPP_PUBLIC void - spin_timers(bool spin_once); + handle_events(); - /// Execute events in the queue until is empty RCLCPP_PUBLIC void - execute_events(); + spin_once_impl(std::chrono::nanoseconds timeout) override; private: RCLCPP_DISABLE_COPY(EventsExecutor) EventsExecutorEntitiesCollector::SharedPtr entities_collector_; + /// Extract and execute events from the queue until it is empty + RCLCPP_PUBLIC + void + consume_all_events(std::queue &queue); + + // Execute a single event + RCLCPP_PUBLIC + void + execute_event(const EventQ &event); + // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. @@ -141,19 +158,19 @@ class EventsExecutor : public rclcpp::Executor { std::unique_lock lock(this_executor->event_queue_mutex_); - this_executor->event_queue.push(event); + this_executor->event_queue_.push(event); } // Notify that the event queue has some events in it. - this_executor->event_queue_cv.notify_one(); + this_executor->event_queue_cv_.notify_one(); } // Event queue members - std::queue event_queue; + std::queue event_queue_; std::mutex event_queue_mutex_; - std::condition_variable event_queue_cv; + std::condition_variable event_queue_cv_; // Timers heap manager - TimersHeap timers; + std::shared_ptr timers_manager_; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/timers_heap.hpp b/rclcpp/include/rclcpp/executors/timers_heap.hpp deleted file mode 100644 index af4520cc21..0000000000 --- a/rclcpp/include/rclcpp/executors/timers_heap.hpp +++ /dev/null @@ -1,225 +0,0 @@ -#include -#include - -#include -#include - -using namespace std::chrono_literals; - -namespace rclcpp -{ -namespace executors -{ - -struct TimersHeap -{ -public: - /** - * @brief Construct a new Timers Heap object - */ - TimersHeap() - { - clock = rclcpp::Clock(RCL_ROS_TIME); - size = 0; - } - - /** - * @brief Adds a new TimerBase to the heap - * @param timer the timer to be added - * @return int 0 if success, -1 if the heap is already full - */ - inline int add_timer(rclcpp::TimerBase::SharedPtr timer) - { - // Add timer to vector and order by expiration time - timers_storage.emplace_back(TimerInternal(timer, clock)); - std::sort(timers_storage.begin(), timers_storage.end()); - - // Clear heap as the pointers likely become invalid after the above emplace_back. - heap.clear(); - for (auto& t : timers_storage) { - heap.push_back(&t); - } - - size = heap.size(); - - return 0; - } - - /** - * @brief Get the time before the first timer in the heap expires - * - * @return std::chrono::nanoseconds to wait, 0 if the timer is already expired - */ - inline std::chrono::nanoseconds get_head_timeout() - { - auto min_timeout = std::chrono::nanoseconds::max(); - if (peek(&head) == 0) { - min_timeout = std::chrono::nanoseconds(head->expire_time - clock.now().nanoseconds()); - } - - if (min_timeout < 0ns) { - min_timeout = 0ns; - } - return min_timeout; - } - - /** - * @brief Executes all the ready timers in the heap - * These timers are refreshed and added back to the heap - * NOTE: may block indefinitely if the time for processing callbacks is longer than the timers period - */ - inline void execute_ready_timers() - { - while (peek(&head) == 0 && head->timer->is_ready()) { - head->timer->execute_callback(); - - head->refresh(clock); - remove_at(0); - push(head); - } - } - - inline void clear_all() - { - // Todo: Implement clear all timers. - } - - inline void remove_timer(rclcpp::TimerBase::SharedPtr timer) - { - // Todo: Implement - (void)timer; - } - -private: - struct TimerInternal - { - inline TimerInternal() - { - timer = nullptr; - expire_time = INT64_MAX; - } - - inline TimerInternal(rclcpp::TimerBase::SharedPtr t, rclcpp::Clock& clock) - { - timer = t; - refresh(clock); - } - - inline void refresh(rclcpp::Clock& clock) - { - expire_time = clock.now().nanoseconds() + timer->time_until_trigger().count(); - } - - bool operator < (const TimerInternal& t) const - { - return (timer->time_until_trigger() < t.timer->time_until_trigger()); - } - - rclcpp::TimerBase::SharedPtr timer; - int64_t expire_time; - }; - - using TimerInternalPtr = TimerInternal*; - - inline void push(TimerInternalPtr x) - { - size_t i = size++; - heap[i] = x; - while (i && (x->expire_time < heap[(i-1)/2]->expire_time)) { - heap[i] = heap[(i-1)/2]; - heap[(i-1)/2] = x; - i = (i-1)/2; - } - } - - inline void remove_at(size_t i) - { - TimerInternalPtr y = heap[--size]; - heap[i] = y; - - // Heapify upwards. - while (i > 0) { - size_t parent = (i-1)/2; - if (y->expire_time < heap[parent]->expire_time) { - heap[i] = heap[parent]; - heap[parent] = y; - i = parent; - } else { - break; - } - } - - // Heapify downwards - while (2*i + 1 < size) { - size_t hi = i; - size_t left = 2*i+1; - size_t right = left + 1; - if (y->expire_time > heap[left]->expire_time) { - hi = left; - } - if (right < size && (heap[hi]->expire_time > heap[right]->expire_time)) { - hi = right; - } - if (hi != i) { - heap[i] = heap[hi]; - heap[hi] = y; - i = hi; - } else { - break; - } - } - } - - inline int pop(TimerInternalPtr x) - { - if (size == 0) { - // The heap is empty, can't pop - return -1; - } - - x = heap[0]; - remove_at(0); - return 0; - } - - inline int peek(TimerInternalPtr* x) - { - if (size == 0) { - // The heap is empty, can't peek - return -1; - } - - *x = heap[0]; - return 0; - } - - inline int remove(TimerInternalPtr x) - { - size_t i; - for (i = 0; i < size; ++i) { - if (x == heap[i]) { - break; - } - } - if (i == size) { - return -1; - } - - remove_at(i); - return 0; - } - - // Vector to keep ownership of the timers - std::vector timers_storage; - // Vector of pointers to stored timers used to implement the priority queue - std::vector heap; - // Helper to access first element in the heap - TimerInternalPtr head; - // Current number of elements in the heap - size_t size; - // Clock to update expiration times and generate timeouts - rclcpp::Clock clock; -}; - -} -} diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp new file mode 100644 index 0000000000..3418b573fa --- /dev/null +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -0,0 +1,279 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXECUTORS__TIMERS_MANAGER_HPP_ +#define RCLCPP__EXECUTORS__TIMERS_MANAGER_HPP_ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "rclcpp/context.hpp" +#include "rclcpp/timer.hpp" + +namespace rclcpp +{ +namespace executors +{ + +/** + * @brief This class provides a way for storing and executing timer objects. + * It APIs to suit the needs of different applications and execution models. + * All public APIs provided by this class are thread-safe. + * + * Timers management + * This class provides APIs to add and remove timers. + * This class keeps ownership of the added timers through a shared pointer. + * Timers are kept ordered in a binary-heap priority queue. + * Calls to add/remove APIs will temporarily block the execution of the timers and + * will require to reorder the internal priority queue of timers. + * Because of this, they have a not-negligible impact on the performance. + * + * Timers execution + * The most efficient implementation consists in letting a TimersManager object + * to spawn a thread where timers are monitored and periodically executed. + * Besides this, other APIs allow to either execute a single timer or all the + * currently ready ones. + * This class assumes that the execute_callback API of the stored timer is never + * called by other entities, but can only be called from here. + * If this assumption is not respected, the heap property will be invalidated, + * so timers may be executed out of order. + * + */ +class TimersManager +{ +public: + RCLCPP_SMART_PTR_DEFINITIONS_NOT_COPYABLE(TimersManager) + + /** + * @brief Construct a new TimersManager object + */ + explicit TimersManager(std::shared_ptr context); + + /** + * @brief Destruct the object making sure to stop thread and release memory. + */ + ~TimersManager(); + + /** + * @brief Adds a new TimerBase to the storage. + * This object will keep ownership of the timer. + * @param timer the timer to be added + */ + void add_timer(rclcpp::TimerBase::SharedPtr timer); + + /** + * @brief Starts a thread that takes care of executing timers added to this object. + */ + void start(); + + /** + * @brief Stops the timers thread. + */ + void stop(); + + /** + * @brief Executes all the timers currently ready when the function is invoked + * while keeping the heap correctly sorted. + */ + void execute_ready_timers(); + + /** + * @brief Executes one ready timer if available + * + * @return true if there was a timer ready + */ + bool execute_head_timer(); + + /** + * @brief Get the amount of time before the next timer expires. + * + * @return std::chrono::nanoseconds to wait, + * the returned value could be negative if the timer is already expired + * or MAX_TIME if the heap is empty. + */ + std::chrono::nanoseconds get_head_timeout(); + + /** + * @brief Remove all the timers stored in the object. + */ + void clear_all(); + + /** + * @brief Remove a single timer stored in the object. + * @param timer the timer to remove. + */ + void remove_timer(rclcpp::TimerBase::SharedPtr timer); + + // This is what the TimersManager uses to denote a duration forever. + // We don't use std::chrono::nanoseconds::max because it will overflow. + // See https://en.cppreference.com/w/cpp/thread/condition_variable/wait_for + static constexpr std::chrono::nanoseconds MAX_TIME = std::chrono::hours(90); + +private: + RCLCPP_DISABLE_COPY(TimersManager) + + using TimerPtr = rclcpp::TimerBase::SharedPtr*; + + /** + * @brief Implements a loop that keeps executing ready timers. + * This function is executed in the timers thread. + */ + void run_timers(); + + /** + * @brief Get the amount of time before the next timer expires. + * This function is not thread safe, acquire a mutex before calling it. + * + * @return std::chrono::nanoseconds to wait, + * the returned value could be negative if the timer is already expired + * or MAX_TIME if the heap is empty. + */ + std::chrono::nanoseconds get_head_timeout_unsafe() + { + if (heap_.empty()) { + return MAX_TIME; + } + + return (*heap_[0])->time_until_trigger(); + } + + /** + * @brief Executes all the timers currently ready when the function is invoked + * while keeping the heap correctly sorted. + * This function is not thread safe, acquire a mutex before calling it. + */ + void execute_ready_timers_unsafe(); + + /** + * @brief Helper function that checks whether a timer was already ready + * at a specific timepoint + * @param tp the timepoint to check for + * @param timer a pointer to the timer to check for + * @return true if timer was ready at tp + */ + bool timer_was_ready_at_tp( + std::chrono::time_point tp, + TimerPtr timer) + { + // A ready timer will return a negative duration when calling time_until_trigger + auto time_ready = std::chrono::steady_clock::now() + (*timer)->time_until_trigger(); + return time_ready < tp; + } + + /** + * @brief Rebuilds the heap queue from the timers storage + * This function is meant to be called whenever something changes in the timers storage. + * This function is not thread safe, you need to acquire a mutex before calling it. + */ + void rebuild_heap(); + + /** + * @brief Add a new timer to the heap and sort it. + * @param x pointer to a timer owned by this object. + */ + void add_timer_to_heap(TimerPtr x) + { + size_t i = heap_.size(); // Position where we are going to add timer + heap_.push_back(x); + while (i && ((*x)->time_until_trigger() < (*heap_[(i-1)/2])->time_until_trigger())) { + heap_[i] = heap_[(i-1)/2]; + heap_[(i-1)/2] = x; + i = (i-1)/2; + } + } + + /** + * @brief Restore a valid heap after the root value is replaced. + * This function assumes that the value is increased, because if it was decreased + * then there is nothing to do. + */ + void restore_heap_root() + { + constexpr size_t start = 0; + TimerPtr updated_timer = heap_[0]; + + // Initialize to root and first left child + size_t i = 0; + size_t left_child = 1; + + // Swim up + while (left_child < heap_.size()) { + size_t right_child = left_child + 1; + if (right_child < heap_.size() && + (*heap_[left_child])->time_until_trigger() >= (*heap_[right_child])->time_until_trigger()) + { + left_child = right_child; + } + heap_[i] = heap_[left_child]; + i = left_child; + left_child = 2*i + 1; + } + + // Swim down + while (i > start) { + size_t parent = (i - 1)/2; + if ((*updated_timer)->time_until_trigger() < (*heap_[parent])->time_until_trigger()) { + heap_[i] = heap_[parent]; + i = parent; + continue; + } + break; + } + + heap_[i] = updated_timer; + } + + // Helper function to check the correctness of the heap. + void verify() + { + for (size_t i = 0; i < heap_.size()/2; ++i) { + size_t left = 2*i + 1; + if (left < heap_.size()) { + assert(((*heap_[left])->time_until_trigger().count() >= (*heap_[i])->time_until_trigger().count())); + } + size_t right = left + 1; + if (right < heap_.size()) { + assert(((*heap_[right])->time_until_trigger().count() >= (*heap_[i])->time_until_trigger().count())); + } + } + } + + // Thread used to run the timers monitoring and execution + std::thread timers_thread_; + // Protects access to timers + std::mutex timers_mutex_; + // Notifies the timers thread whenever timers are added/removed + std::condition_variable timers_cv_; + // Indicates whether the timers thread is currently running or requested to stop + std::atomic running_ {false}; + // Context of the parent executor + std::shared_ptr context_; + // Container to keep ownership of the timers + std::list timers_storage_; + // Vector of pointers to stored timers used to implement the priority queue + std::vector heap_; +}; + +} // namespace executors +} // namespace rclcpp + +#endif // RCLCPP__EXECUTORS__TIMERS_MANAGER_HPP_ diff --git a/rclcpp/include/rclcpp/executors/timers_queue.hpp b/rclcpp/include/rclcpp/executors/timers_queue.hpp deleted file mode 100644 index b758649d16..0000000000 --- a/rclcpp/include/rclcpp/executors/timers_queue.hpp +++ /dev/null @@ -1,103 +0,0 @@ -#include -#include - -#include -#include - -using namespace std::chrono_literals; - -namespace rclcpp -{ -namespace executors -{ - -struct TimersQueue -{ -public: - /** - * @brief Construct a new Timers Heap object - */ - TimersQueue() = default; - - /** - * @brief Adds a new TimerBase to the queue - * @param timer the timer to be added - * @return int 0 if success, -1 if the queue is already full - */ - inline void add_timer(rclcpp::TimerBase::SharedPtr timer) - { - timers_queue.emplace_back(std::move(timer)); - reorder_queue(); - } - - /** - * @brief Get the time before the first timer in the queue expires - * - * @return std::chrono::nanoseconds to wait, can return a negative number - * if the timer is already expired - */ - inline std::chrono::nanoseconds get_head_timeout() - { - auto min_timeout = std::chrono::nanoseconds::max(); - - auto head = timers_queue.front(); - - if (head != nullptr) { - min_timeout = head->time_until_trigger(); - } - - return min_timeout; - } - - /** - * @brief Executes all the ready timers in the queue - */ - inline void execute_ready_timers() - { - for (const auto &timer : timers_queue) { - if (!timer->is_ready()) { - break; - } - timer->execute_callback(); - } - reorder_queue(); - } - - inline void clear_all() - { - timers_queue.clear(); - } - - inline void remove_timer(rclcpp::TimerBase::SharedPtr timer) - { - for (auto it = timers_queue.begin(); it != timers_queue.end(); ++it) { - if((*it).get() == timer.get()) { - timers_queue.erase(it); - break; - } - } - } - -private: - - struct timer_less_than_comparison - { - inline bool operator() ( - const rclcpp::TimerBase::SharedPtr& timer1, - const rclcpp::TimerBase::SharedPtr& timer2) - { - return (timer1->time_until_trigger() < timer2->time_until_trigger()); - } - }; - - inline void reorder_queue() - { - std::sort(timers_queue.begin(), timers_queue.end(), timer_less_than_comparison()); - } - - // Ordered queue of timers - std::vector timers_queue; -}; - -} -} diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 193c740795..9bf592130c 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -12,14 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include + #include "rclcpp/executors/events_executor.hpp" +using namespace std::chrono_literals; + using rclcpp::executors::EventsExecutor; EventsExecutor::EventsExecutor( const rclcpp::ExecutorOptions & options) : rclcpp::Executor(options) { + timers_manager_ = std::make_shared(context_); entities_collector_ = std::make_shared(); // Set entities collector callbacks @@ -27,22 +34,36 @@ EventsExecutor::EventsExecutor( this, &EventsExecutor::push_event, [this](const rclcpp::TimerBase::SharedPtr & t) { - timers.add_timer(t); + timers_manager_->add_timer(t); }, [this](const rclcpp::TimerBase::SharedPtr & t) { - timers.remove_timer(t); + timers_manager_->remove_timer(t); }, [this]() { - timers.clear_all(); + timers_manager_->clear_all(); }); - // Set interrupt guard condition callback - rcl_ret_t ret = rcl_guard_condition_set_callback( - this, - &EventsExecutor::push_event, - entities_collector_.get(), - &interrupt_guard_condition_, - false /* Discard previous events */); + rcl_ret_t ret; + + // Set the global ctrl-c guard condition callback + ret = rcl_guard_condition_set_callback( + this, + &EventsExecutor::push_event, + entities_collector_.get(), + options.context->get_interrupt_guard_condition(&wait_set_), + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set ctrl-c guard condition callback"); + } + + // Set the executor interrupt guard condition callback + ret = rcl_guard_condition_set_callback( + this, + &EventsExecutor::push_event, + entities_collector_.get(), + &interrupt_guard_condition_, + false /* Discard previous events */); if (ret != RCL_RET_OK) { throw std::runtime_error("Couldn't set interrupt guard condition callback"); @@ -59,40 +80,153 @@ EventsExecutor::spin() } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - // Start timers thread - std::thread t_spin_timers(&EventsExecutor::spin_timers, this, false); - pthread_setname_np(t_spin_timers.native_handle(), "Timers"); + // When condition variable is notified, check this predicate to proceed + auto predicate = [this]() { return !event_queue_.empty(); }; - while (rclcpp::ok(context_) && spinning.load()) - { - execute_events(); + // Local event queue + std::queue local_event_queue; + + timers_manager_->start(); + + while (rclcpp::ok(context_) && spinning.load()) { + // Scope block for the mutex + { + std::unique_lock lock(event_queue_mutex_); + // We wait here until something has been pushed to the event queue + event_queue_cv_.wait(lock, predicate); + + // We got an event! Swap queues and execute events + std::swap(local_event_queue, event_queue_); + } + + this->consume_all_events(local_event_queue); } - t_spin_timers.join(); + timers_manager_->stop(); } -// Before calling spin_some void EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) { - (void)max_duration; + if (spinning.exchange(true)) { + throw std::runtime_error("spin_some() called while already spinning"); + } + RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + + // This function will wait until the first of the following events occur: + // - The input max_duration is elapsed + // - A timer triggers + // - An executor event is received and processed + + // Select the smallest between input max_duration and timer timeout + auto next_timer_timeout = timers_manager_->get_head_timeout(); + if (next_timer_timeout < max_duration) { + max_duration = next_timer_timeout; + } + + std::queue local_event_queue; + + { + // Wait until timeout or event + std::unique_lock lock(event_queue_mutex_); + event_queue_cv_.wait_for(lock, max_duration); + std::swap(local_event_queue, event_queue_); + } + + timers_manager_->execute_ready_timers(); + this->consume_all_events(local_event_queue); +} + +void +EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) +{ + if (max_duration <= 0ns) { + throw std::invalid_argument("max_duration must be positive"); + } - // Check, are we already spinning? if (spinning.exchange(true)) { throw std::runtime_error("spin_some() called while already spinning"); } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - // Start timers thread - std::thread t_spin_timers(&EventsExecutor::spin_timers, this, true); + std::queue local_event_queue; + + auto start = std::chrono::steady_clock::now(); + auto max_duration_not_elapsed = [max_duration, start]() { + auto elapsed_time = std::chrono::steady_clock::now() - start; + return elapsed_time < max_duration; + }; + + // Wait once + // Select the smallest between input timeout and timer timeout + auto next_timer_timeout = timers_manager_->get_head_timeout(); + if (next_timer_timeout < max_duration) { + max_duration = next_timer_timeout; + } - // Execute events and leave - if (rclcpp::ok(context_) && spinning.load()) { - execute_events(); + // Wait until timeout or event + std::unique_lock lock(event_queue_mutex_); + event_queue_cv_.wait_for(lock, max_duration); } - t_spin_timers.join(); + // Keep executing until work available or timeout expired + while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { + { + std::unique_lock lock(event_queue_mutex_); + std::swap(local_event_queue, event_queue_); + } + + bool ready_timer = timers_manager_->get_head_timeout() < 0ns; + bool has_events = !local_event_queue.empty(); + if (!ready_timer && !has_events) { + // Exit as there is no more work to do + break; + } + + // Execute all ready work + timers_manager_->execute_ready_timers(); + this->consume_all_events(local_event_queue); + } +} + +void +EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) +{ + // In this context a negative input timeout means no timeout + if (timeout < 0ns) { + timeout = timers_manager_->MAX_TIME; + } + + // Select the smallest between input timeout and timer timeout + auto next_timer_timeout = timers_manager_->get_head_timeout(); + if (next_timer_timeout < timeout) { + timeout = next_timer_timeout; + } + + EventQ event; + bool has_event = false; + + { + // Wait until timeout or event arrives + std::unique_lock lock(event_queue_mutex_); + event_queue_cv_.wait_for(lock, timeout); + + // Grab first event from queue if it exists + has_event = !event_queue_.empty(); + if (has_event) { + event = event_queue_.front(); + event_queue_.pop(); + } + } + + // If we wake up from the wait with an event, it means that it + // arrived before any of the timers expired. + if (has_event) { + this->execute_event(event); + } else { + timers_manager_->execute_head_timer(); + } } void @@ -110,11 +244,11 @@ EventsExecutor::add_node( if (!group || !group->can_be_taken_from().load()) { continue; } - // Add timers to timers heap + // Add timers to timers to timer manager group->find_timer_ptrs_if( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { - timers.add_timer(timer); + timers_manager_->add_timer(timer); } return false; }); @@ -187,23 +321,10 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) } void -EventsExecutor::spin_timers(bool spin_once) -{ - while (rclcpp::ok(context_) && spinning.load()) - { - timers.execute_ready_timers(); - if (spin_once) { - break; - } - std::this_thread::sleep_for(timers.get_head_timeout()); - } -} - -void -EventsExecutor::execute_events() +EventsExecutor::handle_events() { // When condition variable is notified, check this predicate to proceed - auto predicate = [this]() { return !event_queue.empty(); }; + auto predicate = [this]() { return !event_queue_.empty(); }; // Local event queue std::queue local_event_queue; @@ -212,53 +333,64 @@ EventsExecutor::execute_events() { std::unique_lock lock(event_queue_mutex_); // We wait here until something has been pushed to the event queue - event_queue_cv.wait(lock, predicate); + event_queue_cv_.wait(lock, predicate); // We got an event! Swap queues and execute events - swap(local_event_queue, event_queue); + std::swap(local_event_queue, event_queue_); } - // Execute events - do { - EventQ event = local_event_queue.front(); + this->consume_all_events(local_event_queue); +} - local_event_queue.pop(); +void +EventsExecutor::consume_all_events(std::queue &event_queue) +{ + while (!event_queue.empty()) { + EventQ event = event_queue.front(); + event_queue.pop(); - switch(event.type) + this->execute_event(event); + } +} + +void +EventsExecutor::execute_event(const EventQ &event) +{ + switch(event.type) { + case SUBSCRIPTION_EVENT: { - case SUBSCRIPTION_EVENT: - { - auto subscription = const_cast( - static_cast(event.entity)); - execute_subscription(subscription); - break; - } - - case SERVICE_EVENT: - { - auto service = const_cast( - static_cast(event.entity)); - execute_service(service); - break; - } - - case CLIENT_EVENT: - { - auto client = const_cast( - static_cast(event.entity)); - execute_client(client); - break; - } - - case GUARD_CONDITION_EVENT: - { - auto waitable = const_cast( - static_cast(event.entity)); - waitable->execute(); - break; - } - - default: break; + auto subscription = const_cast( + static_cast(event.entity)); + execute_subscription(subscription); + break; } - } while (!local_event_queue.empty()); + + case SERVICE_EVENT: + { + auto service = const_cast( + static_cast(event.entity)); + execute_service(service); + break; + } + + case CLIENT_EVENT: + { + auto client = const_cast( + static_cast(event.entity)); + execute_client(client); + break; + } + + case GUARD_CONDITION_EVENT: + { + auto waitable = const_cast( + static_cast(event.entity)); + waitable->execute(); + break; + } + + default: + throw std::runtime_error("EventsExecutor received unrecognized event"); + break; + } } diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp new file mode 100644 index 0000000000..9088221348 --- /dev/null +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -0,0 +1,170 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "rclcpp/executors/timers_manager.hpp" + +#include + +using rclcpp::executors::TimersManager; + +constexpr std::chrono::nanoseconds TimersManager::MAX_TIME; + +TimersManager::TimersManager(std::shared_ptr context) +{ + context_ = context; +} + +TimersManager::~TimersManager() +{ + // Make sure timers thread is stopped before destroying this object + this->stop(); + // Remove all timers + this->clear_all(); +} + +void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) +{ + { + std::unique_lock lock(timers_mutex_); + + // Make sure that the provided timer is not already in the timers storage + if (std::find(timers_storage_.begin(), timers_storage_.end(), timer) != timers_storage_.end()) { + return; + } + + // Store ownership of timer and add it to heap + timers_storage_.emplace_back(timer); + this->add_timer_to_heap(&(timers_storage_.back())); + } + + // Notify that a timer has been added to the heap + timers_cv_.notify_one(); + + //verify(); +} + +void TimersManager::start() +{ + // Make sure that the thread is not already running + if (running_) { + throw std::runtime_error("TimersManager::start() can't start timers thread as already running"); + } + + timers_thread_ = std::thread(&TimersManager::run_timers, this); + pthread_setname_np(timers_thread_.native_handle(), "TimersManager"); +} + +void TimersManager::stop() +{ + // Notify stop to timers thread + running_ = false; + timers_cv_.notify_one(); + + // Join timers thread if it's running + if (timers_thread_.joinable()) { + timers_thread_.join(); + } +} + +std::chrono::nanoseconds TimersManager::get_head_timeout() +{ + std::unique_lock lock(timers_mutex_); + return this->get_head_timeout_unsafe(); +} + +void TimersManager::execute_ready_timers() +{ + std::unique_lock lock(timers_mutex_); + this->execute_ready_timers_unsafe(); +} + +void TimersManager::execute_ready_timers_unsafe() +{ + if (heap_.empty()) { + return; + } + + // Keep executing timers until they are read and they were already ready when we started. + // The second check prevents this function from blocking indefinitely if the + // time required for executing the timers is longer than their period. + + auto start = std::chrono::steady_clock::now(); + TimerPtr head = heap_.front(); + while ((*head)->is_ready() && timer_was_ready_at_tp(start, head)) { + (*head)->execute_callback(); + this->restore_heap_root(); + //verify(); + } +} + +bool TimersManager::execute_head_timer() +{ + std::unique_lock lock(timers_mutex_); + + if (heap_.empty()) { + return false; + } + + TimerPtr head = heap_.front(); + if ((*head)->is_ready()) { + (*head)->execute_callback(); + this->restore_heap_root(); + return true; + } else { + return false; + } +} + +void TimersManager::run_timers() +{ + running_ = true; + while (rclcpp::ok(context_) && running_) { + std::unique_lock lock(timers_mutex_); + auto time_to_sleep = this->get_head_timeout_unsafe(); + timers_cv_.wait_for(lock, time_to_sleep); + this->execute_ready_timers_unsafe(); + } + + running_ = false; +} + +void TimersManager::clear_all() +{ + std::unique_lock lock(timers_mutex_); + heap_.clear(); + timers_storage_.clear(); +} + +void TimersManager::remove_timer(rclcpp::TimerBase::SharedPtr timer) +{ + std::unique_lock lock(timers_mutex_); + + // Make sure that we are currently storing the provided timer before proceeding + auto it = std::find(timers_storage_.begin(), timers_storage_.end(), timer); + if (it == timers_storage_.end()) { + return; + } + + // Remove timer from the storage and rebuild heap + timers_storage_.erase(it); + this->rebuild_heap(); +} + +void TimersManager::rebuild_heap() +{ + heap_.clear(); + for (auto& t : timers_storage_) { + this->add_timer_to_heap(&t); + } +} diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 5bb7ad1ebf..c3df2b5004 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -39,15 +39,15 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_guard_condition_callback( - void * executor_context, - Event_callback executor_callback) const + void * executor_context, + Event_callback executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_callback( - executor_context, - executor_callback, - this, - &gc_, - true /*Use previous events*/); + executor_context, + executor_callback, + this, + &gc_, + true /*Use previous events*/); if (RCL_RET_OK != ret) { throw std::runtime_error(std::string("Couldn't set guard condition callback")); diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 75c5744868..7764e4b818 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include + #include "rclcpp/waitable.hpp" using rclcpp::Waitable; @@ -65,4 +67,6 @@ Waitable::set_guard_condition_callback( { (void)executor_context; (void)executor_callback; + + throw std::runtime_error("Custom waitables should override set_guard_condition_callback() to use events executor"); } diff --git a/rclcpp/test/CMakeLists.txt b/rclcpp/test/CMakeLists.txt index 46f2fd57ec..98facf2f08 100644 --- a/rclcpp/test/CMakeLists.txt +++ b/rclcpp/test/CMakeLists.txt @@ -532,7 +532,7 @@ ament_add_gtest( test_executors rclcpp/executors/test_executors.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}" - TIMEOUT 180) + TIMEOUT 20) if(TARGET test_executors) ament_target_dependencies(test_executors "rcl" diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index dd306ba938..9ac6934e06 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -90,7 +90,8 @@ using ExecutorTypes = ::testing::Types< rclcpp::executors::SingleThreadedExecutor, rclcpp::executors::MultiThreadedExecutor, - rclcpp::executors::StaticSingleThreadedExecutor>; + rclcpp::executors::StaticSingleThreadedExecutor, + rclcpp::executors::EventsExecutor>; class ExecutorTypeNames { @@ -111,6 +112,10 @@ class ExecutorTypeNames return "StaticSingleThreadedExecutor"; } + if (std::is_same()) { + return "EventsExecutor"; + } + return ""; } }; @@ -123,8 +128,8 @@ TYPED_TEST_CASE(TestExecutors, ExecutorTypes, ExecutorTypeNames); // https://github.com/ros2/rclcpp/issues/1219 using StandardExecutors = ::testing::Types< - rclcpp::executors::SingleThreadedExecutor, - rclcpp::executors::MultiThreadedExecutor>; + rclcpp::executors::EventsExecutor, + rclcpp::executors::SingleThreadedExecutor>; TYPED_TEST_CASE(TestExecutorsStable, StandardExecutors, ExecutorTypeNames); // Make sure that executors detach from nodes when destructing @@ -143,6 +148,7 @@ TYPED_TEST(TestExecutors, detachOnDestruction) { // Make sure that the executor can automatically remove expired nodes correctly // Currently fails for StaticSingleThreadedExecutor so it is being skipped, see: // https://github.com/ros2/rclcpp/issues/1231 +// This test is also flaky for the MultiThreadedExecutor TYPED_TEST(TestExecutorsStable, addTemporaryNode) { using ExecutorType = TypeParam; ExecutorType executor; @@ -414,10 +420,12 @@ class TestWaitable : public rclcpp::Waitable add_to_wait_set(rcl_wait_set_t * wait_set) override { rcl_ret_t ret = rcl_wait_set_add_guard_condition(wait_set, &gc_, NULL); - if (RCL_RET_OK != ret) { - return false; - } - ret = rcl_trigger_guard_condition(&gc_); + return RCL_RET_OK == ret; + } + + bool trigger() + { + rcl_ret_t ret = rcl_trigger_guard_condition(&gc_); return RCL_RET_OK == ret; } @@ -432,7 +440,7 @@ class TestWaitable : public rclcpp::Waitable execute() override { count_++; - std::this_thread::sleep_for(1ms); + std::this_thread::sleep_for(3ms); } size_t @@ -444,6 +452,23 @@ class TestWaitable : public rclcpp::Waitable return count_; } + void + set_guard_condition_callback( + void * executor_context, + Event_callback executor_callback) const override + { + rcl_ret_t ret = rcl_guard_condition_set_callback( + executor_context, + executor_callback, + this, + &gc_, + true /*Use previous events*/); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } + } + private: size_t count_ = 0; rcl_guard_condition_t gc_; @@ -473,6 +498,7 @@ TYPED_TEST(TestExecutorsStable, spinAll) { !spin_exited && (std::chrono::steady_clock::now() - start < 1s)) { + my_waitable->trigger(); this->publisher->publish(test_msgs::msg::Empty()); std::this_thread::sleep_for(1ms); } @@ -506,14 +532,20 @@ TYPED_TEST(TestExecutorsStable, spinSome) { spin_exited = true; }); + // Give some time for executor to start spinning + // otherwise when it will start looking for work to do it will already find + // more than 1 notification + std::this_thread::sleep_for(10ms); + // Do some work until sufficient calls to the waitable occur, but keep going until either // count becomes too large, spin exits, or the 1 second timeout completes. auto start = std::chrono::steady_clock::now(); while ( - my_waitable->get_count() <= 1 && + my_waitable->get_count() <= 10 && !spin_exited && (std::chrono::steady_clock::now() - start < 1s)) { + my_waitable->trigger(); this->publisher->publish(test_msgs::msg::Empty()); std::this_thread::sleep_for(1ms); } From f25d4c64b681d6aa811831c705afb3de7956efd5 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 12 Oct 2020 13:28:03 +0100 Subject: [PATCH 033/168] Rename EventQ -> ExecutorEvent --- .../include/rclcpp/executors/events_executor.hpp | 8 ++++---- rclcpp/src/rclcpp/executors/events_executor.cpp | 16 ++++++++-------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index b077eb727f..f3d3ddfe35 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -137,18 +137,18 @@ class EventsExecutor : public rclcpp::Executor /// Extract and execute events from the queue until it is empty RCLCPP_PUBLIC void - consume_all_events(std::queue &queue); + consume_all_events(std::queue &queue); // Execute a single event RCLCPP_PUBLIC void - execute_event(const EventQ &event); + execute_event(const ExecutorEvent &event); // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. static void - push_event(const void * executor_ptr, EventQ event) + push_event(const void * executor_ptr, ExecutorEvent event) { // Cast executor_ptr to this (need to remove constness) auto this_executor = const_cast( @@ -165,7 +165,7 @@ class EventsExecutor : public rclcpp::Executor } // Event queue members - std::queue event_queue_; + std::queue event_queue_; std::mutex event_queue_mutex_; std::condition_variable event_queue_cv_; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 9bf592130c..9bf0b4e0aa 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -84,7 +84,7 @@ EventsExecutor::spin() auto predicate = [this]() { return !event_queue_.empty(); }; // Local event queue - std::queue local_event_queue; + std::queue local_event_queue; timers_manager_->start(); @@ -124,7 +124,7 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) max_duration = next_timer_timeout; } - std::queue local_event_queue; + std::queue local_event_queue; { // Wait until timeout or event @@ -149,7 +149,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - std::queue local_event_queue; + std::queue local_event_queue; auto start = std::chrono::steady_clock::now(); auto max_duration_not_elapsed = [max_duration, start]() { @@ -204,7 +204,7 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) timeout = next_timer_timeout; } - EventQ event; + ExecutorEvent event; bool has_event = false; { @@ -327,7 +327,7 @@ EventsExecutor::handle_events() auto predicate = [this]() { return !event_queue_.empty(); }; // Local event queue - std::queue local_event_queue; + std::queue local_event_queue; // Scope block for the mutex { @@ -343,10 +343,10 @@ EventsExecutor::handle_events() } void -EventsExecutor::consume_all_events(std::queue &event_queue) +EventsExecutor::consume_all_events(std::queue &event_queue) { while (!event_queue.empty()) { - EventQ event = event_queue.front(); + ExecutorEvent event = event_queue.front(); event_queue.pop(); this->execute_event(event); @@ -354,7 +354,7 @@ EventsExecutor::consume_all_events(std::queue &event_queue) } void -EventsExecutor::execute_event(const EventQ &event) +EventsExecutor::execute_event(const ExecutorEvent &event) { switch(event.type) { case SUBSCRIPTION_EVENT: From a8feedc933fcebb854da25a11429d5e5bc141253 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 12 Oct 2020 13:43:04 +0100 Subject: [PATCH 034/168] rename set_guard_condition_callback -> set_callback --- .../rclcpp/experimental/subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 +- .../rclcpp/executors/events_executor_entities_collector.cpp | 4 ++-- rclcpp/src/rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 4 ++-- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index e75fe5d65c..ba443c830f 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -72,7 +72,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void - set_guard_condition_callback( + set_callback( void * executor_context, Event_callback executor_callback) const override; diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index e39e01b294..aa19af1ff9 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -168,7 +168,7 @@ class Waitable RCLCPP_PUBLIC virtual void - set_guard_condition_callback( + set_callback( void * executor_context, Event_callback executor_callback) const; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 9bf0b4e0aa..7121d43f36 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -277,7 +277,7 @@ EventsExecutor::add_node( group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_guard_condition_callback(this, &EventsExecutor::push_event); + waitable->set_callback(this, &EventsExecutor::push_event); } return false; }); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 1379cdb75e..329cc8af9d 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -131,7 +131,7 @@ EventsExecutorEntitiesCollector::remove_node( group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_guard_condition_callback(nullptr, nullptr); + waitable->set_callback(nullptr, nullptr); } return false; }); @@ -190,7 +190,7 @@ EventsExecutorEntitiesCollector::set_entities_callbacks() group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_guard_condition_callback(executor_context_, executor_callback_); + waitable->set_callback(executor_context_, executor_callback_); } return false; }); diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index c3df2b5004..c0df487dc2 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -38,7 +38,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const } void -SubscriptionIntraProcessBase::set_guard_condition_callback( +SubscriptionIntraProcessBase::set_callback( void * executor_context, Event_callback executor_callback) const { diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 7764e4b818..72c64af365 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -61,12 +61,12 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) } void -Waitable::set_guard_condition_callback( +Waitable::set_callback( void * executor_context, Event_callback executor_callback) const { (void)executor_context; (void)executor_callback; - throw std::runtime_error("Custom waitables should override set_guard_condition_callback() to use events executor"); + throw std::runtime_error("Custom waitables should override set_callback() to use events executor"); } diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 353186c683..846a780109 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -453,7 +453,7 @@ class TestWaitable : public rclcpp::Waitable } void - set_guard_condition_callback( + set_callback( void * executor_context, Event_callback executor_callback) const override { From 43a6088f9ce13859e351a447f385dcf6f05ff7ed Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Mon, 12 Oct 2020 13:06:20 +0000 Subject: [PATCH 035/168] remove unused handle_events() function --- .../rclcpp/executors/events_executor.hpp | 5 ----- .../src/rclcpp/executors/events_executor.cpp | 22 ------------------- 2 files changed, 27 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index b077eb727f..d88e37bac4 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -120,11 +120,6 @@ class EventsExecutor : public rclcpp::Executor remove_node(std::shared_ptr node_ptr, bool notify = true) override; protected: - /// Waits for events and then executes them - RCLCPP_PUBLIC - void - handle_events(); - RCLCPP_PUBLIC void spin_once_impl(std::chrono::nanoseconds timeout) override; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 9bf592130c..5db354dd27 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -320,28 +320,6 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) this->remove_node(node_ptr->get_node_base_interface(), notify); } -void -EventsExecutor::handle_events() -{ - // When condition variable is notified, check this predicate to proceed - auto predicate = [this]() { return !event_queue_.empty(); }; - - // Local event queue - std::queue local_event_queue; - - // Scope block for the mutex - { - std::unique_lock lock(event_queue_mutex_); - // We wait here until something has been pushed to the event queue - event_queue_cv_.wait(lock, predicate); - - // We got an event! Swap queues and execute events - std::swap(local_event_queue, event_queue_); - } - - this->consume_all_events(local_event_queue); -} - void EventsExecutor::consume_all_events(std::queue &event_queue) { From 54054f675c9d25f79103cf9ff90210d7be72d4c4 Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Mon, 12 Oct 2020 13:19:55 +0000 Subject: [PATCH 036/168] add missing predicate to event_queue_cv wait --- .../src/rclcpp/executors/events_executor.cpp | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 9bf592130c..bc47db6e94 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -81,7 +81,7 @@ EventsExecutor::spin() RCLCPP_SCOPE_EXIT(this->spinning.store(false);); // When condition variable is notified, check this predicate to proceed - auto predicate = [this]() { return !event_queue_.empty(); }; + auto has_event_predicate = [this]() { return !event_queue_.empty(); }; // Local event queue std::queue local_event_queue; @@ -93,7 +93,7 @@ EventsExecutor::spin() { std::unique_lock lock(event_queue_mutex_); // We wait here until something has been pushed to the event queue - event_queue_cv_.wait(lock, predicate); + event_queue_cv_.wait(lock, has_event_predicate); // We got an event! Swap queues and execute events std::swap(local_event_queue, event_queue_); @@ -118,18 +118,21 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) // - A timer triggers // - An executor event is received and processed + // When condition variable is notified, check this predicate to proceed + auto has_event_predicate = [this]() { return !event_queue_.empty(); }; + + std::queue local_event_queue; + // Select the smallest between input max_duration and timer timeout auto next_timer_timeout = timers_manager_->get_head_timeout(); if (next_timer_timeout < max_duration) { max_duration = next_timer_timeout; } - std::queue local_event_queue; - { // Wait until timeout or event std::unique_lock lock(event_queue_mutex_); - event_queue_cv_.wait_for(lock, max_duration); + event_queue_cv_.wait_for(lock, max_duration, has_event_predicate); std::swap(local_event_queue, event_queue_); } @@ -149,6 +152,9 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + // When condition variable is notified, check this predicate to proceed + auto has_event_predicate = [this]() { return !event_queue_.empty(); }; + std::queue local_event_queue; auto start = std::chrono::steady_clock::now(); @@ -167,7 +173,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) { // Wait until timeout or event std::unique_lock lock(event_queue_mutex_); - event_queue_cv_.wait_for(lock, max_duration); + event_queue_cv_.wait_for(lock, max_duration, has_event_predicate); } // Keep executing until work available or timeout expired @@ -204,13 +210,16 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) timeout = next_timer_timeout; } + // When condition variable is notified, check this predicate to proceed + auto has_event_predicate = [this]() { return !event_queue_.empty(); }; + EventQ event; bool has_event = false; { // Wait until timeout or event arrives std::unique_lock lock(event_queue_mutex_); - event_queue_cv_.wait_for(lock, timeout); + event_queue_cv_.wait_for(lock, timeout, has_event_predicate); // Grab first event from queue if it exists has_event = !event_queue_.empty(); From 78ac3242b4a576a408e527f4517e5444f05510d6 Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Mon, 12 Oct 2020 15:09:04 +0100 Subject: [PATCH 037/168] Update events_executor.cpp Fix code style errors --- rclcpp/src/rclcpp/executors/events_executor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index bc47db6e94..b49d97a757 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -81,7 +81,7 @@ EventsExecutor::spin() RCLCPP_SCOPE_EXIT(this->spinning.store(false);); // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() { return !event_queue_.empty(); }; + auto has_event_predicate = [this]() {return !event_queue_.empty();}; // Local event queue std::queue local_event_queue; @@ -119,7 +119,7 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) // - An executor event is received and processed // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() { return !event_queue_.empty(); }; + auto has_event_predicate = [this]() {return !event_queue_.empty();}; std::queue local_event_queue; @@ -153,7 +153,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) RCLCPP_SCOPE_EXIT(this->spinning.store(false);); // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() { return !event_queue_.empty(); }; + auto has_event_predicate = [this]() {return !event_queue_.empty();}; std::queue local_event_queue; @@ -211,7 +211,7 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) } // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() { return !event_queue_.empty(); }; + auto has_event_predicate = [this]() {return !event_queue_.empty();}; EventQ event; bool has_event = false; From b91edeac52ec6d51adb99b19ca39f7cefd5a7463 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 12 Oct 2020 15:27:08 +0100 Subject: [PATCH 038/168] Rename to set_events_executor_callback --- rclcpp/include/rclcpp/client.hpp | 2 +- .../subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 4 ++-- .../src/rclcpp/executors/events_executor.cpp | 14 +++++++------- .../events_executor_entities_collector.cpp | 18 +++++++++--------- rclcpp/src/rclcpp/service.cpp | 4 ++-- rclcpp/src/rclcpp/subscription_base.cpp | 4 ++-- .../rclcpp/subscription_intra_process_base.cpp | 4 ++-- rclcpp/src/rclcpp/waitable.cpp | 4 ++-- 12 files changed, 31 insertions(+), 31 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index ab6ccf91db..625030e8b2 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -152,7 +152,7 @@ class ClientBase RCLCPP_PUBLIC void - set_callback(const void * executor_context, Event_callback executor_callback) const; + set_events_executor_callback(const void * executor_context, Event_callback executor_callback) const; protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index ba443c830f..fbeada7432 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -72,7 +72,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void - set_callback( + set_events_executor_callback( void * executor_context, Event_callback executor_callback) const override; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index fc0ebe6f8e..e6690c1ef6 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -123,7 +123,7 @@ class ServiceBase RCLCPP_PUBLIC void - set_callback(const void * executor_context, Event_callback executor_callback) const; + set_events_executor_callback(const void * executor_context, Event_callback executor_callback) const; protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 8e82f430d2..a5cd265cb4 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -265,7 +265,7 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void - set_callback(const void * executor_context, Event_callback executor_callback) const; + set_events_executor_callback(const void * executor_context, Event_callback executor_callback) const; protected: template diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index aa19af1ff9..4aa9883b47 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -168,7 +168,7 @@ class Waitable RCLCPP_PUBLIC virtual void - set_callback( + set_events_executor_callback( void * executor_context, Event_callback executor_callback) const; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index adae1e250a..626aa92945 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -200,11 +200,11 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ClientBase::set_callback( +ClientBase::set_events_executor_callback( const void * executor_context, Event_callback executor_callback) const { - rcl_ret_t ret = rcl_client_set_callback( + rcl_ret_t ret = rcl_client_set_events_executor_callback( executor_context, executor_callback, this, diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 7121d43f36..0de36e491a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -46,7 +46,7 @@ EventsExecutor::EventsExecutor( rcl_ret_t ret; // Set the global ctrl-c guard condition callback - ret = rcl_guard_condition_set_callback( + ret = rcl_guard_condition_set_events_executor_callback( this, &EventsExecutor::push_event, entities_collector_.get(), @@ -58,7 +58,7 @@ EventsExecutor::EventsExecutor( } // Set the executor interrupt guard condition callback - ret = rcl_guard_condition_set_callback( + ret = rcl_guard_condition_set_events_executor_callback( this, &EventsExecutor::push_event, entities_collector_.get(), @@ -256,28 +256,28 @@ EventsExecutor::add_node( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_callback(this, &EventsExecutor::push_event); + subscription->set_events_executor_callback(this, &EventsExecutor::push_event); } return false; }); group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_callback(this, &EventsExecutor::push_event); + service->set_events_executor_callback(this, &EventsExecutor::push_event); } return false; }); group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_callback(this, &EventsExecutor::push_event); + client->set_events_executor_callback(this, &EventsExecutor::push_event); } return false; }); group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_callback(this, &EventsExecutor::push_event); + waitable->set_events_executor_callback(this, &EventsExecutor::push_event); } return false; }); @@ -285,7 +285,7 @@ EventsExecutor::add_node( // Set node's guard condition callback, so if new entities are added while // spinning we can set their callback. - rcl_ret_t ret = rcl_guard_condition_set_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( this, &EventsExecutor::push_event, entities_collector_.get(), diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 329cc8af9d..ac9481da59 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -85,7 +85,7 @@ EventsExecutorEntitiesCollector::remove_node( bool matched = (node_it->lock() == node_ptr); if (matched) { // Node found: unset its entities callbacks - rcl_ret_t ret = rcl_guard_condition_set_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( nullptr, nullptr, nullptr, node_ptr->get_notify_guard_condition(), false); @@ -110,28 +110,28 @@ EventsExecutorEntitiesCollector::remove_node( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_callback(nullptr, nullptr); + subscription->set_events_executor_callback(nullptr, nullptr); } return false; }); group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_callback(nullptr, nullptr); + service->set_events_executor_callback(nullptr, nullptr); } return false; }); group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_callback(nullptr, nullptr); + client->set_events_executor_callback(nullptr, nullptr); } return false; }); group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_callback(nullptr, nullptr); + waitable->set_events_executor_callback(nullptr, nullptr); } return false; }); @@ -169,28 +169,28 @@ EventsExecutorEntitiesCollector::set_entities_callbacks() group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_callback(executor_context_, executor_callback_); + subscription->set_events_executor_callback(executor_context_, executor_callback_); } return false; }); group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_callback(executor_context_, executor_callback_); + service->set_events_executor_callback(executor_context_, executor_callback_); } return false; }); group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_callback(executor_context_, executor_callback_); + client->set_events_executor_callback(executor_context_, executor_callback_); } return false; }); group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_callback(executor_context_, executor_callback_); + waitable->set_events_executor_callback(executor_context_, executor_callback_); } return false; }); diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 3f1302c875..a87fdefc24 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -86,11 +86,11 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ServiceBase::set_callback( +ServiceBase::set_events_executor_callback( const void * executor_context, Event_callback executor_callback) const { - rcl_ret_t ret = rcl_service_set_callback( + rcl_ret_t ret = rcl_service_set_events_executor_callback( executor_context, executor_callback, this, diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index d9c72812a3..850701cc7c 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -290,11 +290,11 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( } void -SubscriptionBase::set_callback( +SubscriptionBase::set_events_executor_callback( const void * executor_context, Event_callback executor_callback) const { - rcl_ret_t ret = rcl_subscription_set_callback( + rcl_ret_t ret = rcl_subscription_set_events_executor_callback( executor_context, executor_callback, this, diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index c0df487dc2..b245b569d7 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -38,11 +38,11 @@ SubscriptionIntraProcessBase::get_actual_qos() const } void -SubscriptionIntraProcessBase::set_callback( +SubscriptionIntraProcessBase::set_events_executor_callback( void * executor_context, Event_callback executor_callback) const { - rcl_ret_t ret = rcl_guard_condition_set_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor_context, executor_callback, this, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 72c64af365..90a91f2f05 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -61,12 +61,12 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) } void -Waitable::set_callback( +Waitable::set_events_executor_callback( void * executor_context, Event_callback executor_callback) const { (void)executor_context; (void)executor_callback; - throw std::runtime_error("Custom waitables should override set_callback() to use events executor"); + throw std::runtime_error("Custom waitables should override set_events_executor_callback() to use events executor"); } From caf353224dd346a214afc885133c2705390d6492 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 12 Oct 2020 16:16:59 +0100 Subject: [PATCH 039/168] Rename to ExecutorEventCallback --- rclcpp/include/rclcpp/client.hpp | 2 +- rclcpp/include/rclcpp/executors/events_executor.hpp | 2 +- .../rclcpp/executors/events_executor_entities_collector.hpp | 4 ++-- .../rclcpp/experimental/subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 +- .../rclcpp/executors/events_executor_entities_collector.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- rclcpp/src/rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 625030e8b2..129f7209af 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -152,7 +152,7 @@ class ClientBase RCLCPP_PUBLIC void - set_events_executor_callback(const void * executor_context, Event_callback executor_callback) const; + set_events_executor_callback(const void * executor_context, ExecutorEventCallback executor_callback) const; protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index f3d3ddfe35..a5ba7c51e1 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -19,7 +19,7 @@ #include #include -#include "rcutils/event_types.h" +#include "rcutils/executor_event_types.h" #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 96540f74ce..8e4f57b3ac 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -45,7 +45,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable void init( void * executor_context, - Event_callback executor_callback, + ExecutorEventCallback executor_callback, TimerFn push_timer, TimerFn clear_timer, ClearTimersFn clear_all_timers); @@ -91,7 +91,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable void * executor_context_ = nullptr; /// Events callback - Event_callback executor_callback_ = nullptr; + ExecutorEventCallback executor_callback_ = nullptr; /// Function pointers to push and clear timers from the timers heap TimerFn push_timer_ = nullptr; diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index fbeada7432..a961ef23c8 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -74,7 +74,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable void set_events_executor_callback( void * executor_context, - Event_callback executor_callback) const override; + ExecutorEventCallback executor_callback) const override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index e6690c1ef6..eb5fd197f2 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -123,7 +123,7 @@ class ServiceBase RCLCPP_PUBLIC void - set_events_executor_callback(const void * executor_context, Event_callback executor_callback) const; + set_events_executor_callback(const void * executor_context, ExecutorEventCallback executor_callback) const; protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index a5cd265cb4..320895bd33 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -265,7 +265,7 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void - set_events_executor_callback(const void * executor_context, Event_callback executor_callback) const; + set_events_executor_callback(const void * executor_context, ExecutorEventCallback executor_callback) const; protected: template diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 4aa9883b47..73ff886f44 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -170,7 +170,7 @@ class Waitable void set_events_executor_callback( void * executor_context, - Event_callback executor_callback) const; + ExecutorEventCallback executor_callback) const; private: std::atomic in_use_by_wait_set_{false}; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 626aa92945..f4cfc40b31 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -202,7 +202,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( const void * executor_context, - Event_callback executor_callback) const + ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_client_set_events_executor_callback( executor_context, diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 0de36e491a..63be1570b8 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -381,7 +381,7 @@ EventsExecutor::execute_event(const ExecutorEvent &event) break; } - case GUARD_CONDITION_EVENT: + case WAITABLE_EVENT: { auto waitable = const_cast( static_cast(event.entity)); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index ac9481da59..92e8cd38ce 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -33,7 +33,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() void EventsExecutorEntitiesCollector::init( void * executor_context, - Event_callback executor_callback, + ExecutorEventCallback executor_callback, TimerFn push_timer, TimerFn clear_timer, ClearTimersFn clear_all_timers) diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index a87fdefc24..a178314e91 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -88,7 +88,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( const void * executor_context, - Event_callback executor_callback) const + ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_service_set_events_executor_callback( executor_context, diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 850701cc7c..073168748e 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -292,7 +292,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( const void * executor_context, - Event_callback executor_callback) const + ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_subscription_set_events_executor_callback( executor_context, diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index b245b569d7..067cadf7eb 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -40,7 +40,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( void * executor_context, - Event_callback executor_callback) const + ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor_context, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 90a91f2f05..f354145a79 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -63,7 +63,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( void * executor_context, - Event_callback executor_callback) const + ExecutorEventCallback executor_callback) const { (void)executor_context; (void)executor_callback; diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 846a780109..e3ed8d47ab 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -455,7 +455,7 @@ class TestWaitable : public rclcpp::Waitable void set_callback( void * executor_context, - Event_callback executor_callback) const override + ExecutorEventCallback executor_callback) const override { rcl_ret_t ret = rcl_guard_condition_set_callback( executor_context, From 89721e9384925692d0e21b3a16f293dceab1b911 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 12 Oct 2020 16:56:27 +0100 Subject: [PATCH 040/168] fix linter errors Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/executor.hpp | 6 +- .../rclcpp/executors/events_executor.hpp | 8 +- .../rclcpp/executors/timers_manager.hpp | 23 ++--- rclcpp/src/rclcpp/client.cpp | 12 +-- rclcpp/src/rclcpp/executor.cpp | 6 +- .../src/rclcpp/executors/events_executor.cpp | 89 ++++++++++--------- .../events_executor_entities_collector.cpp | 8 +- .../src/rclcpp/executors/timers_manager.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 12 +-- rclcpp/src/rclcpp/subscription_base.cpp | 12 +-- .../subscription_intra_process_base.cpp | 4 +- rclcpp/src/rclcpp/waitable.cpp | 7 +- 12 files changed, 98 insertions(+), 91 deletions(-) diff --git a/rclcpp/include/rclcpp/executor.hpp b/rclcpp/include/rclcpp/executor.hpp index 5936d31826..c9994a49a1 100644 --- a/rclcpp/include/rclcpp/executor.hpp +++ b/rclcpp/include/rclcpp/executor.hpp @@ -439,15 +439,15 @@ class Executor RCLCPP_PUBLIC static void - execute_subscription(rclcpp::SubscriptionBase* subscription); + execute_subscription(rclcpp::SubscriptionBase * subscription); RCLCPP_PUBLIC static void - execute_service(rclcpp::ServiceBase* service); + execute_service(rclcpp::ServiceBase * service); RCLCPP_PUBLIC static void - execute_client(rclcpp::ClientBase* client); + execute_client(rclcpp::ClientBase * client); /** * \throws std::runtime_error if the wait set can be cleared diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 4112362e58..dbff765279 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -132,12 +132,12 @@ class EventsExecutor : public rclcpp::Executor /// Extract and execute events from the queue until it is empty RCLCPP_PUBLIC void - consume_all_events(std::queue &queue); + consume_all_events(std::queue & queue); // Execute a single event RCLCPP_PUBLIC void - execute_event(const ExecutorEvent &event); + execute_event(const ExecutorEvent & event); // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, @@ -146,8 +146,8 @@ class EventsExecutor : public rclcpp::Executor push_event(const void * executor_ptr, ExecutorEvent event) { // Cast executor_ptr to this (need to remove constness) - auto this_executor = const_cast( - static_cast(executor_ptr)); + auto this_executor = const_cast( + static_cast(executor_ptr)); // Event queue mutex scope { diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 3418b573fa..8ef3f9a3bc 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -98,7 +98,7 @@ class TimersManager /** * @brief Executes one ready timer if available - * + * * @return true if there was a timer ready */ bool execute_head_timer(); @@ -106,7 +106,7 @@ class TimersManager /** * @brief Get the amount of time before the next timer expires. * - * @return std::chrono::nanoseconds to wait, + * @return std::chrono::nanoseconds to wait, * the returned value could be negative if the timer is already expired * or MAX_TIME if the heap is empty. */ @@ -131,7 +131,7 @@ class TimersManager private: RCLCPP_DISABLE_COPY(TimersManager) - using TimerPtr = rclcpp::TimerBase::SharedPtr*; + using TimerPtr = rclcpp::TimerBase::SharedPtr *; /** * @brief Implements a loop that keeps executing ready timers. @@ -143,7 +143,7 @@ class TimersManager * @brief Get the amount of time before the next timer expires. * This function is not thread safe, acquire a mutex before calling it. * - * @return std::chrono::nanoseconds to wait, + * @return std::chrono::nanoseconds to wait, * the returned value could be negative if the timer is already expired * or MAX_TIME if the heap is empty. */ @@ -194,10 +194,13 @@ class TimersManager { size_t i = heap_.size(); // Position where we are going to add timer heap_.push_back(x); - while (i && ((*x)->time_until_trigger() < (*heap_[(i-1)/2])->time_until_trigger())) { - heap_[i] = heap_[(i-1)/2]; - heap_[(i-1)/2] = x; - i = (i-1)/2; + + size_t parent = (i - 1) / 2; + while (i > 0 && ((*x)->time_until_trigger() < (*heap_[parent])->time_until_trigger())) { + heap_[i] = heap_[parent]; + heap_[parent] = x; + i = parent; + parent = (i - 1) / 2; } } @@ -225,12 +228,12 @@ class TimersManager } heap_[i] = heap_[left_child]; i = left_child; - left_child = 2*i + 1; + left_child = 2 * i + 1; } // Swim down while (i > start) { - size_t parent = (i - 1)/2; + size_t parent = (i - 1) / 2; if ((*updated_timer)->time_until_trigger() < (*heap_[parent])->time_until_trigger()) { heap_[i] = heap_[parent]; i = parent; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index f4cfc40b31..905b5fdbba 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -201,14 +201,14 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( - const void * executor_context, - ExecutorEventCallback executor_callback) const + const void * executor_context, + ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_client_set_events_executor_callback( - executor_context, - executor_callback, - this, - client_handle_.get()); + executor_context, + executor_callback, + this, + client_handle_.get()); if (RCL_RET_OK != ret) { throw std::runtime_error(std::string("Couldn't set client callback")); diff --git a/rclcpp/src/rclcpp/executor.cpp b/rclcpp/src/rclcpp/executor.cpp index 12225e3cef..be138fc487 100644 --- a/rclcpp/src/rclcpp/executor.cpp +++ b/rclcpp/src/rclcpp/executor.cpp @@ -561,7 +561,7 @@ Executor::execute_subscription(rclcpp::SubscriptionBase::SharedPtr subscription) } void -Executor::execute_subscription(rclcpp::SubscriptionBase* subscription) +Executor::execute_subscription(rclcpp::SubscriptionBase * subscription) { rclcpp::MessageInfo message_info; message_info.get_rmw_message_info().from_intra_process = false; @@ -641,7 +641,7 @@ Executor::execute_service(rclcpp::ServiceBase::SharedPtr service) } void -Executor::execute_service(rclcpp::ServiceBase* service) +Executor::execute_service(rclcpp::ServiceBase * service) { auto request_header = service->create_request_header(); std::shared_ptr request = service->create_request(); @@ -659,7 +659,7 @@ Executor::execute_client(rclcpp::ClientBase::SharedPtr client) } void -Executor::execute_client(rclcpp::ClientBase* client) +Executor::execute_client(rclcpp::ClientBase * client) { auto request_header = client->create_request_header(); std::shared_ptr response = client->create_response(); diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 099ed64340..6f81915dd3 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include "rclcpp/executors/events_executor.hpp" @@ -112,7 +113,7 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) throw std::runtime_error("spin_some() called while already spinning"); } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - + // This function will wait until the first of the following events occur: // - The input max_duration is elapsed // - A timer triggers @@ -251,7 +252,7 @@ EventsExecutor::add_node( timers_manager_->add_timer(timer); } return false; - }); + }); // Set the callbacks to all the entities group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { @@ -286,11 +287,11 @@ EventsExecutor::add_node( // Set node's guard condition callback, so if new entities are added while // spinning we can set their callback. rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - this, - &EventsExecutor::push_event, - entities_collector_.get(), - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); + this, + &EventsExecutor::push_event, + entities_collector_.get(), + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); if (ret != RCL_RET_OK) { throw std::runtime_error("Couldn't set node guard condition callback"); @@ -321,7 +322,7 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) } void -EventsExecutor::consume_all_events(std::queue &event_queue) +EventsExecutor::consume_all_events(std::queue & event_queue) { while (!event_queue.empty()) { ExecutorEvent event = event_queue.front(); @@ -332,43 +333,43 @@ EventsExecutor::consume_all_events(std::queue &event_queue) } void -EventsExecutor::execute_event(const ExecutorEvent &event) +EventsExecutor::execute_event(const ExecutorEvent & event) { - switch(event.type) { - case SUBSCRIPTION_EVENT: - { - auto subscription = const_cast( - static_cast(event.entity)); - execute_subscription(subscription); - break; - } - - case SERVICE_EVENT: - { - auto service = const_cast( - static_cast(event.entity)); - execute_service(service); - break; - } - - case CLIENT_EVENT: - { - auto client = const_cast( - static_cast(event.entity)); - execute_client(client); + switch (event.type) { + case SUBSCRIPTION_EVENT: + { + auto subscription = const_cast( + static_cast(event.entity)); + execute_subscription(subscription); + break; + } + + case SERVICE_EVENT: + { + auto service = const_cast( + static_cast(event.entity)); + execute_service(service); + break; + } + + case CLIENT_EVENT: + { + auto client = const_cast( + static_cast(event.entity)); + execute_client(client); + break; + } + + case WAITABLE_EVENT: + { + auto waitable = const_cast( + static_cast(event.entity)); + waitable->execute(); + break; + } + + default: + throw std::runtime_error("EventsExecutor received unrecognized event"); break; - } - - case WAITABLE_EVENT: - { - auto waitable = const_cast( - static_cast(event.entity)); - waitable->execute(); - break; - } - - default: - throw std::runtime_error("EventsExecutor received unrecognized event"); - break; } } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 92e8cd38ce..e09c3b642b 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -62,7 +62,7 @@ EventsExecutorEntitiesCollector::execute() void EventsExecutorEntitiesCollector::add_node( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { // If the node already has an executor std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); @@ -86,9 +86,9 @@ EventsExecutorEntitiesCollector::remove_node( if (matched) { // Node found: unset its entities callbacks rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node_ptr->get_notify_guard_condition(), - false); + nullptr, nullptr, nullptr, + node_ptr->get_notify_guard_condition(), + false); if (ret != RCL_RET_OK) { throw std::runtime_error(std::string("Couldn't set guard condition callback")); diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 9088221348..1652ebbb21 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -164,7 +164,7 @@ void TimersManager::remove_timer(rclcpp::TimerBase::SharedPtr timer) void TimersManager::rebuild_heap() { heap_.clear(); - for (auto& t : timers_storage_) { + for (auto & t : timers_storage_) { this->add_timer_to_heap(&t); } } diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index a178314e91..ee2865f067 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -87,14 +87,14 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( - const void * executor_context, - ExecutorEventCallback executor_callback) const + const void * executor_context, + ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_service_set_events_executor_callback( - executor_context, - executor_callback, - this, - service_handle_.get()); + executor_context, + executor_callback, + this, + service_handle_.get()); if (RCL_RET_OK != ret) { throw std::runtime_error(std::string("Couldn't set service callback")); diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 073168748e..e163f32c0a 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -291,14 +291,14 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( - const void * executor_context, - ExecutorEventCallback executor_callback) const + const void * executor_context, + ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_subscription_set_events_executor_callback( - executor_context, - executor_callback, - this, - subscription_handle_.get()); + executor_context, + executor_callback, + this, + subscription_handle_.get()); if (RCL_RET_OK != ret) { throw std::runtime_error(std::string("Couldn't set subscription callback")); diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 067cadf7eb..ae2bad93f4 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include + #include "rclcpp/experimental/subscription_intra_process_base.hpp" using rclcpp::experimental::SubscriptionIntraProcessBase; @@ -52,4 +54,4 @@ SubscriptionIntraProcessBase::set_events_executor_callback( if (RCL_RET_OK != ret) { throw std::runtime_error(std::string("Couldn't set guard condition callback")); } -} \ No newline at end of file +} diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index f354145a79..a266c91eb7 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -62,11 +62,12 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( - void * executor_context, - ExecutorEventCallback executor_callback) const + void * executor_context, + ExecutorEventCallback executor_callback) const { (void)executor_context; (void)executor_callback; - throw std::runtime_error("Custom waitables should override set_events_executor_callback() to use events executor"); + throw std::runtime_error( + "Custom waitables should override set_events_executor_callback() to use events executor"); } From cf8fc9a7068e21d5665c22e7908e5606ba514491 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 12 Oct 2020 17:09:46 +0100 Subject: [PATCH 041/168] rename functions in unit-tests Signed-off-by: Soragna, Alberto --- rclcpp/test/rclcpp/executors/test_executors.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index e3ed8d47ab..d9e34305bb 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -453,11 +453,11 @@ class TestWaitable : public rclcpp::Waitable } void - set_callback( + set_events_executor_callback( void * executor_context, ExecutorEventCallback executor_callback) const override { - rcl_ret_t ret = rcl_guard_condition_set_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor_context, executor_callback, this, From 07876aa92e2daa547d4810f67ba37f56ec8e6870 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 12 Oct 2020 19:24:11 +0100 Subject: [PATCH 042/168] add timers manager test and bug fixes Signed-off-by: Soragna, Alberto --- .../src/rclcpp/executors/timers_manager.cpp | 3 +- rclcpp/test/CMakeLists.txt | 10 +- .../rclcpp/executors/test_timers_manager.cpp | 363 ++++++++++++++++++ 3 files changed, 374 insertions(+), 2 deletions(-) create mode 100644 rclcpp/test/rclcpp/executors/test_timers_manager.cpp diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 9088221348..a93428ac7b 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -61,6 +61,7 @@ void TimersManager::start() throw std::runtime_error("TimersManager::start() can't start timers thread as already running"); } + running_ = true; timers_thread_ = std::thread(&TimersManager::run_timers, this); pthread_setname_np(timers_thread_.native_handle(), "TimersManager"); } @@ -104,6 +105,7 @@ void TimersManager::execute_ready_timers_unsafe() while ((*head)->is_ready() && timer_was_ready_at_tp(start, head)) { (*head)->execute_callback(); this->restore_heap_root(); + head = heap_.front(); //verify(); } } @@ -128,7 +130,6 @@ bool TimersManager::execute_head_timer() void TimersManager::run_timers() { - running_ = true; while (rclcpp::ok(context_) && running_) { std::unique_lock lock(timers_mutex_); auto time_to_sleep = this->get_head_timeout_unsafe(); diff --git a/rclcpp/test/CMakeLists.txt b/rclcpp/test/CMakeLists.txt index 6656d05b58..e5c8b98b36 100644 --- a/rclcpp/test/CMakeLists.txt +++ b/rclcpp/test/CMakeLists.txt @@ -540,7 +540,7 @@ ament_add_gtest( test_executors rclcpp/executors/test_executors.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}" - TIMEOUT 20) + TIMEOUT 180) if(TARGET test_executors) ament_target_dependencies(test_executors "rcl" @@ -573,6 +573,14 @@ if(TARGET test_static_executor_entities_collector) target_link_libraries(test_static_executor_entities_collector ${PROJECT_NAME} mimick) endif() +ament_add_gtest(test_timers_manager rclcpp/executors/test_timers_manager.cpp + APPEND_LIBRARY_DIRS "${append_library_dirs}") +if(TARGET test_timers_manager) + ament_target_dependencies(test_timers_manager + "rcl") + target_link_libraries(test_timers_manager ${PROJECT_NAME}) +endif() + ament_add_gtest(test_guard_condition rclcpp/test_guard_condition.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}") if(TARGET test_guard_condition) diff --git a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp new file mode 100644 index 0000000000..21e746ec12 --- /dev/null +++ b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp @@ -0,0 +1,363 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include + +#include +#include + +#include "rclcpp/contexts/default_context.hpp" +#include "rclcpp/executors/timers_manager.hpp" + +using namespace std::chrono_literals; + +using rclcpp::executors::TimersManager; + +using CallbackT = std::function; +using TimerT = rclcpp::WallTimer; + +class TestTimersManager : public ::testing::Test +{ +public: + void SetUp() + { + rclcpp::init(0, nullptr); + } + + void TearDown() + { + rclcpp::shutdown(); + } +}; + +TEST_F(TestTimersManager, empty_manager) { + + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + EXPECT_EQ(TimersManager::MAX_TIME, timers_manager->get_head_timeout()); + EXPECT_FALSE(timers_manager->execute_head_timer()); + EXPECT_NO_THROW(timers_manager->execute_ready_timers()); + EXPECT_NO_THROW(timers_manager->clear_all()); + EXPECT_NO_THROW(timers_manager->start()); + EXPECT_NO_THROW(timers_manager->stop()); +} + +TEST_F(TestTimersManager, add_run_remove_timer) { + + size_t t_runs = 0; + auto t = TimerT::make_shared( + 1ms, + [&t_runs](){ + t_runs++; + }, + rclcpp::contexts::get_global_default_context()); + std::weak_ptr t_weak = t; + + // Add the timer to the timers manager + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + timers_manager->add_timer(std::move(t)); + + // Sleep for more 3 times the timer period + std::this_thread::sleep_for(3ms); + + // The timer is executed only once, even if we slept 3 times the period + timers_manager->execute_ready_timers(); + EXPECT_EQ(1u, t_runs); + + // The timer is still valid after execution + auto t_tmp = t_weak.lock(); + EXPECT_TRUE(t_tmp != nullptr); + + // Remove the timer from the manager + timers_manager->remove_timer(std::move(t_tmp)); + + // The timer is now not valid anymore + t_tmp = t_weak.lock(); + EXPECT_FALSE(t_tmp != nullptr); +} + +TEST_F(TestTimersManager, clear_all) { + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + auto t1 = TimerT::make_shared(1ms, CallbackT(), rclcpp::contexts::get_global_default_context()); + std::weak_ptr t1_weak = t1; + auto t2 = TimerT::make_shared(1ms, CallbackT(), rclcpp::contexts::get_global_default_context()); + std::weak_ptr t2_weak = t2; + + timers_manager->add_timer(std::move(t1)); + timers_manager->add_timer(std::move(t2)); + + EXPECT_TRUE(t1_weak.lock() != nullptr); + EXPECT_TRUE(t2_weak.lock() != nullptr); + + timers_manager->clear_all(); + + EXPECT_FALSE(t1_weak.lock() != nullptr); + EXPECT_FALSE(t2_weak.lock() != nullptr); +} + +TEST_F(TestTimersManager, remove_not_existing_timer) { + + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + // Try to remove a nullptr timer + EXPECT_NO_THROW(timers_manager->remove_timer(nullptr)); + + auto t = TimerT::make_shared(1ms, CallbackT(), rclcpp::contexts::get_global_default_context()); + timers_manager->add_timer(t); + + // Remove twice the same timer + timers_manager->remove_timer(t); + EXPECT_NO_THROW(timers_manager->remove_timer(t)); +} + +TEST_F(TestTimersManager, timers_order) { + + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + size_t t1_runs = 0; + auto t1 = TimerT::make_shared( + 1ms, + [&t1_runs](){ + t1_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + size_t t2_runs = 0; + auto t2 = TimerT::make_shared( + 3ms, + [&t2_runs](){ + t2_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + size_t t3_runs = 0; + auto t3 = TimerT::make_shared( + 5ms, + [&t3_runs](){ + t3_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + // Add timers in a random order + timers_manager->add_timer(t2); + timers_manager->add_timer(t3); + timers_manager->add_timer(t1); + + std::this_thread::sleep_for(1ms); + timers_manager->execute_ready_timers(); + EXPECT_EQ(1u, t1_runs); + EXPECT_EQ(0u, t2_runs); + EXPECT_EQ(0u, t3_runs); + + std::this_thread::sleep_for(1ms); + timers_manager->execute_ready_timers(); + EXPECT_EQ(2u, t1_runs); + EXPECT_EQ(0u, t2_runs); + EXPECT_EQ(0u, t3_runs); + + std::this_thread::sleep_for(1ms); + timers_manager->execute_ready_timers(); + EXPECT_EQ(3u, t1_runs); + EXPECT_EQ(1u, t2_runs); + EXPECT_EQ(0u, t3_runs); + + std::this_thread::sleep_for(5ms); + timers_manager->execute_ready_timers(); + EXPECT_EQ(4u, t1_runs); + EXPECT_EQ(2u, t2_runs); + EXPECT_EQ(1u, t3_runs); +} + +TEST_F(TestTimersManager, start_stop_timers_thread) { + + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + auto t = TimerT::make_shared(1ms, [](){}, rclcpp::contexts::get_global_default_context()); + timers_manager->add_timer(t); + + // Calling start multiple times will throw an error + EXPECT_NO_THROW(timers_manager->start()); + EXPECT_THROW(timers_manager->start(), std::exception); + + // Calling stop multiple times does not throw an error + EXPECT_NO_THROW(timers_manager->stop()); + EXPECT_NO_THROW(timers_manager->stop()); +} + +TEST_F(TestTimersManager, timers_thread) { + + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + size_t t1_runs = 0; + auto t1 = TimerT::make_shared( + 1ms, + [&t1_runs](){ + t1_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + size_t t2_runs = 0; + auto t2 = TimerT::make_shared( + 1ms, + [&t2_runs](){ + t2_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + // Add timers + timers_manager->add_timer(t1); + timers_manager->add_timer(t2); + + // Run timers thread for a while + timers_manager->start(); + std::this_thread::sleep_for(5ms); + timers_manager->stop(); + + EXPECT_LT(1u, t1_runs); + EXPECT_LT(1u, t2_runs); + EXPECT_EQ(t1_runs, t2_runs); +} + +TEST_F(TestTimersManager, destructor) { + + size_t t_runs = 0; + auto t = TimerT::make_shared( + 1ms, + [&t_runs](){ + t_runs++; + }, + rclcpp::contexts::get_global_default_context()); + std::weak_ptr t_weak = t; + + // When the timers manager is destroyed, it will stop the thread + // and clear the timers + { + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + timers_manager->add_timer(std::move(t)); + + timers_manager->start(); + std::this_thread::sleep_for(3ms); + + EXPECT_LT(1u, t_runs); + } + + // The thread is not running anymore, so this value does not increase + size_t runs = t_runs; + std::this_thread::sleep_for(3ms); + EXPECT_EQ(runs, t_runs); + EXPECT_FALSE(t_weak.lock() != nullptr); +} + +TEST_F(TestTimersManager, add_remove_while_thread_running) { + + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + size_t t1_runs = 0; + auto t1 = TimerT::make_shared( + 1ms, + [&t1_runs](){ + t1_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + size_t t2_runs = 0; + auto t2 = TimerT::make_shared( + 1ms, + [&t2_runs](){ + t2_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + // Add timers + timers_manager->add_timer(t1); + + // Start timers thread + timers_manager->start(); + + // After a while remove t1 and add t2 + std::this_thread::sleep_for(5ms); + timers_manager->remove_timer(t1); + size_t tmp_t1 = t1_runs; + timers_manager->add_timer(t2); + + // Wait some more time and then stop + std::this_thread::sleep_for(5ms); + timers_manager->stop(); + + // t1 has stopped running + EXPECT_EQ(tmp_t1, t1_runs); + // t2 is correctly running + EXPECT_LT(1u, t2_runs); +} + +TEST_F(TestTimersManager, infinite_loop) { + + // This test makes sure that even if timers have a period shorter than the duration + // of their callback the functions never block indefinitely. + + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + size_t t1_runs = 0; + auto t1 = TimerT::make_shared( + 1ms, + [&t1_runs](){ + t1_runs++; + std::this_thread::sleep_for(5ms); + }, + rclcpp::contexts::get_global_default_context()); + + size_t t2_runs = 0; + auto t2 = TimerT::make_shared( + 1ms, + [&t2_runs](){ + t2_runs++; + std::this_thread::sleep_for(5ms); + }, + rclcpp::contexts::get_global_default_context()); + + timers_manager->add_timer(t1); + timers_manager->add_timer(t2); + + // Sleep for enough time to trigger timers + std::this_thread::sleep_for(3ms); + timers_manager->execute_ready_timers(); + EXPECT_EQ(1u, t1_runs); + EXPECT_EQ(1u, t2_runs); + + // Due to the long execution of timer callbacks, timers are already ready + bool ret = timers_manager->execute_head_timer(); + EXPECT_TRUE(ret); + EXPECT_EQ(3u, t1_runs + t2_runs); + + // Start a timers thread + timers_manager->start(); + std::this_thread::sleep_for(10ms); + timers_manager->stop(); + + EXPECT_LT(3u, t1_runs + t2_runs); + EXPECT_LT(1u, t1_runs); + EXPECT_LT(1u, t2_runs); +} From 3552a80531a265fc9b705f32554d1a70f9280ad6 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Tue, 13 Oct 2020 08:44:37 +0100 Subject: [PATCH 043/168] cleanup entities collector Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/events_executor.hpp | 40 ++--- .../events_executor_entities_collector.hpp | 54 ++++--- .../src/rclcpp/executors/events_executor.cpp | 74 +-------- .../events_executor_entities_collector.cpp | 151 +++++++++--------- 4 files changed, 130 insertions(+), 189 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index dbff765279..9da6b028ee 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -119,26 +119,6 @@ class EventsExecutor : public rclcpp::Executor void remove_node(std::shared_ptr node_ptr, bool notify = true) override; -protected: - RCLCPP_PUBLIC - void - spin_once_impl(std::chrono::nanoseconds timeout) override; - -private: - RCLCPP_DISABLE_COPY(EventsExecutor) - - EventsExecutorEntitiesCollector::SharedPtr entities_collector_; - - /// Extract and execute events from the queue until it is empty - RCLCPP_PUBLIC - void - consume_all_events(std::queue & queue); - - // Execute a single event - RCLCPP_PUBLIC - void - execute_event(const ExecutorEvent & event); - // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. @@ -159,6 +139,26 @@ class EventsExecutor : public rclcpp::Executor this_executor->event_queue_cv_.notify_one(); } +protected: + RCLCPP_PUBLIC + void + spin_once_impl(std::chrono::nanoseconds timeout) override; + +private: + RCLCPP_DISABLE_COPY(EventsExecutor) + + EventsExecutorEntitiesCollector::SharedPtr entities_collector_; + + /// Extract and execute events from the queue until it is empty + RCLCPP_PUBLIC + void + consume_all_events(std::queue & queue); + + // Execute a single event + RCLCPP_PUBLIC + void + execute_event(const ExecutorEvent & event); + // Event queue members std::queue event_queue_; std::mutex event_queue_mutex_; diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 8e4f57b3ac..0b4db0ae17 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -17,6 +17,7 @@ #include +#include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node_interfaces/node_base_interface.hpp" #include "rclcpp/waitable.hpp" @@ -25,31 +26,41 @@ namespace rclcpp namespace executors { +// forward declaration of EventsExecutor to avoid circular dependency +class EventsExecutor; + +/** + * @brief This class provides a waitable object that is used for managing the + * entities (i.e. nodes and their subscriptions, timers, services, etc) + * added to an EventsExecutor. + * The add/remove node APIs are used when a node is added/removed from + * the associated EventsExecutor and result in setting/unsetting the + * events callbacks and adding timers to the timers manager. + * + * Being this class derived from Waitable, it can be used to wake up an + * executor thread while it's spinning. + * When this occurs, the execute API takes care of handling changes + * in the entities currently used by the executor. + */ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable { public: RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorEntitiesCollector) - using TimerFn = std::function; - using ClearTimersFn = std::function; - // Constructor RCLCPP_PUBLIC - EventsExecutorEntitiesCollector() = default; + EventsExecutorEntitiesCollector( + EventsExecutor * executor_context, + std::shared_ptr timers_manager); // Destructor RCLCPP_PUBLIC ~EventsExecutorEntitiesCollector(); - RCLCPP_PUBLIC - void - init( - void * executor_context, - ExecutorEventCallback executor_callback, - TimerFn push_timer, - TimerFn clear_timer, - ClearTimersFn clear_all_timers); - + // The purpose of "execute" is handling the situation of a new entity added to + // a node, while the executor is already spinning. + // This consists in setting that entitiy's callback. + // If a entity is removed from a node, we should unset its callback RCLCPP_PUBLIC void execute() override; @@ -64,6 +75,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr); + // Stub API: not used by EventsExecutor RCLCPP_PUBLIC bool is_ready(rcl_wait_set_t * wait_set) override @@ -72,6 +84,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable return false; } + // Stub API: not used by EventsExecutor RCLCPP_PUBLIC bool add_to_wait_set(rcl_wait_set_t * wait_set) override @@ -82,21 +95,16 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable private: void - set_entities_callbacks(); + set_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); /// List of weak nodes registered in the events executor std::list weak_nodes_; - /// Context (associated executor) - void * executor_context_ = nullptr; - - /// Events callback - ExecutorEventCallback executor_callback_ = nullptr; + /// Executor using this entities collector object + EventsExecutor * associated_executor_ = nullptr; - /// Function pointers to push and clear timers from the timers heap - TimerFn push_timer_ = nullptr; - TimerFn clear_timer_ = nullptr; - ClearTimersFn clear_all_timers_ = nullptr; + // Instance of the timers manager used by the associated executor + TimersManager::SharedPtr timers_manager_; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 58b664745c..3c9db4f389 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -28,21 +28,7 @@ EventsExecutor::EventsExecutor( : rclcpp::Executor(options) { timers_manager_ = std::make_shared(context_); - entities_collector_ = std::make_shared(); - - // Set entities collector callbacks - entities_collector_->init( - this, - &EventsExecutor::push_event, - [this](const rclcpp::TimerBase::SharedPtr & t) { - timers_manager_->add_timer(t); - }, - [this](const rclcpp::TimerBase::SharedPtr & t) { - timers_manager_->remove_timer(t); - }, - [this]() { - timers_manager_->clear_all(); - }); + entities_collector_ = std::make_shared(this, timers_manager_); rcl_ret_t ret; @@ -247,64 +233,6 @@ EventsExecutor::add_node( // Add node to entities collector entities_collector_->add_node(node_ptr); - - // Get nodes entities, and assign their callbaks - for (auto & weak_group : node_ptr->get_callback_groups()) { - auto group = weak_group.lock(); - if (!group || !group->can_be_taken_from().load()) { - continue; - } - // Add timers to timers to timer manager - group->find_timer_ptrs_if( - [this](const rclcpp::TimerBase::SharedPtr & timer) { - if (timer) { - timers_manager_->add_timer(timer); - } - return false; - }); - // Set the callbacks to all the entities - group->find_subscription_ptrs_if( - [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { - if (subscription) { - subscription->set_events_executor_callback(this, &EventsExecutor::push_event); - } - return false; - }); - group->find_service_ptrs_if( - [this](const rclcpp::ServiceBase::SharedPtr & service) { - if (service) { - service->set_events_executor_callback(this, &EventsExecutor::push_event); - } - return false; - }); - group->find_client_ptrs_if( - [this](const rclcpp::ClientBase::SharedPtr & client) { - if (client) { - client->set_events_executor_callback(this, &EventsExecutor::push_event); - } - return false; - }); - group->find_waitable_ptrs_if( - [this](const rclcpp::Waitable::SharedPtr & waitable) { - if (waitable) { - waitable->set_events_executor_callback(this, &EventsExecutor::push_event); - } - return false; - }); - } - - // Set node's guard condition callback, so if new entities are added while - // spinning we can set their callback. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - this, - &EventsExecutor::push_event, - entities_collector_.get(), - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } } void diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index e09c3b642b..bc1f6cbdd0 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -17,6 +17,14 @@ using rclcpp::executors::EventsExecutorEntitiesCollector; +EventsExecutorEntitiesCollector::EventsExecutorEntitiesCollector( + EventsExecutor * executor_context, + TimersManager::SharedPtr timers_manager) +{ + associated_executor_ = executor_context; + timers_manager_ = timers_manager; +} + EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() { // Disassociate all nodes @@ -30,34 +38,21 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() weak_nodes_.clear(); } -void -EventsExecutorEntitiesCollector::init( - void * executor_context, - ExecutorEventCallback executor_callback, - TimerFn push_timer, - TimerFn clear_timer, - ClearTimersFn clear_all_timers) -{ - // These callbacks are used when: - // 1. A new entity is added/removed to/from a new node - // 2. A node is removed from the executor - executor_context_ = executor_context; - executor_callback_ = executor_callback; - push_timer_ = push_timer; - clear_timer_ = clear_timer; - clear_all_timers_ = clear_all_timers; -} - -// The purpose of "execute" is handling the situation of a new entity added to -// a node, while the executor is already spinning. -// With the new approach, "execute" should only take care of setting that -// entitiy's callback. -// If a entity is removed from a node, we should unset its callback void EventsExecutorEntitiesCollector::execute() { - clear_all_timers_(); - set_entities_callbacks(); + // This function is called when the associated executor is notified that something changed. + // We do not know if an entity has been added or remode so we have to rebuild everything. + + timers_manager_->clear_all(); + + for (auto & weak_node : weak_nodes_) { + auto node = weak_node.lock(); + if (!node) { + continue; + } + set_entities_callbacks(node); + } } void @@ -72,15 +67,30 @@ EventsExecutorEntitiesCollector::add_node( } weak_nodes_.push_back(node_ptr); + + set_entities_callbacks(node_ptr); + + // Set node's guard condition callback, so if new entities are added while + // spinning we can set their callback. + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event, + this, + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set node guard condition callback"); + } } -// Here we unset the node entities callback. void EventsExecutorEntitiesCollector::remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { auto node_it = weak_nodes_.begin(); + // Here we unset the node entities callback. while (node_it != weak_nodes_.end()) { bool matched = (node_it->lock() == node_ptr); if (matched) { @@ -103,7 +113,7 @@ EventsExecutorEntitiesCollector::remove_node( group->find_timer_ptrs_if( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { - clear_timer_(timer); + timers_manager_->remove_timer(timer); } return false; }); @@ -146,54 +156,49 @@ EventsExecutorEntitiesCollector::remove_node( } void -EventsExecutorEntitiesCollector::set_entities_callbacks() +EventsExecutorEntitiesCollector::set_entities_callbacks( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node) { - for (auto & weak_node : weak_nodes_) { - auto node = weak_node.lock(); - if (!node) { + // Check in all the callback groups + for (auto & weak_group : node->get_callback_groups()) { + auto group = weak_group.lock(); + if (!group || !group->can_be_taken_from().load()) { continue; } - // Check in all the callback groups - for (auto & weak_group : node->get_callback_groups()) { - auto group = weak_group.lock(); - if (!group || !group->can_be_taken_from().load()) { - continue; - } - group->find_timer_ptrs_if( - [this](const rclcpp::TimerBase::SharedPtr & timer) { - if (timer) { - push_timer_(timer); - } - return false; - }); - group->find_subscription_ptrs_if( - [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { - if (subscription) { - subscription->set_events_executor_callback(executor_context_, executor_callback_); - } - return false; - }); - group->find_service_ptrs_if( - [this](const rclcpp::ServiceBase::SharedPtr & service) { - if (service) { - service->set_events_executor_callback(executor_context_, executor_callback_); - } - return false; - }); - group->find_client_ptrs_if( - [this](const rclcpp::ClientBase::SharedPtr & client) { - if (client) { - client->set_events_executor_callback(executor_context_, executor_callback_); - } - return false; - }); - group->find_waitable_ptrs_if( - [this](const rclcpp::Waitable::SharedPtr & waitable) { - if (waitable) { - waitable->set_events_executor_callback(executor_context_, executor_callback_); - } - return false; - }); - } + group->find_timer_ptrs_if( + [this](const rclcpp::TimerBase::SharedPtr & timer) { + if (timer) { + timers_manager_->add_timer(timer); + } + return false; + }); + group->find_subscription_ptrs_if( + [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { + if (subscription) { + subscription->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + } + return false; + }); + group->find_service_ptrs_if( + [this](const rclcpp::ServiceBase::SharedPtr & service) { + if (service) { + service->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + } + return false; + }); + group->find_client_ptrs_if( + [this](const rclcpp::ClientBase::SharedPtr & client) { + if (client) { + client->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + } + return false; + }); + group->find_waitable_ptrs_if( + [this](const rclcpp::Waitable::SharedPtr & waitable) { + if (waitable) { + waitable->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + } + return false; + }); } } From 652ccf8e242e22caf9ee95a863c9f42f61660ceb Mon Sep 17 00:00:00 2001 From: Mauro Date: Tue, 13 Oct 2020 12:26:46 +0100 Subject: [PATCH 044/168] Support events --- rclcpp/include/rclcpp/qos_event.hpp | 19 +++++++++++++++++++ rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- 4 files changed, 22 insertions(+), 3 deletions(-) diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 0bce0aac37..58bfa0331d 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -148,6 +148,25 @@ class QOSEventHandler : public QOSEventHandlerBase event_callback_(callback_info); } + /// Set EventsExecutor's callback + RCLCPP_PUBLIC + void + set_events_executor_callback( + void * executor_context, + ExecutorEventCallback executor_callback) const override + { + rcl_ret_t ret = rcl_event_set_events_executor_callback( + executor_context, + executor_callback, + this, + &event_handle_, + false /* Discard previous events */); + + if (RCL_RET_OK != ret) { + throw std::runtime_error("Couldn't set EventsExecutor's callback to event"); + } + } + private: using EventCallbackInfoT = typename std::remove_reference::template argument_type<0>>::type; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 905b5fdbba..4f8604ed90 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -211,6 +211,6 @@ ClientBase::set_events_executor_callback( client_handle_.get()); if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set client callback")); + throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); } } diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index ee2865f067..914582e4ed 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -97,6 +97,6 @@ ServiceBase::set_events_executor_callback( service_handle_.get()); if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set service callback")); + throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); } } diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index e163f32c0a..345bbf7181 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -301,6 +301,6 @@ SubscriptionBase::set_events_executor_callback( subscription_handle_.get()); if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set subscription callback")); + throw std::runtime_error("Couldn't set the EventsExecutor's callback to subscription"); } } From 292e7a9cb7bcd8c8b8c8f4777d871b1099ac2d1b Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Tue, 13 Oct 2020 13:45:14 +0100 Subject: [PATCH 045/168] Update test_timers_manager.cpp --- rclcpp/test/rclcpp/executors/test_timers_manager.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp index 21e746ec12..0aea7e5c58 100644 --- a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp +++ b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp @@ -166,21 +166,15 @@ TEST_F(TestTimersManager, timers_order) { EXPECT_EQ(0u, t2_runs); EXPECT_EQ(0u, t3_runs); - std::this_thread::sleep_for(1ms); + std::this_thread::sleep_for(2ms); timers_manager->execute_ready_timers(); EXPECT_EQ(2u, t1_runs); - EXPECT_EQ(0u, t2_runs); - EXPECT_EQ(0u, t3_runs); - - std::this_thread::sleep_for(1ms); - timers_manager->execute_ready_timers(); - EXPECT_EQ(3u, t1_runs); EXPECT_EQ(1u, t2_runs); EXPECT_EQ(0u, t3_runs); std::this_thread::sleep_for(5ms); timers_manager->execute_ready_timers(); - EXPECT_EQ(4u, t1_runs); + EXPECT_EQ(3u, t1_runs); EXPECT_EQ(2u, t2_runs); EXPECT_EQ(1u, t3_runs); } From 5c36010c7385a7e85a91b0a225783b80ff4b062d Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Tue, 13 Oct 2020 13:52:26 +0100 Subject: [PATCH 046/168] override cancel Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/executor.hpp | 2 +- .../include/rclcpp/executors/events_executor.hpp | 9 +++++++++ rclcpp/src/rclcpp/executors/events_executor.cpp | 15 +++++++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executor.hpp b/rclcpp/include/rclcpp/executor.hpp index c9994a49a1..0d0497168f 100644 --- a/rclcpp/include/rclcpp/executor.hpp +++ b/rclcpp/include/rclcpp/executor.hpp @@ -385,7 +385,7 @@ class Executor * \throws std::runtime_error if there is an issue triggering the guard condition */ RCLCPP_PUBLIC - void + virtual void cancel(); /// Support dynamic switching of the memory strategy. diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 9da6b028ee..338687e8b8 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -119,6 +119,15 @@ class EventsExecutor : public rclcpp::Executor void remove_node(std::shared_ptr node_ptr, bool notify = true) override; + /// Cancel any running spin* function, causing it to return. + /** + * This function can be called asynchonously from any thread. + * \throws std::runtime_error if there is an issue triggering the guard condition + */ + RCLCPP_PUBLIC + void + cancel() override; + // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 3c9db4f389..066adafff6 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -17,6 +17,7 @@ #include #include +#include "rclcpp/exceptions/exceptions.hpp" #include "rclcpp/executors/events_executor.hpp" using namespace std::chrono_literals; @@ -258,6 +259,20 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) this->remove_node(node_ptr->get_node_base_interface(), notify); } +void +EventsExecutor::cancel() +{ + spinning.store(false); + rcl_ret_t ret = rcl_trigger_guard_condition(&interrupt_guard_condition_); + if (ret != RCL_RET_OK) { + rclcpp::exceptions::throw_from_rcl_error(ret, "Failed to trigger guard condition in cancel"); + } + + // This makes sure that the timers manager is stopped when we return from this function + // otherwise applications may call rclcpp::shutdown() while that thread is still running. + timers_manager_->stop(); +} + void EventsExecutor::consume_all_events(std::queue & event_queue) { From 73d73e859eceda500c9b6a577509221dc5641ce9 Mon Sep 17 00:00:00 2001 From: Mauro Date: Wed, 14 Oct 2020 11:58:14 +0100 Subject: [PATCH 047/168] void return on set_events_executor_callback --- rclcpp/include/rclcpp/client.hpp | 1 + rclcpp/include/rclcpp/qos_event.hpp | 7 ++----- rclcpp/src/rclcpp/client.cpp | 6 +----- rclcpp/src/rclcpp/executors/events_executor.cpp | 14 ++------------ .../events_executor_entities_collector.cpp | 12 ++---------- rclcpp/src/rclcpp/service.cpp | 6 +----- rclcpp/src/rclcpp/subscription_base.cpp | 6 +----- .../src/rclcpp/subscription_intra_process_base.cpp | 6 +----- rclcpp/test/rclcpp/executors/test_executors.cpp | 6 +----- 9 files changed, 12 insertions(+), 52 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 129f7209af..055a9d728d 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -38,6 +38,7 @@ #include "rclcpp/visibility_control.hpp" #include "rcutils/logging_macros.h" +#include "rcutils/executor_event_types.h" #include "rmw/error_handling.h" #include "rmw/rmw.h" diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 58bfa0331d..259aa5ab54 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -22,6 +22,7 @@ #include "rmw/incompatible_qos_events_statuses.h" #include "rcutils/logging_macros.h" +#include "rcutils/executor_event_types.h" #include "rclcpp/exceptions.hpp" #include "rclcpp/function_traits.hpp" @@ -155,16 +156,12 @@ class QOSEventHandler : public QOSEventHandlerBase void * executor_context, ExecutorEventCallback executor_callback) const override { - rcl_ret_t ret = rcl_event_set_events_executor_callback( + rcl_event_set_events_executor_callback( executor_context, executor_callback, this, &event_handle_, false /* Discard previous events */); - - if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set EventsExecutor's callback to event"); - } } private: diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 4f8604ed90..c9e87384fa 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -204,13 +204,9 @@ ClientBase::set_events_executor_callback( const void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_ret_t ret = rcl_client_set_events_executor_callback( + rcl_client_set_events_executor_callback( executor_context, executor_callback, this, client_handle_.get()); - - if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); - } } diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 066adafff6..41e796ece5 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -31,31 +31,21 @@ EventsExecutor::EventsExecutor( timers_manager_ = std::make_shared(context_); entities_collector_ = std::make_shared(this, timers_manager_); - rcl_ret_t ret; - // Set the global ctrl-c guard condition callback - ret = rcl_guard_condition_set_events_executor_callback( + rcl_guard_condition_set_events_executor_callback( this, &EventsExecutor::push_event, entities_collector_.get(), options.context->get_interrupt_guard_condition(&wait_set_), false /* Discard previous events */); - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set ctrl-c guard condition callback"); - } - // Set the executor interrupt guard condition callback - ret = rcl_guard_condition_set_events_executor_callback( + rcl_guard_condition_set_events_executor_callback( this, &EventsExecutor::push_event, entities_collector_.get(), &interrupt_guard_condition_, false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set interrupt guard condition callback"); - } } EventsExecutor::~EventsExecutor() {} diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index bc1f6cbdd0..f204bcae21 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -72,16 +72,12 @@ EventsExecutorEntitiesCollector::add_node( // Set node's guard condition callback, so if new entities are added while // spinning we can set their callback. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_guard_condition_set_events_executor_callback( associated_executor_, &EventsExecutor::push_event, this, node_ptr->get_notify_guard_condition(), false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } } void @@ -95,15 +91,11 @@ EventsExecutorEntitiesCollector::remove_node( bool matched = (node_it->lock() == node_ptr); if (matched) { // Node found: unset its entities callbacks - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_guard_condition_set_events_executor_callback( nullptr, nullptr, nullptr, node_ptr->get_notify_guard_condition(), false); - if (ret != RCL_RET_OK) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } - // Unset entities callbacks for (auto & weak_group : node_ptr->get_callback_groups()) { auto group = weak_group.lock(); diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 914582e4ed..f23761e235 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -90,13 +90,9 @@ ServiceBase::set_events_executor_callback( const void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_ret_t ret = rcl_service_set_events_executor_callback( + rcl_service_set_events_executor_callback( executor_context, executor_callback, this, service_handle_.get()); - - if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); - } } diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 345bbf7181..1cab2fcdd8 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -294,13 +294,9 @@ SubscriptionBase::set_events_executor_callback( const void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_ret_t ret = rcl_subscription_set_events_executor_callback( + rcl_subscription_set_events_executor_callback( executor_context, executor_callback, this, subscription_handle_.get()); - - if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set the EventsExecutor's callback to subscription"); - } } diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index ae2bad93f4..673a5d27c8 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -44,14 +44,10 @@ SubscriptionIntraProcessBase::set_events_executor_callback( void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_guard_condition_set_events_executor_callback( executor_context, executor_callback, this, &gc_, true /*Use previous events*/); - - if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } } diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index d9e34305bb..a4dfabad72 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -457,16 +457,12 @@ class TestWaitable : public rclcpp::Waitable void * executor_context, ExecutorEventCallback executor_callback) const override { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_guard_condition_set_events_executor_callback( executor_context, executor_callback, this, &gc_, true /*Use previous events*/); - - if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); - } } private: From 75635382c3a3d7fc0d048eae3a1468374d1701f0 Mon Sep 17 00:00:00 2001 From: iRobot ROS <49500531+irobot-ros@users.noreply.github.com> Date: Wed, 14 Oct 2020 12:16:23 +0100 Subject: [PATCH 048/168] Update client.hpp --- rclcpp/include/rclcpp/client.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 055a9d728d..3885861647 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -37,8 +37,8 @@ #include "rclcpp/expand_topic_or_service_name.hpp" #include "rclcpp/visibility_control.hpp" -#include "rcutils/logging_macros.h" #include "rcutils/executor_event_types.h" +#include "rcutils/logging_macros.h" #include "rmw/error_handling.h" #include "rmw/rmw.h" From 24ccf6e0d5cb590e00233d77f0a62f00c88b753e Mon Sep 17 00:00:00 2001 From: iRobot ROS <49500531+irobot-ros@users.noreply.github.com> Date: Wed, 14 Oct 2020 12:16:40 +0100 Subject: [PATCH 049/168] Update qos_event.hpp --- rclcpp/include/rclcpp/qos_event.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 259aa5ab54..2ca4c3d02f 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -21,8 +21,8 @@ #include "rcl/error_handling.h" #include "rmw/incompatible_qos_events_statuses.h" -#include "rcutils/logging_macros.h" #include "rcutils/executor_event_types.h" +#include "rcutils/logging_macros.h" #include "rclcpp/exceptions.hpp" #include "rclcpp/function_traits.hpp" From 92d2883f55f16111bc99792d9eb66ef3bae9c717 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Wed, 14 Oct 2020 15:05:00 +0100 Subject: [PATCH 050/168] clean add/remove node APIs and make sure that event callbacks are removed when destroying executor Signed-off-by: Soragna, Alberto --- .../events_executor_entities_collector.hpp | 3 + .../src/rclcpp/executors/events_executor.cpp | 5 +- .../events_executor_entities_collector.cpp | 162 ++++++++++-------- 3 files changed, 99 insertions(+), 71 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 0b4db0ae17..f27cb3c2b0 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -97,6 +97,9 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable void set_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); + void + unset_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); + /// List of weak nodes registered in the events executor std::list weak_nodes_; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 41e796ece5..def8289c12 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -237,10 +237,9 @@ EventsExecutor::remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { (void)notify; - entities_collector_->remove_node(node_ptr); - std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); - has_executor.store(false); + // Remove node from entities collector + entities_collector_->remove_node(node_ptr); } void diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index f204bcae21..eb821c4bb4 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -33,8 +33,11 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() if (node) { std::atomic_bool & has_executor = node->get_associated_with_executor_atomic(); has_executor.store(false); + this->unset_entities_callbacks(node); } } + + // Make sure that the list is empty weak_nodes_.clear(); } @@ -59,7 +62,7 @@ void EventsExecutorEntitiesCollector::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { - // If the node already has an executor + // Check if the node already has an executor and if not, set this to true std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); if (has_executor.exchange(true)) { @@ -69,94 +72,46 @@ EventsExecutorEntitiesCollector::add_node( weak_nodes_.push_back(node_ptr); set_entities_callbacks(node_ptr); - - // Set node's guard condition callback, so if new entities are added while - // spinning we can set their callback. - rcl_guard_condition_set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event, - this, - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); } void EventsExecutorEntitiesCollector::remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { + // Check if this node is currently stored here auto node_it = weak_nodes_.begin(); - - // Here we unset the node entities callback. while (node_it != weak_nodes_.end()) { - bool matched = (node_it->lock() == node_ptr); - if (matched) { - // Node found: unset its entities callbacks - rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node_ptr->get_notify_guard_condition(), - false); - - // Unset entities callbacks - for (auto & weak_group : node_ptr->get_callback_groups()) { - auto group = weak_group.lock(); - if (!group || !group->can_be_taken_from().load()) { - continue; - } - group->find_timer_ptrs_if( - [this](const rclcpp::TimerBase::SharedPtr & timer) { - if (timer) { - timers_manager_->remove_timer(timer); - } - return false; - }); - group->find_subscription_ptrs_if( - [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { - if (subscription) { - subscription->set_events_executor_callback(nullptr, nullptr); - } - return false; - }); - group->find_service_ptrs_if( - [this](const rclcpp::ServiceBase::SharedPtr & service) { - if (service) { - service->set_events_executor_callback(nullptr, nullptr); - } - return false; - }); - group->find_client_ptrs_if( - [this](const rclcpp::ClientBase::SharedPtr & client) { - if (client) { - client->set_events_executor_callback(nullptr, nullptr); - } - return false; - }); - group->find_waitable_ptrs_if( - [this](const rclcpp::Waitable::SharedPtr & waitable) { - if (waitable) { - waitable->set_events_executor_callback(nullptr, nullptr); - } - return false; - }); - } - - weak_nodes_.erase(node_it); - return; - } else { - ++node_it; + if (node_it->lock() == node_ptr) { + break; } } + if (node_it == weak_nodes_.end()) { + // The node is not stored here, so nothing to do + return; + } + + // Set that the node does not have an executor anymore + std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); + has_executor.store(false); + + weak_nodes_.erase(node_it); + + unset_entities_callbacks(node_ptr); } void EventsExecutorEntitiesCollector::set_entities_callbacks( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node) { - // Check in all the callback groups + // Set event callbacks for all entities in this node + // by searching them in all callback groups for (auto & weak_group : node->get_callback_groups()) { auto group = weak_group.lock(); if (!group || !group->can_be_taken_from().load()) { continue; } + + // Timers are handled by the timers manager group->find_timer_ptrs_if( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { @@ -164,6 +119,8 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( } return false; }); + + // Set callbacks for all other entity types group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { @@ -193,4 +150,73 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( return false; }); } + + // Set an event callback for the node's guard condition, so if new entities are added + // or remove to this node we will receive an event. + rcl_guard_condition_set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event, + this, + node->get_notify_guard_condition(), + false /* Discard previous events */); +} + +void +EventsExecutorEntitiesCollector::unset_entities_callbacks( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node) +{ + // Unset event callbacks for all entities in this node + // by searching them in all callback groups + for (auto & weak_group : node->get_callback_groups()) { + auto group = weak_group.lock(); + if (!group || !group->can_be_taken_from().load()) { + continue; + } + + // Timers are handled by the timers manager + group->find_timer_ptrs_if( + [this](const rclcpp::TimerBase::SharedPtr & timer) { + if (timer) { + timers_manager_->remove_timer(timer); + } + return false; + }); + + // Unset callbacks for all other entity types + group->find_subscription_ptrs_if( + [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { + if (subscription) { + subscription->set_events_executor_callback(nullptr, nullptr); + } + return false; + }); + group->find_service_ptrs_if( + [this](const rclcpp::ServiceBase::SharedPtr & service) { + if (service) { + service->set_events_executor_callback(nullptr, nullptr); + } + return false; + }); + group->find_client_ptrs_if( + [this](const rclcpp::ClientBase::SharedPtr & client) { + if (client) { + client->set_events_executor_callback(nullptr, nullptr); + } + return false; + }); + group->find_waitable_ptrs_if( + [this](const rclcpp::Waitable::SharedPtr & waitable) { + if (waitable) { + waitable->set_events_executor_callback(nullptr, nullptr); + } + return false; + }); + } + + // Unset the event callback for the node's guard condition, to stop receiving events + // if entities are added or removed to this node. + rcl_guard_condition_set_events_executor_callback( + nullptr, nullptr, nullptr, + node->get_notify_guard_condition(), + false); } From 6f32325fda698f7f10aebb66d0242460a7736498 Mon Sep 17 00:00:00 2001 From: iRobot ROS <49500531+irobot-ros@users.noreply.github.com> Date: Wed, 14 Oct 2020 15:50:22 +0100 Subject: [PATCH 051/168] Revert "void return on set_events_executor_callback" --- rclcpp/include/rclcpp/client.hpp | 1 - rclcpp/include/rclcpp/qos_event.hpp | 7 +++++-- rclcpp/src/rclcpp/client.cpp | 6 +++++- rclcpp/src/rclcpp/executors/events_executor.cpp | 14 ++++++++++++-- .../events_executor_entities_collector.cpp | 12 ++++++++++-- rclcpp/src/rclcpp/service.cpp | 6 +++++- rclcpp/src/rclcpp/subscription_base.cpp | 6 +++++- .../src/rclcpp/subscription_intra_process_base.cpp | 6 +++++- rclcpp/test/rclcpp/executors/test_executors.cpp | 6 +++++- 9 files changed, 52 insertions(+), 12 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 3885861647..129f7209af 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -37,7 +37,6 @@ #include "rclcpp/expand_topic_or_service_name.hpp" #include "rclcpp/visibility_control.hpp" -#include "rcutils/executor_event_types.h" #include "rcutils/logging_macros.h" #include "rmw/error_handling.h" diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 2ca4c3d02f..58bfa0331d 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -21,7 +21,6 @@ #include "rcl/error_handling.h" #include "rmw/incompatible_qos_events_statuses.h" -#include "rcutils/executor_event_types.h" #include "rcutils/logging_macros.h" #include "rclcpp/exceptions.hpp" @@ -156,12 +155,16 @@ class QOSEventHandler : public QOSEventHandlerBase void * executor_context, ExecutorEventCallback executor_callback) const override { - rcl_event_set_events_executor_callback( + rcl_ret_t ret = rcl_event_set_events_executor_callback( executor_context, executor_callback, this, &event_handle_, false /* Discard previous events */); + + if (RCL_RET_OK != ret) { + throw std::runtime_error("Couldn't set EventsExecutor's callback to event"); + } } private: diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index c9e87384fa..4f8604ed90 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -204,9 +204,13 @@ ClientBase::set_events_executor_callback( const void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_client_set_events_executor_callback( + rcl_ret_t ret = rcl_client_set_events_executor_callback( executor_context, executor_callback, this, client_handle_.get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); + } } diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 41e796ece5..066adafff6 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -31,21 +31,31 @@ EventsExecutor::EventsExecutor( timers_manager_ = std::make_shared(context_); entities_collector_ = std::make_shared(this, timers_manager_); + rcl_ret_t ret; + // Set the global ctrl-c guard condition callback - rcl_guard_condition_set_events_executor_callback( + ret = rcl_guard_condition_set_events_executor_callback( this, &EventsExecutor::push_event, entities_collector_.get(), options.context->get_interrupt_guard_condition(&wait_set_), false /* Discard previous events */); + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set ctrl-c guard condition callback"); + } + // Set the executor interrupt guard condition callback - rcl_guard_condition_set_events_executor_callback( + ret = rcl_guard_condition_set_events_executor_callback( this, &EventsExecutor::push_event, entities_collector_.get(), &interrupt_guard_condition_, false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set interrupt guard condition callback"); + } } EventsExecutor::~EventsExecutor() {} diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index f204bcae21..bc1f6cbdd0 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -72,12 +72,16 @@ EventsExecutorEntitiesCollector::add_node( // Set node's guard condition callback, so if new entities are added while // spinning we can set their callback. - rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( associated_executor_, &EventsExecutor::push_event, this, node_ptr->get_notify_guard_condition(), false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set node guard condition callback"); + } } void @@ -91,11 +95,15 @@ EventsExecutorEntitiesCollector::remove_node( bool matched = (node_it->lock() == node_ptr); if (matched) { // Node found: unset its entities callbacks - rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( nullptr, nullptr, nullptr, node_ptr->get_notify_guard_condition(), false); + if (ret != RCL_RET_OK) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } + // Unset entities callbacks for (auto & weak_group : node_ptr->get_callback_groups()) { auto group = weak_group.lock(); diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index f23761e235..914582e4ed 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -90,9 +90,13 @@ ServiceBase::set_events_executor_callback( const void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_service_set_events_executor_callback( + rcl_ret_t ret = rcl_service_set_events_executor_callback( executor_context, executor_callback, this, service_handle_.get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); + } } diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 1cab2fcdd8..345bbf7181 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -294,9 +294,13 @@ SubscriptionBase::set_events_executor_callback( const void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_subscription_set_events_executor_callback( + rcl_ret_t ret = rcl_subscription_set_events_executor_callback( executor_context, executor_callback, this, subscription_handle_.get()); + + if (RCL_RET_OK != ret) { + throw std::runtime_error("Couldn't set the EventsExecutor's callback to subscription"); + } } diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 673a5d27c8..ae2bad93f4 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -44,10 +44,14 @@ SubscriptionIntraProcessBase::set_events_executor_callback( void * executor_context, ExecutorEventCallback executor_callback) const { - rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor_context, executor_callback, this, &gc_, true /*Use previous events*/); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } } diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index a4dfabad72..d9e34305bb 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -457,12 +457,16 @@ class TestWaitable : public rclcpp::Waitable void * executor_context, ExecutorEventCallback executor_callback) const override { - rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor_context, executor_callback, this, &gc_, true /*Use previous events*/); + + if (RCL_RET_OK != ret) { + throw std::runtime_error(std::string("Couldn't set guard condition callback")); + } } private: From 0e022f62a12a6866a13865d038aa9ce459a3d317 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Wed, 14 Oct 2020 16:17:22 +0100 Subject: [PATCH 052/168] more thread safety and events executor test Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/client.hpp | 4 +- .../rclcpp/executors/events_executor.hpp | 2 +- .../events_executor_entities_collector.hpp | 3 +- .../rclcpp/executors/timers_manager.hpp | 35 +-- rclcpp/include/rclcpp/service.hpp | 4 +- rclcpp/include/rclcpp/subscription_base.hpp | 4 +- .../src/rclcpp/executors/events_executor.cpp | 4 - .../events_executor_entities_collector.cpp | 26 +- .../src/rclcpp/executors/timers_manager.cpp | 146 ++++++--- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/CMakeLists.txt | 9 + .../rclcpp/executors/test_events_executor.cpp | 284 ++++++++++++++++++ .../test/rclcpp/executors/test_executors.cpp | 51 ++++ .../rclcpp/executors/test_timers_manager.cpp | 143 +++++++-- 14 files changed, 593 insertions(+), 124 deletions(-) create mode 100644 rclcpp/test/rclcpp/executors/test_events_executor.cpp diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 129f7209af..ab4f6755d2 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -152,7 +152,9 @@ class ClientBase RCLCPP_PUBLIC void - set_events_executor_callback(const void * executor_context, ExecutorEventCallback executor_callback) const; + set_events_executor_callback( + const void * executor_context, + ExecutorEventCallback executor_callback) const; protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 338687e8b8..ede9eaa4c4 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -173,7 +173,7 @@ class EventsExecutor : public rclcpp::Executor std::mutex event_queue_mutex_; std::condition_variable event_queue_cv_; - // Timers heap manager + // Timers manager std::shared_ptr timers_manager_; }; diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index f27cb3c2b0..bd20f6de22 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -16,6 +16,7 @@ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ #include +#include #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node_interfaces/node_base_interface.hpp" @@ -90,7 +91,7 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable add_to_wait_set(rcl_wait_set_t * wait_set) override { (void)wait_set; - return true; + return false; } private: diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 8ef3f9a3bc..0fd60131c4 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -15,14 +15,12 @@ #ifndef RCLCPP__EXECUTORS__TIMERS_MANAGER_HPP_ #define RCLCPP__EXECUTORS__TIMERS_MANAGER_HPP_ -#include #include #include #include #include #include #include -#include #include #include @@ -152,7 +150,6 @@ class TimersManager if (heap_.empty()) { return MAX_TIME; } - return (*heap_[0])->time_until_trigger(); } @@ -166,26 +163,19 @@ class TimersManager /** * @brief Helper function that checks whether a timer was already ready * at a specific timepoint - * @param tp the timepoint to check for * @param timer a pointer to the timer to check for + * @param tp the timepoint to check for * @return true if timer was ready at tp */ bool timer_was_ready_at_tp( - std::chrono::time_point tp, - TimerPtr timer) + TimerPtr timer, + std::chrono::time_point tp) { // A ready timer will return a negative duration when calling time_until_trigger auto time_ready = std::chrono::steady_clock::now() + (*timer)->time_until_trigger(); return time_ready < tp; } - /** - * @brief Rebuilds the heap queue from the timers storage - * This function is meant to be called whenever something changes in the timers storage. - * This function is not thread safe, you need to acquire a mutex before calling it. - */ - void rebuild_heap(); - /** * @brief Add a new timer to the heap and sort it. * @param x pointer to a timer owned by this object. @@ -245,27 +235,14 @@ class TimersManager heap_[i] = updated_timer; } - // Helper function to check the correctness of the heap. - void verify() - { - for (size_t i = 0; i < heap_.size()/2; ++i) { - size_t left = 2*i + 1; - if (left < heap_.size()) { - assert(((*heap_[left])->time_until_trigger().count() >= (*heap_[i])->time_until_trigger().count())); - } - size_t right = left + 1; - if (right < heap_.size()) { - assert(((*heap_[right])->time_until_trigger().count() >= (*heap_[i])->time_until_trigger().count())); - } - } - } - - // Thread used to run the timers monitoring and execution + // Thread used to run the timers monitoring and execution task std::thread timers_thread_; // Protects access to timers std::mutex timers_mutex_; // Notifies the timers thread whenever timers are added/removed std::condition_variable timers_cv_; + // Flag used as predicate by timers_cv + bool timers_updated_ {false}; // Indicates whether the timers thread is currently running or requested to stop std::atomic running_ {false}; // Context of the parent executor diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index eb5fd197f2..7ec076be52 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -123,7 +123,9 @@ class ServiceBase RCLCPP_PUBLIC void - set_events_executor_callback(const void * executor_context, ExecutorEventCallback executor_callback) const; + set_events_executor_callback( + const void * executor_context, + ExecutorEventCallback executor_callback) const; protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 320895bd33..06d8bd7974 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -265,7 +265,9 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void - set_events_executor_callback(const void * executor_context, ExecutorEventCallback executor_callback) const; + set_events_executor_callback( + const void * executor_context, + ExecutorEventCallback executor_callback) const; protected: template diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 414d988f0e..20c99ea751 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -318,9 +318,5 @@ EventsExecutor::execute_event(const ExecutorEvent & event) waitable->execute(); break; } - - default: - throw std::runtime_error("EventsExecutor received unrecognized event"); - break; } } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 8fe18b4f1b..449cca8a83 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include + #include "rclcpp/executors/events_executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" @@ -110,7 +112,7 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( if (!group || !group->can_be_taken_from().load()) { continue; } - + // Timers are handled by the timers manager group->find_timer_ptrs_if( [this](const rclcpp::TimerBase::SharedPtr & timer) { @@ -119,45 +121,53 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( } return false; }); - + // Set callbacks for all other entity types group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + subscription->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); } return false; }); group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + service->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); } return false; }); group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + client->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); } return false; }); group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_events_executor_callback(associated_executor_, &EventsExecutor::push_event); + waitable->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); } return false; }); } // Set an event callback for the node's notify guard condition, so if new entities are added - // or remove to this node we will receive an event. + // or removed to this node we will receive an event. rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( associated_executor_, &EventsExecutor::push_event, this, - node_ptr->get_notify_guard_condition(), + node->get_notify_guard_condition(), false /* Discard previous events */); if (ret != RCL_RET_OK) { diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 3d47c6597c..f4a92345a5 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "rclcpp/executors/timers_manager.hpp" - #include +#include + +#include "rclcpp/executors/timers_manager.hpp" using rclcpp::executors::TimersManager; @@ -29,14 +30,19 @@ TimersManager::~TimersManager() { // Make sure timers thread is stopped before destroying this object this->stop(); + // Remove all timers this->clear_all(); } void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) { + if (!timer) { + throw std::invalid_argument("TimersManager::add_timer() trying to add nullptr timer"); + } + { - std::unique_lock lock(timers_mutex_); + std::unique_lock timers_lock(timers_mutex_); // Make sure that the provided timer is not already in the timers storage if (std::find(timers_storage_.begin(), timers_storage_.end(), timer) != timers_storage_.end()) { @@ -46,12 +52,12 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) // Store ownership of timer and add it to heap timers_storage_.emplace_back(timer); this->add_timer_to_heap(&(timers_storage_.back())); + + timers_updated_ = true; } // Notify that a timer has been added to the heap timers_cv_.notify_one(); - - //verify(); } void TimersManager::start() @@ -61,6 +67,7 @@ void TimersManager::start() throw std::runtime_error("TimersManager::start() can't start timers thread as already running"); } + std::unique_lock timers_lock(timers_mutex_); running_ = true; timers_thread_ = std::thread(&TimersManager::run_timers, this); pthread_setname_np(timers_thread_.native_handle(), "TimersManager"); @@ -68,8 +75,17 @@ void TimersManager::start() void TimersManager::stop() { - // Notify stop to timers thread + // Nothing to do if the timers thread is not running + // or if another thred already signaled to stop. + if (!running_) { + return; + } + running_ = false; + { + std::unique_lock timers_lock(timers_mutex_); + timers_updated_ = true; + } timers_cv_.notify_one(); // Join timers thread if it's running @@ -80,92 +96,134 @@ void TimersManager::stop() std::chrono::nanoseconds TimersManager::get_head_timeout() { + // Do not allow to interfere with the thread running + if (running_) { + throw std::runtime_error( + "TimersManager::get_head_timeout() can't be used while timers thread is running"); + } + std::unique_lock lock(timers_mutex_); return this->get_head_timeout_unsafe(); } void TimersManager::execute_ready_timers() { + // Do not allow to interfere with the thread running + if (running_) { + throw std::runtime_error( + "TimersManager::execute_ready_timers() can't be used while timers thread is running"); + } + std::unique_lock lock(timers_mutex_); this->execute_ready_timers_unsafe(); } -void TimersManager::execute_ready_timers_unsafe() +bool TimersManager::execute_head_timer() { - if (heap_.empty()) { - return; + // Do not allow to interfere with the thread running + if (running_) { + throw std::runtime_error( + "TimersManager::execute_head_timer() can't be used while timers thread is running"); } - // Keep executing timers until they are read and they were already ready when we started. - // The second check prevents this function from blocking indefinitely if the - // time required for executing the timers is longer than their period. + std::unique_lock lock(timers_mutex_); + + // Nothing to do if we don't have any timer + if (heap_.empty()) { + return false; + } - auto start = std::chrono::steady_clock::now(); TimerPtr head = heap_.front(); - while ((*head)->is_ready() && timer_was_ready_at_tp(start, head)) { + if ((*head)->is_ready()) { + // Head timer is ready, execute and re-heapify (*head)->execute_callback(); this->restore_heap_root(); - head = heap_.front(); - //verify(); + return true; + } else { + // Head timer was not ready yet + return false; } } -bool TimersManager::execute_head_timer() +void TimersManager::execute_ready_timers_unsafe() { - std::unique_lock lock(timers_mutex_); - + // Nothing to do if we don't have any timer if (heap_.empty()) { - return false; + return; } + // Keep executing timers until they are read and they were already ready when we started. + // The second check prevents this function from blocking indefinitely if the + // time required for executing the timers is longer than their period. + + auto start = std::chrono::steady_clock::now(); TimerPtr head = heap_.front(); - if ((*head)->is_ready()) { + while ((*head)->is_ready() && this->timer_was_ready_at_tp(head, start)) { + // Execute head timer (*head)->execute_callback(); + // Executing a timer will result in updating its time_until_trigger, so re-heapify this->restore_heap_root(); - return true; - } else { - return false; + // Get new head timer + head = heap_.front(); } } void TimersManager::run_timers() { while (rclcpp::ok(context_) && running_) { - std::unique_lock lock(timers_mutex_); + // Lock mutex + std::unique_lock timers_lock(timers_mutex_); + // Get timeout before next timer expires auto time_to_sleep = this->get_head_timeout_unsafe(); - timers_cv_.wait_for(lock, time_to_sleep); + // Wait until timeout or notification that timers have been updated + timers_cv_.wait_for(timers_lock, time_to_sleep, [this]() {return timers_updated_;}); + // Reset timers updated flag + timers_updated_ = false; + // Execute timers this->execute_ready_timers_unsafe(); } + // Make sure the running flag is set to false when we exit from this function + // to allow restarting the timers thread. running_ = false; } void TimersManager::clear_all() { - std::unique_lock lock(timers_mutex_); - heap_.clear(); - timers_storage_.clear(); + { + // Lock mutex and then clear all data structures + std::unique_lock timers_lock(timers_mutex_); + heap_.clear(); + timers_storage_.clear(); + + timers_updated_ = true; + } + + // Notify timers thead such that it can re-compute its timeout + timers_cv_.notify_one(); } void TimersManager::remove_timer(rclcpp::TimerBase::SharedPtr timer) { - std::unique_lock lock(timers_mutex_); + { + std::unique_lock timers_lock(timers_mutex_); - // Make sure that we are currently storing the provided timer before proceeding - auto it = std::find(timers_storage_.begin(), timers_storage_.end(), timer); - if (it == timers_storage_.end()) { - return; - } + // Make sure that we are currently storing the provided timer before proceeding + auto it = std::find(timers_storage_.begin(), timers_storage_.end(), timer); + if (it == timers_storage_.end()) { + return; + } - // Remove timer from the storage and rebuild heap - timers_storage_.erase(it); - this->rebuild_heap(); -} + // Remove timer from the storage and rebuild heap + timers_storage_.erase(it); + heap_.clear(); + for (auto & t : timers_storage_) { + this->add_timer_to_heap(&t); + } -void TimersManager::rebuild_heap() -{ - heap_.clear(); - for (auto & t : timers_storage_) { - this->add_timer_to_heap(&t); + timers_updated_ = true; } + + // Notify timers thead such that it can re-compute its timeout + timers_cv_.notify_one(); } diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index a266c91eb7..372826a912 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -69,5 +69,5 @@ Waitable::set_events_executor_callback( (void)executor_callback; throw std::runtime_error( - "Custom waitables should override set_events_executor_callback() to use events executor"); + "Custom waitables should override set_events_executor_callback() to use events executor"); } diff --git a/rclcpp/test/CMakeLists.txt b/rclcpp/test/CMakeLists.txt index e5c8b98b36..939077ace7 100644 --- a/rclcpp/test/CMakeLists.txt +++ b/rclcpp/test/CMakeLists.txt @@ -548,6 +548,15 @@ if(TARGET test_executors) target_link_libraries(test_executors ${PROJECT_NAME}) endif() +ament_add_gtest(test_events_executor rclcpp/executors/test_events_executor.cpp + APPEND_LIBRARY_DIRS "${append_library_dirs}") +if(TARGET test_events_executor) + ament_target_dependencies(test_events_executor + "rcl" + "test_msgs") + target_link_libraries(test_events_executor ${PROJECT_NAME} mimick) +endif() + ament_add_gtest(test_static_single_threaded_executor rclcpp/executors/test_static_single_threaded_executor.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}") if(TARGET test_static_single_threaded_executor) diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp new file mode 100644 index 0000000000..c6b66deae1 --- /dev/null +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -0,0 +1,284 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include + +#include +#include + +#include "rclcpp/executors/events_executor.hpp" + +#include "test_msgs/srv/empty.hpp" + +using namespace std::chrono_literals; + +using rclcpp::executors::EventsExecutor; + +class TestEventsExecutor : public ::testing::Test +{ +public: + void SetUp() + { + rclcpp::init(0, nullptr); + } + + void TearDown() + { + rclcpp::shutdown(); + } +}; + +TEST_F(TestEventsExecutor, run_clients_servers) +{ + auto node = std::make_shared("node"); + + bool request_received = false; + bool response_received = false; + auto service = + node->create_service( + "service", + [&request_received]( + const test_msgs::srv::Empty::Request::SharedPtr, + test_msgs::srv::Empty::Response::SharedPtr) + { + request_received = true; + }); + auto client = node->create_client("service"); + + EventsExecutor executor; + executor.add_node(node); + + bool spin_exited = false; + std::thread spinner([&spin_exited, &executor, this]() { + executor.spin(); + spin_exited = true; + }); + + auto request = std::make_shared(); + client->async_send_request( + request, + [&response_received](rclcpp::Client::SharedFuture result_future) { + (void)result_future; + response_received = true; + }); + + // Wait some time for the client-server to be invoked + auto start = std::chrono::steady_clock::now(); + while ( + !response_received && + !spin_exited && + (std::chrono::steady_clock::now() - start < 1s)) + { + std::this_thread::sleep_for(5ms); + } + + executor.cancel(); + spinner.join(); + executor.remove_node(node); + + EXPECT_TRUE(request_received); + EXPECT_TRUE(response_received); + EXPECT_TRUE(spin_exited); +} + +TEST_F(TestEventsExecutor, spin_once_max_duration) +{ + { + auto node = std::make_shared("node"); + + size_t t_runs = 0; + auto t = node->create_wall_timer( + 10s, + [&]() { + t_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + + auto start = node->now(); + executor.spin_once(10ms); + + EXPECT_EQ(0u, t_runs); + EXPECT_TRUE(node->now() - start < 200ms); + } + + { + auto node = std::make_shared("node"); + + size_t t_runs = 0; + auto t = node->create_wall_timer( + 10ms, + [&]() { + t_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + + auto start = node->now(); + executor.spin_once(10s); + + EXPECT_EQ(1u, t_runs); + EXPECT_TRUE(node->now() - start < 200ms); + } +} + +TEST_F(TestEventsExecutor, spin_some_max_duration) +{ + { + auto node = std::make_shared("node"); + + size_t t_runs = 0; + auto t = node->create_wall_timer( + 10s, + [&]() { + t_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + + auto start = node->now(); + executor.spin_some(10ms); + + EXPECT_EQ(0u, t_runs); + EXPECT_TRUE(node->now() - start < 200ms); + } + + { + auto node = std::make_shared("node"); + + size_t t_runs = 0; + auto t = node->create_wall_timer( + 10ms, + [&]() { + t_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + + auto start = node->now(); + executor.spin_some(10s); + + EXPECT_EQ(1u, t_runs); + EXPECT_TRUE(node->now() - start < 200ms); + } +} + +TEST_F(TestEventsExecutor, spin_all_max_duration) +{ + { + auto node = std::make_shared("node"); + + size_t t_runs = 0; + auto t = node->create_wall_timer( + 10s, + [&]() { + t_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + + auto start = node->now(); + executor.spin_all(10ms); + + EXPECT_EQ(0u, t_runs); + EXPECT_TRUE(node->now() - start < 200ms); + } + + { + auto node = std::make_shared("node"); + + size_t t_runs = 0; + auto t = node->create_wall_timer( + 10ms, + [&]() { + t_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + + auto start = node->now(); + executor.spin_all(10s); + + EXPECT_EQ(1u, t_runs); + EXPECT_TRUE(node->now() - start < 200ms); + } + + EventsExecutor executor; + EXPECT_THROW(executor.spin_all(0ms), std::invalid_argument); + EXPECT_THROW(executor.spin_all(-5ms), std::invalid_argument); +} + +TEST_F(TestEventsExecutor, cancel_while_timers_running) +{ + auto node = std::make_shared("node"); + + size_t t1_runs = 0; + auto t1 = node->create_wall_timer( + 1ms, + [&]() { + t1_runs++; + std::this_thread::sleep_for(25ms); + }); + + size_t t2_runs = 0; + auto t2 = node->create_wall_timer( + 1ms, + [&]() { + t2_runs++; + std::this_thread::sleep_for(25ms); + }); + + EventsExecutor executor; + executor.add_node(node); + + std::thread spinner([&executor, this]() {executor.spin();}); + + std::this_thread::sleep_for(10ms); + // Call cancel while t1 callback is still being executed + executor.cancel(); + spinner.join(); + + // Depending on the latency on the system, t2 may start to execute before cancel is signaled + EXPECT_GE(1u, t1_runs); + EXPECT_GE(1u, t2_runs); +} + +TEST_F(TestEventsExecutor, cancel_while_timers_waiting) +{ + auto node = std::make_shared("node"); + + size_t t1_runs = 0; + auto t1 = node->create_wall_timer( + 100s, + [&]() { + t1_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + + std::thread spinner([&executor, this]() {executor.spin();}); + + std::this_thread::sleep_for(10ms); + executor.cancel(); + spinner.join(); + + EXPECT_EQ(0u, t1_runs); +} diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index d9e34305bb..b05ca0db6d 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -589,6 +589,57 @@ TYPED_TEST(TestExecutorsStable, testSpinNodeUntilFutureCompleteNodePtr) { EXPECT_EQ(rclcpp::FutureReturnCode::SUCCESS, ret); } +TYPED_TEST(TestExecutorsStable, testSpinSomeWhileSpinning) { + using ExecutorType = TypeParam; + ExecutorType executor; + + std::thread spinner([&]() {executor.spin();}); + + // Wait to make sure thread started + do { + std::this_thread::sleep_for(5ms); + } while (!spinner.joinable()); + + EXPECT_THROW(executor.spin_some(1s), std::runtime_error); + + executor.cancel(); + spinner.join(); +} + +TYPED_TEST(TestExecutorsStable, testSpinAllWhileSpinning) { + using ExecutorType = TypeParam; + ExecutorType executor; + + std::thread spinner([&]() {executor.spin();}); + + // Wait to make sure thread started + do { + std::this_thread::sleep_for(5ms); + } while (!spinner.joinable()); + + EXPECT_THROW(executor.spin_all(1s), std::runtime_error); + + executor.cancel(); + spinner.join(); +} + +TYPED_TEST(TestExecutorsStable, testSpinOnceWhileSpinning) { + using ExecutorType = TypeParam; + ExecutorType executor; + + std::thread spinner([&]() {executor.spin();}); + + // Wait to make sure thread started + do { + std::this_thread::sleep_for(5ms); + } while (!spinner.joinable()); + + EXPECT_THROW(executor.spin_once(), std::runtime_error); + + executor.cancel(); + spinner.join(); +} + // Check spin_until_future_complete with node base pointer (instantiates its own executor) TEST(TestExecutors, testSpinUntilFutureCompleteNodeBasePtr) { rclcpp::init(0, nullptr); diff --git a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp index 0aea7e5c58..446bf74037 100644 --- a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp +++ b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp @@ -16,6 +16,7 @@ #include #include +#include #include "rclcpp/contexts/default_context.hpp" #include "rclcpp/executors/timers_manager.hpp" @@ -24,7 +25,7 @@ using namespace std::chrono_literals; using rclcpp::executors::TimersManager; -using CallbackT = std::function; +using CallbackT = std::function; using TimerT = rclcpp::WallTimer; class TestTimersManager : public ::testing::Test @@ -41,8 +42,8 @@ class TestTimersManager : public ::testing::Test } }; -TEST_F(TestTimersManager, empty_manager) { - +TEST_F(TestTimersManager, empty_manager) +{ auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); @@ -54,12 +55,21 @@ TEST_F(TestTimersManager, empty_manager) { EXPECT_NO_THROW(timers_manager->stop()); } -TEST_F(TestTimersManager, add_run_remove_timer) { +TEST_F(TestTimersManager, max_time) +{ + // Timers manager max time should be a very big duration that does not cause overflow + + EXPECT_LT(std::chrono::hours(50).count(), TimersManager::MAX_TIME.count()); + auto tp = std::chrono::steady_clock::now() + TimersManager::MAX_TIME; + EXPECT_TRUE(std::chrono::steady_clock::now() < tp); +} +TEST_F(TestTimersManager, add_run_remove_timer) +{ size_t t_runs = 0; auto t = TimerT::make_shared( 1ms, - [&t_runs](){ + [&t_runs]() { t_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -89,7 +99,8 @@ TEST_F(TestTimersManager, add_run_remove_timer) { EXPECT_FALSE(t_tmp != nullptr); } -TEST_F(TestTimersManager, clear_all) { +TEST_F(TestTimersManager, clear_all) +{ auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); @@ -110,8 +121,8 @@ TEST_F(TestTimersManager, clear_all) { EXPECT_FALSE(t2_weak.lock() != nullptr); } -TEST_F(TestTimersManager, remove_not_existing_timer) { - +TEST_F(TestTimersManager, remove_not_existing_timer) +{ auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); @@ -126,15 +137,73 @@ TEST_F(TestTimersManager, remove_not_existing_timer) { EXPECT_NO_THROW(timers_manager->remove_timer(t)); } -TEST_F(TestTimersManager, timers_order) { +TEST_F(TestTimersManager, timers_thread_exclusive_usage) +{ + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + timers_manager->start(); + + EXPECT_THROW(timers_manager->get_head_timeout(), std::exception); + EXPECT_THROW(timers_manager->execute_ready_timers(), std::exception); + EXPECT_THROW(timers_manager->execute_head_timer(), std::exception); + + timers_manager->stop(); + + EXPECT_NO_THROW(timers_manager->get_head_timeout()); + EXPECT_NO_THROW(timers_manager->execute_ready_timers()); + EXPECT_NO_THROW(timers_manager->execute_head_timer()); +} + +TEST_F(TestTimersManager, add_timer_twice) +{ + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + auto t = TimerT::make_shared(1ms, CallbackT(), rclcpp::contexts::get_global_default_context()); + + timers_manager->add_timer(t); + EXPECT_NO_THROW(timers_manager->add_timer(t)); +} + +TEST_F(TestTimersManager, add_nullptr) +{ + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + EXPECT_THROW(timers_manager->add_timer(nullptr), std::exception); +} +TEST_F(TestTimersManager, head_not_ready) +{ + auto timers_manager = std::make_shared( + rclcpp::contexts::get_global_default_context()); + + size_t t_runs = 0; + auto t = TimerT::make_shared( + 10s, + [&t_runs]() { + t_runs++; + }, + rclcpp::contexts::get_global_default_context()); + + timers_manager->add_timer(t); + + // Timer will take 10s to get ready, so nothing to execute here + bool ret = timers_manager->execute_head_timer(); + EXPECT_FALSE(ret); + EXPECT_EQ(0u, t_runs); +} + +TEST_F(TestTimersManager, timers_order) +{ auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); size_t t1_runs = 0; auto t1 = TimerT::make_shared( 1ms, - [&t1_runs](){ + [&t1_runs]() { t1_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -142,15 +211,15 @@ TEST_F(TestTimersManager, timers_order) { size_t t2_runs = 0; auto t2 = TimerT::make_shared( 3ms, - [&t2_runs](){ + [&t2_runs]() { t2_runs++; }, rclcpp::contexts::get_global_default_context()); size_t t3_runs = 0; auto t3 = TimerT::make_shared( - 5ms, - [&t3_runs](){ + 10ms, + [&t3_runs]() { t3_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -166,25 +235,33 @@ TEST_F(TestTimersManager, timers_order) { EXPECT_EQ(0u, t2_runs); EXPECT_EQ(0u, t3_runs); - std::this_thread::sleep_for(2ms); + std::this_thread::sleep_for(3ms); timers_manager->execute_ready_timers(); EXPECT_EQ(2u, t1_runs); EXPECT_EQ(1u, t2_runs); EXPECT_EQ(0u, t3_runs); - std::this_thread::sleep_for(5ms); + std::this_thread::sleep_for(10ms); timers_manager->execute_ready_timers(); EXPECT_EQ(3u, t1_runs); EXPECT_EQ(2u, t2_runs); EXPECT_EQ(1u, t3_runs); -} -TEST_F(TestTimersManager, start_stop_timers_thread) { + timers_manager->remove_timer(t1); + std::this_thread::sleep_for(3ms); + timers_manager->execute_ready_timers(); + EXPECT_EQ(3u, t1_runs); + EXPECT_EQ(3u, t2_runs); + EXPECT_EQ(1u, t3_runs); +} + +TEST_F(TestTimersManager, start_stop_timers_thread) +{ auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); - auto t = TimerT::make_shared(1ms, [](){}, rclcpp::contexts::get_global_default_context()); + auto t = TimerT::make_shared(1ms, []() {}, rclcpp::contexts::get_global_default_context()); timers_manager->add_timer(t); // Calling start multiple times will throw an error @@ -196,15 +273,15 @@ TEST_F(TestTimersManager, start_stop_timers_thread) { EXPECT_NO_THROW(timers_manager->stop()); } -TEST_F(TestTimersManager, timers_thread) { - +TEST_F(TestTimersManager, timers_thread) +{ auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); size_t t1_runs = 0; auto t1 = TimerT::make_shared( 1ms, - [&t1_runs](){ + [&t1_runs]() { t1_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -212,7 +289,7 @@ TEST_F(TestTimersManager, timers_thread) { size_t t2_runs = 0; auto t2 = TimerT::make_shared( 1ms, - [&t2_runs](){ + [&t2_runs]() { t2_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -231,12 +308,12 @@ TEST_F(TestTimersManager, timers_thread) { EXPECT_EQ(t1_runs, t2_runs); } -TEST_F(TestTimersManager, destructor) { - +TEST_F(TestTimersManager, destructor) +{ size_t t_runs = 0; auto t = TimerT::make_shared( 1ms, - [&t_runs](){ + [&t_runs]() { t_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -263,15 +340,15 @@ TEST_F(TestTimersManager, destructor) { EXPECT_FALSE(t_weak.lock() != nullptr); } -TEST_F(TestTimersManager, add_remove_while_thread_running) { - +TEST_F(TestTimersManager, add_remove_while_thread_running) +{ auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); size_t t1_runs = 0; auto t1 = TimerT::make_shared( 1ms, - [&t1_runs](){ + [&t1_runs]() { t1_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -279,7 +356,7 @@ TEST_F(TestTimersManager, add_remove_while_thread_running) { size_t t2_runs = 0; auto t2 = TimerT::make_shared( 1ms, - [&t2_runs](){ + [&t2_runs]() { t2_runs++; }, rclcpp::contexts::get_global_default_context()); @@ -306,8 +383,8 @@ TEST_F(TestTimersManager, add_remove_while_thread_running) { EXPECT_LT(1u, t2_runs); } -TEST_F(TestTimersManager, infinite_loop) { - +TEST_F(TestTimersManager, infinite_loop) +{ // This test makes sure that even if timers have a period shorter than the duration // of their callback the functions never block indefinitely. @@ -317,7 +394,7 @@ TEST_F(TestTimersManager, infinite_loop) { size_t t1_runs = 0; auto t1 = TimerT::make_shared( 1ms, - [&t1_runs](){ + [&t1_runs]() { t1_runs++; std::this_thread::sleep_for(5ms); }, @@ -326,7 +403,7 @@ TEST_F(TestTimersManager, infinite_loop) { size_t t2_runs = 0; auto t2 = TimerT::make_shared( 1ms, - [&t2_runs](){ + [&t2_runs]() { t2_runs++; std::this_thread::sleep_for(5ms); }, From da55bd9bca659812c61d9bd0aa0f3719af8bb9dc Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Wed, 14 Oct 2020 16:42:22 +0100 Subject: [PATCH 053/168] remove atomic and use mutex to protect running_ flag Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/timers_manager.hpp | 2 +- .../src/rclcpp/executors/timers_manager.cpp | 35 ++++++++++++------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 0fd60131c4..6fb0fa767c 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -244,7 +244,7 @@ class TimersManager // Flag used as predicate by timers_cv bool timers_updated_ {false}; // Indicates whether the timers thread is currently running or requested to stop - std::atomic running_ {false}; + bool running_ {false}; // Context of the parent executor std::shared_ptr context_; // Container to keep ownership of the timers diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index f4a92345a5..b60a42f3cc 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -62,12 +62,13 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) void TimersManager::start() { + std::unique_lock timers_lock(timers_mutex_); + // Make sure that the thread is not already running if (running_) { throw std::runtime_error("TimersManager::start() can't start timers thread as already running"); } - std::unique_lock timers_lock(timers_mutex_); running_ = true; timers_thread_ = std::thread(&TimersManager::run_timers, this); pthread_setname_np(timers_thread_.native_handle(), "TimersManager"); @@ -75,15 +76,15 @@ void TimersManager::start() void TimersManager::stop() { - // Nothing to do if the timers thread is not running - // or if another thred already signaled to stop. - if (!running_) { - return; - } - - running_ = false; { std::unique_lock timers_lock(timers_mutex_); + // Nothing to do if the timers thread is not running + // or if another thred already signaled to stop. + if (!running_) { + return; + } + + running_ = false; timers_updated_ = true; } timers_cv_.notify_one(); @@ -96,38 +97,40 @@ void TimersManager::stop() std::chrono::nanoseconds TimersManager::get_head_timeout() { + std::unique_lock lock(timers_mutex_); + // Do not allow to interfere with the thread running if (running_) { throw std::runtime_error( "TimersManager::get_head_timeout() can't be used while timers thread is running"); } - std::unique_lock lock(timers_mutex_); return this->get_head_timeout_unsafe(); } void TimersManager::execute_ready_timers() { + std::unique_lock lock(timers_mutex_); + // Do not allow to interfere with the thread running if (running_) { throw std::runtime_error( "TimersManager::execute_ready_timers() can't be used while timers thread is running"); } - std::unique_lock lock(timers_mutex_); this->execute_ready_timers_unsafe(); } bool TimersManager::execute_head_timer() { + std::unique_lock lock(timers_mutex_); + // Do not allow to interfere with the thread running if (running_) { throw std::runtime_error( "TimersManager::execute_head_timer() can't be used while timers thread is running"); } - std::unique_lock lock(timers_mutex_); - // Nothing to do if we don't have any timer if (heap_.empty()) { return false; @@ -170,9 +173,14 @@ void TimersManager::execute_ready_timers_unsafe() void TimersManager::run_timers() { - while (rclcpp::ok(context_) && running_) { + while (rclcpp::ok(context_)) { // Lock mutex std::unique_lock timers_lock(timers_mutex_); + + if (!running_) { + break; + } + // Get timeout before next timer expires auto time_to_sleep = this->get_head_timeout_unsafe(); // Wait until timeout or notification that timers have been updated @@ -185,6 +193,7 @@ void TimersManager::run_timers() // Make sure the running flag is set to false when we exit from this function // to allow restarting the timers thread. + std::unique_lock timers_lock(timers_mutex_); running_ = false; } From a77ecdbf415ca2296d3b728a8c86fb44c42e95c9 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Thu, 15 Oct 2020 12:55:21 +0100 Subject: [PATCH 054/168] minor cleanup of comments and spin_some timeout Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/executors/events_executor.hpp | 5 +++-- .../executors/events_executor_entities_collector.hpp | 4 +--- rclcpp/src/rclcpp/executors/events_executor.cpp | 7 +++++++ 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index ede9eaa4c4..700225c81c 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -168,11 +168,12 @@ class EventsExecutor : public rclcpp::Executor void execute_event(const ExecutorEvent & event); - // Event queue members + // Event queue std::queue event_queue_; + // Mutex to protect insertion and extraction of events in the queue std::mutex event_queue_mutex_; + // Variable used to notify when an event is added to the queue std::condition_variable event_queue_cv_; - // Timers manager std::shared_ptr timers_manager_; }; diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index bd20f6de22..d34e709b35 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -103,11 +103,9 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable /// List of weak nodes registered in the events executor std::list weak_nodes_; - /// Executor using this entities collector object EventsExecutor * associated_executor_ = nullptr; - - // Instance of the timers manager used by the associated executor + /// Instance of the timers manager used by the associated executor TimersManager::SharedPtr timers_manager_; }; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 20c99ea751..87c0686a1f 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -101,6 +101,11 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); + // In this context a 0 input max_duration means no duration limit + if (std::chrono::nanoseconds(0) == max_duration) { + max_duration = timers_manager_->MAX_TIME; + } + // This function will wait until the first of the following events occur: // - The input max_duration is elapsed // - A timer triggers @@ -230,6 +235,7 @@ void EventsExecutor::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { + // This field is unused because we don't have to wake up the executor when a node is added. (void) notify; // Add node to entities collector @@ -246,6 +252,7 @@ void EventsExecutor::remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, bool notify) { + // This field is unused because we don't have to wake up the executor when a node is removed. (void)notify; // Remove node from entities collector From 9a17991f695f8968c2ad57dda78575b2d29184c8 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Fri, 16 Oct 2020 09:20:20 +0100 Subject: [PATCH 055/168] complete destruction callback implementation Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/client.hpp | 7 ++ .../rclcpp/executors/events_executor.hpp | 46 +++++++-- .../rclcpp/executors/timers_manager.hpp | 16 ++-- rclcpp/include/rclcpp/service.hpp | 6 ++ rclcpp/include/rclcpp/subscription_base.hpp | 9 +- rclcpp/include/rclcpp/timer.hpp | 5 + rclcpp/include/rclcpp/waitable.hpp | 8 +- rclcpp/src/rclcpp/client.cpp | 8 ++ .../src/rclcpp/executors/events_executor.cpp | 79 ++++++++-------- .../events_executor_entities_collector.cpp | 17 ++++ .../src/rclcpp/executors/timers_manager.cpp | 30 ++++-- rclcpp/src/rclcpp/service.cpp | 11 ++- rclcpp/src/rclcpp/subscription_base.cpp | 14 +++ rclcpp/src/rclcpp/timer.cpp | 11 ++- rclcpp/src/rclcpp/waitable.cpp | 13 +++ .../rclcpp/executors/test_events_executor.cpp | 93 ++++++++++--------- .../rclcpp/executors/test_timers_manager.cpp | 22 ++--- 17 files changed, 275 insertions(+), 120 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index ab4f6755d2..8045f60d05 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -16,6 +16,7 @@ #define RCLCPP__CLIENT_HPP_ #include +#include #include #include #include @@ -156,6 +157,10 @@ class ClientBase const void * executor_context, ExecutorEventCallback executor_callback) const; + RCLCPP_PUBLIC + void + set_on_destruction_callback(std::function on_destruction_callback); + protected: RCLCPP_DISABLE_COPY(ClientBase) @@ -171,6 +176,8 @@ class ClientBase const rcl_node_t * get_rcl_node_handle() const; + std::function on_destruction_callback_; + rclcpp::node_interfaces::NodeGraphInterface::WeakPtr node_graph_; std::shared_ptr node_handle_; std::shared_ptr context_; diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 700225c81c..7ca0bdbc86 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -16,6 +16,7 @@ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ #include +#include #include #include @@ -140,14 +141,37 @@ class EventsExecutor : public rclcpp::Executor // Event queue mutex scope { - std::unique_lock lock(this_executor->event_queue_mutex_); + std::unique_lock lock(this_executor->push_mutex_); - this_executor->event_queue_.push(event); + this_executor->event_queue_.push_back(event); } // Notify that the event queue has some events in it. this_executor->event_queue_cv_.notify_one(); } + template + void + remove_entity(T* entity) + { + // We need to unset the callbacks to make sure that after removing events from the + // queues, this entity will not push anymore before being completely destroyed. + entity->set_events_executor_callback(nullptr, nullptr); + + // Remove events associated with this entity from the event queue + { + std::unique_lock lock(push_mutex_); + event_queue_.erase(std::remove_if(event_queue_.begin(), event_queue_.end(), + [&entity](ExecutorEvent event) { return event.entity == entity; }), event_queue_.end()); + } + + // Remove events associated with this entity from the local event queue + { + std::unique_lock lock(execution_mutex_); + local_event_queue_.erase(std::remove_if(local_event_queue_.begin(), local_event_queue_.end(), + [&entity](ExecutorEvent event) { return event.entity == entity; }), local_event_queue_.end()); + } + } + protected: RCLCPP_PUBLIC void @@ -156,22 +180,30 @@ class EventsExecutor : public rclcpp::Executor private: RCLCPP_DISABLE_COPY(EventsExecutor) + // Event queue implementation is a deque only to + // facilitate the removal of events from expired entities. + using EventQueue = std::deque; + EventsExecutorEntitiesCollector::SharedPtr entities_collector_; /// Extract and execute events from the queue until it is empty RCLCPP_PUBLIC void - consume_all_events(std::queue & queue); + consume_all_events(EventQueue & queue); // Execute a single event RCLCPP_PUBLIC void execute_event(const ExecutorEvent & event); - // Event queue - std::queue event_queue_; - // Mutex to protect insertion and extraction of events in the queue - std::mutex event_queue_mutex_; + // We use two instances of EventQueue to allow threads to push events while we execute them + EventQueue event_queue_; + EventQueue local_event_queue_; + + // Mutex to protect the insertion of events in the queue + std::mutex push_mutex_; + // Mutex to protect the execution of events + std::mutex execution_mutex_; // Variable used to notify when an event is added to the queue std::condition_variable event_queue_cv_; // Timers manager diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 6fb0fa767c..07331626c4 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -121,6 +121,8 @@ class TimersManager */ void remove_timer(rclcpp::TimerBase::SharedPtr timer); + void remove_timer_raw(rclcpp::TimerBase* timer); + // This is what the TimersManager uses to denote a duration forever. // We don't use std::chrono::nanoseconds::max because it will overflow. // See https://en.cppreference.com/w/cpp/thread/condition_variable/wait_for @@ -129,7 +131,7 @@ class TimersManager private: RCLCPP_DISABLE_COPY(TimersManager) - using TimerPtr = rclcpp::TimerBase::SharedPtr *; + using TimerPtr = rclcpp::TimerBase *; /** * @brief Implements a loop that keeps executing ready timers. @@ -150,7 +152,7 @@ class TimersManager if (heap_.empty()) { return MAX_TIME; } - return (*heap_[0])->time_until_trigger(); + return (heap_[0])->time_until_trigger(); } /** @@ -172,7 +174,7 @@ class TimersManager std::chrono::time_point tp) { // A ready timer will return a negative duration when calling time_until_trigger - auto time_ready = std::chrono::steady_clock::now() + (*timer)->time_until_trigger(); + auto time_ready = std::chrono::steady_clock::now() + (timer)->time_until_trigger(); return time_ready < tp; } @@ -186,7 +188,7 @@ class TimersManager heap_.push_back(x); size_t parent = (i - 1) / 2; - while (i > 0 && ((*x)->time_until_trigger() < (*heap_[parent])->time_until_trigger())) { + while (i > 0 && ((x)->time_until_trigger() < (heap_[parent])->time_until_trigger())) { heap_[i] = heap_[parent]; heap_[parent] = x; i = parent; @@ -212,7 +214,7 @@ class TimersManager while (left_child < heap_.size()) { size_t right_child = left_child + 1; if (right_child < heap_.size() && - (*heap_[left_child])->time_until_trigger() >= (*heap_[right_child])->time_until_trigger()) + (heap_[left_child])->time_until_trigger() >= (heap_[right_child])->time_until_trigger()) { left_child = right_child; } @@ -224,7 +226,7 @@ class TimersManager // Swim down while (i > start) { size_t parent = (i - 1) / 2; - if ((*updated_timer)->time_until_trigger() < (*heap_[parent])->time_until_trigger()) { + if ((updated_timer)->time_until_trigger() < (heap_[parent])->time_until_trigger()) { heap_[i] = heap_[parent]; i = parent; continue; @@ -248,7 +250,7 @@ class TimersManager // Context of the parent executor std::shared_ptr context_; // Container to keep ownership of the timers - std::list timers_storage_; + std::list timers_storage_; // Vector of pointers to stored timers used to implement the priority queue std::vector heap_; }; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 7ec076be52..7fe3f51c77 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -127,6 +127,10 @@ class ServiceBase const void * executor_context, ExecutorEventCallback executor_callback) const; + RCLCPP_PUBLIC + void + set_on_destruction_callback(std::function on_destruction_callback); + protected: RCLCPP_DISABLE_COPY(ServiceBase) @@ -138,6 +142,8 @@ class ServiceBase const rcl_node_t * get_rcl_node_handle() const; + std::function on_destruction_callback_; + std::shared_ptr node_handle_; std::shared_ptr service_handle_; diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 06d8bd7974..163188dca7 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -16,6 +16,7 @@ #define RCLCPP__SUBSCRIPTION_BASE_HPP_ #include +#include #include #include #include @@ -269,6 +270,10 @@ class SubscriptionBase : public std::enable_shared_from_this const void * executor_context, ExecutorEventCallback executor_callback) const; + RCLCPP_PUBLIC + void + set_on_destruction_callback(std::function on_destruction_callback); + protected: template void @@ -311,10 +316,12 @@ class SubscriptionBase : public std::enable_shared_from_this rosidl_message_type_support_t type_support_; bool is_serialized_; + std::function on_destruction_callback_; + std::atomic subscription_in_use_by_wait_set_{false}; std::atomic intra_process_subscription_waitable_in_use_by_wait_set_{false}; std::unordered_map> qos_events_in_use_by_wait_set_; + std::atomic> qos_events_in_use_by_wait_set_; }; } // namespace rclcpp diff --git a/rclcpp/include/rclcpp/timer.hpp b/rclcpp/include/rclcpp/timer.hpp index a2662f4abe..421eb08355 100644 --- a/rclcpp/include/rclcpp/timer.hpp +++ b/rclcpp/include/rclcpp/timer.hpp @@ -137,9 +137,14 @@ class TimerBase bool exchange_in_use_by_wait_set_state(bool in_use_state); + RCLCPP_PUBLIC + void + set_on_destruction_callback(std::function on_destruction_callback); + protected: Clock::SharedPtr clock_; std::shared_ptr timer_handle_; + std::function on_destruction_callback_; std::atomic in_use_by_wait_set_{false}; }; diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 73ff886f44..dc4307d4ff 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -16,6 +16,7 @@ #define RCLCPP__WAITABLE_HPP_ #include +#include #include "rclcpp/macros.hpp" #include "rclcpp/visibility_control.hpp" @@ -31,7 +32,7 @@ class Waitable RCLCPP_SMART_PTR_DEFINITIONS_NOT_COPYABLE(Waitable) RCLCPP_PUBLIC - virtual ~Waitable() = default; + virtual ~Waitable(); /// Get the number of ready subscriptions /** @@ -172,7 +173,12 @@ class Waitable void * executor_context, ExecutorEventCallback executor_callback) const; + RCLCPP_PUBLIC + void + set_on_destruction_callback(std::function on_destruction_callback); + private: + std::function on_destruction_callback_; std::atomic in_use_by_wait_set_{false}; }; // class Waitable diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 4f8604ed90..7d8666344f 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -66,6 +66,9 @@ ClientBase::ClientBase( ClientBase::~ClientBase() { + if (on_destruction_callback_) { + on_destruction_callback_(this); + } // Make sure the client handle is destructed as early as possible and before the node handle client_handle_.reset(); } @@ -214,3 +217,8 @@ ClientBase::set_events_executor_callback( throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); } } + +void ClientBase::set_on_destruction_callback(std::function on_destruction_callback) +{ + on_destruction_callback_ = on_destruction_callback; +} diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 87c0686a1f..02de1f6b03 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -71,25 +71,20 @@ EventsExecutor::spin() // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !event_queue_.empty();}; - // Local event queue - std::queue local_event_queue; - timers_manager_->start(); while (rclcpp::ok(context_) && spinning.load()) { - // Scope block for the mutex - { - std::unique_lock lock(event_queue_mutex_); - // We wait here until something has been pushed to the event queue - event_queue_cv_.wait(lock, has_event_predicate); - - // We got an event! Swap queues and execute events - std::swap(local_event_queue, event_queue_); - } - - this->consume_all_events(local_event_queue); + std::unique_lock push_lock(push_mutex_); + // We wait here until something has been pushed to the event queue + event_queue_cv_.wait(push_lock, has_event_predicate); + std::unique_lock execution_lock(execution_mutex_); + // We got an event! Swap queues and execute events while we hold both mutexes + std::swap(local_event_queue_, event_queue_); + // After swapping the queues, we don't need the push lock anymore + push_lock.~unique_lock(); + // Consume events while under the execution lock only + this->consume_all_events(local_event_queue_); } - timers_manager_->stop(); } @@ -114,23 +109,23 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !event_queue_.empty();}; - std::queue local_event_queue; - // Select the smallest between input max_duration and timer timeout auto next_timer_timeout = timers_manager_->get_head_timeout(); if (next_timer_timeout < max_duration) { max_duration = next_timer_timeout; } - { - // Wait until timeout or event - std::unique_lock lock(event_queue_mutex_); - event_queue_cv_.wait_for(lock, max_duration, has_event_predicate); - std::swap(local_event_queue, event_queue_); - } + + // Wait until timeout or event + std::unique_lock push_lock(push_mutex_); + event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); + std::unique_lock execution_lock(execution_mutex_); + std::swap(local_event_queue_, event_queue_); + push_lock.~unique_lock(); + this->consume_all_events(local_event_queue_); + execution_lock.~unique_lock(); timers_manager_->execute_ready_timers(); - this->consume_all_events(local_event_queue); } void @@ -148,8 +143,6 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !event_queue_.empty();}; - std::queue local_event_queue; - auto start = std::chrono::steady_clock::now(); auto max_duration_not_elapsed = [max_duration, start]() { auto elapsed_time = std::chrono::steady_clock::now() - start; @@ -165,27 +158,30 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) { // Wait until timeout or event - std::unique_lock lock(event_queue_mutex_); - event_queue_cv_.wait_for(lock, max_duration, has_event_predicate); + std::unique_lock push_lock(push_mutex_); + event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); } // Keep executing until work available or timeout expired while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { - { - std::unique_lock lock(event_queue_mutex_); - std::swap(local_event_queue, event_queue_); - } + + std::unique_lock push_lock(push_mutex_); + std::unique_lock execution_lock(execution_mutex_); + std::swap(local_event_queue_, event_queue_); + push_lock.~unique_lock(); bool ready_timer = timers_manager_->get_head_timeout() < 0ns; - bool has_events = !local_event_queue.empty(); + bool has_events = !local_event_queue_.empty(); if (!ready_timer && !has_events) { // Exit as there is no more work to do break; } - // Execute all ready work + + this->consume_all_events(local_event_queue_); + execution_lock.~unique_lock(); + timers_manager_->execute_ready_timers(); - this->consume_all_events(local_event_queue); } } @@ -211,14 +207,14 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) { // Wait until timeout or event arrives - std::unique_lock lock(event_queue_mutex_); + std::unique_lock lock(push_mutex_); event_queue_cv_.wait_for(lock, timeout, has_event_predicate); // Grab first event from queue if it exists has_event = !event_queue_.empty(); if (has_event) { event = event_queue_.front(); - event_queue_.pop(); + event_queue_.pop_front(); } } @@ -255,7 +251,10 @@ EventsExecutor::remove_node( // This field is unused because we don't have to wake up the executor when a node is removed. (void)notify; - // Remove node from entities collector + // Remove node from entities collector. + // This will result in un-setting all the event callbacks from its entities. + // After this function returns, this executor will not receive any more events associated + // to these entities. entities_collector_->remove_node(node_ptr); } @@ -280,11 +279,11 @@ EventsExecutor::cancel() } void -EventsExecutor::consume_all_events(std::queue & event_queue) +EventsExecutor::consume_all_events(EventQueue & event_queue) { while (!event_queue.empty()) { ExecutorEvent event = event_queue.front(); - event_queue.pop(); + event_queue.pop_front(); this->execute_event(event); } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 449cca8a83..84fb0a1195 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -29,13 +29,18 @@ EventsExecutorEntitiesCollector::EventsExecutorEntitiesCollector( EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() { + std::cout<<"entities collector destructor"<get_associated_with_executor_atomic(); has_executor.store(false); this->unset_entities_callbacks(node); + } else { + std::cout<<"skipping node in entities collector destructor"<add_timer(timer); + timer->set_on_destruction_callback(std::bind(&TimersManager::remove_timer_raw, timers_manager_, std::placeholders::_1)); } return false; }); @@ -129,6 +135,8 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( subscription->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); + std::cout<<"Setting destruction callback on "<< subscription.get()<< " with count "<< subscription.use_count()<set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); } return false; }); @@ -138,6 +146,7 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( service->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); + service->set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); } return false; }); @@ -147,6 +156,7 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( client->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); + client->set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); } return false; }); @@ -156,6 +166,7 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( waitable->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); + waitable->set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); } return false; }); @@ -192,6 +203,7 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { timers_manager_->remove_timer(timer); + timer->set_on_destruction_callback(nullptr); } return false; }); @@ -201,6 +213,8 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { subscription->set_events_executor_callback(nullptr, nullptr); + std::cout<<"Removing destruction callback on "<< subscription.get()<set_on_destruction_callback(nullptr); } return false; }); @@ -208,6 +222,7 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { service->set_events_executor_callback(nullptr, nullptr); + service->set_on_destruction_callback(nullptr); } return false; }); @@ -215,6 +230,7 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { client->set_events_executor_callback(nullptr, nullptr); + client->set_on_destruction_callback(nullptr); } return false; }); @@ -222,6 +238,7 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { waitable->set_events_executor_callback(nullptr, nullptr); + waitable->set_on_destruction_callback(nullptr); } return false; }); diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index b60a42f3cc..5b1b8aeb74 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -45,13 +45,15 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) std::unique_lock timers_lock(timers_mutex_); // Make sure that the provided timer is not already in the timers storage - if (std::find(timers_storage_.begin(), timers_storage_.end(), timer) != timers_storage_.end()) { - return; + for (auto t : timers_storage_) { + if (t.lock() == timer) { + return; + } } // Store ownership of timer and add it to heap timers_storage_.emplace_back(timer); - this->add_timer_to_heap(&(timers_storage_.back())); + this->add_timer_to_heap(timer.get()); timers_updated_ = true; } @@ -137,9 +139,9 @@ bool TimersManager::execute_head_timer() } TimerPtr head = heap_.front(); - if ((*head)->is_ready()) { + if ((head)->is_ready()) { // Head timer is ready, execute and re-heapify - (*head)->execute_callback(); + (head)->execute_callback(); this->restore_heap_root(); return true; } else { @@ -161,9 +163,9 @@ void TimersManager::execute_ready_timers_unsafe() auto start = std::chrono::steady_clock::now(); TimerPtr head = heap_.front(); - while ((*head)->is_ready() && this->timer_was_ready_at_tp(head, start)) { + while ((head)->is_ready() && this->timer_was_ready_at_tp(head, start)) { // Execute head timer - (*head)->execute_callback(); + (head)->execute_callback(); // Executing a timer will result in updating its time_until_trigger, so re-heapify this->restore_heap_root(); // Get new head timer @@ -213,12 +215,22 @@ void TimersManager::clear_all() } void TimersManager::remove_timer(rclcpp::TimerBase::SharedPtr timer) +{ + this->remove_timer_raw(timer.get()); +} + +void TimersManager::remove_timer_raw(rclcpp::TimerBase* timer) { { std::unique_lock timers_lock(timers_mutex_); // Make sure that we are currently storing the provided timer before proceeding - auto it = std::find(timers_storage_.begin(), timers_storage_.end(), timer); + std::list::iterator it; + for (it = timers_storage_.begin(); it != timers_storage_.end(); ++it){ + if ((*it).lock().get() == timer) { + break; + } + } if (it == timers_storage_.end()) { return; } @@ -227,7 +239,7 @@ void TimersManager::remove_timer(rclcpp::TimerBase::SharedPtr timer) timers_storage_.erase(it); heap_.clear(); for (auto & t : timers_storage_) { - this->add_timer_to_heap(&t); + this->add_timer_to_heap(t.lock().get()); } timers_updated_ = true; diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 914582e4ed..b0c81d73e1 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -32,7 +32,11 @@ ServiceBase::ServiceBase(std::shared_ptr node_handle) {} ServiceBase::~ServiceBase() -{} +{ + if (on_destruction_callback_) { + on_destruction_callback_(this); + } +} bool ServiceBase::take_type_erased_request(void * request_out, rmw_request_id_t & request_id_out) @@ -100,3 +104,8 @@ ServiceBase::set_events_executor_callback( throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); } } + +void ServiceBase::set_on_destruction_callback(std::function on_destruction_callback) +{ + on_destruction_callback_ = on_destruction_callback; +} diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 345bbf7181..2a78fd6448 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -79,10 +79,18 @@ SubscriptionBase::SubscriptionBase( rclcpp::exceptions::throw_from_rcl_error(ret, "could not create subscription"); } + + std::cout<<"Construction subscription "<< this< on_destruction_callback) +{ + on_destruction_callback_ = on_destruction_callback; +} diff --git a/rclcpp/src/rclcpp/timer.cpp b/rclcpp/src/rclcpp/timer.cpp index 70414f38c6..64545b91b3 100644 --- a/rclcpp/src/rclcpp/timer.cpp +++ b/rclcpp/src/rclcpp/timer.cpp @@ -71,7 +71,11 @@ TimerBase::TimerBase( } TimerBase::~TimerBase() -{} +{ + if (on_destruction_callback_) { + on_destruction_callback_(this); + } +} void TimerBase::cancel() @@ -136,3 +140,8 @@ TimerBase::exchange_in_use_by_wait_set_state(bool in_use_state) { return in_use_by_wait_set_.exchange(in_use_state); } + +void TimerBase::set_on_destruction_callback(std::function on_destruction_callback) +{ + on_destruction_callback_ = on_destruction_callback; +} diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 372826a912..f2726ff842 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -18,6 +18,13 @@ using rclcpp::Waitable; +Waitable::~Waitable() +{ + if (on_destruction_callback_) { + on_destruction_callback_(this); + } +} + size_t Waitable::get_number_of_ready_subscriptions() { @@ -71,3 +78,9 @@ Waitable::set_events_executor_callback( throw std::runtime_error( "Custom waitables should override set_events_executor_callback() to use events executor"); } + +void +Waitable::set_on_destruction_callback(std::function on_destruction_callback) +{ + on_destruction_callback_ = on_destruction_callback; +} diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index c6b66deae1..406fd68d07 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -20,6 +20,7 @@ #include "rclcpp/executors/events_executor.hpp" #include "test_msgs/srv/empty.hpp" +#include "test_msgs/msg/empty.hpp" using namespace std::chrono_literals; @@ -107,11 +108,11 @@ TEST_F(TestEventsExecutor, spin_once_max_duration) EventsExecutor executor; executor.add_node(node); - auto start = node->now(); + auto start = std::chrono::steady_clock::now(); executor.spin_once(10ms); EXPECT_EQ(0u, t_runs); - EXPECT_TRUE(node->now() - start < 200ms); + EXPECT_TRUE(std::chrono::steady_clock::now() - start < 200ms); } { @@ -127,11 +128,11 @@ TEST_F(TestEventsExecutor, spin_once_max_duration) EventsExecutor executor; executor.add_node(node); - auto start = node->now(); + auto start = std::chrono::steady_clock::now(); executor.spin_once(10s); EXPECT_EQ(1u, t_runs); - EXPECT_TRUE(node->now() - start < 200ms); + EXPECT_TRUE(std::chrono::steady_clock::now() - start < 200ms); } } @@ -150,11 +151,11 @@ TEST_F(TestEventsExecutor, spin_some_max_duration) EventsExecutor executor; executor.add_node(node); - auto start = node->now(); + auto start = std::chrono::steady_clock::now(); executor.spin_some(10ms); EXPECT_EQ(0u, t_runs); - EXPECT_TRUE(node->now() - start < 200ms); + EXPECT_TRUE(std::chrono::steady_clock::now() - start < 200ms); } { @@ -170,11 +171,11 @@ TEST_F(TestEventsExecutor, spin_some_max_duration) EventsExecutor executor; executor.add_node(node); - auto start = node->now(); + auto start = std::chrono::steady_clock::now(); executor.spin_some(10s); EXPECT_EQ(1u, t_runs); - EXPECT_TRUE(node->now() - start < 200ms); + EXPECT_TRUE(std::chrono::steady_clock::now() - start < 200ms); } } @@ -193,11 +194,11 @@ TEST_F(TestEventsExecutor, spin_all_max_duration) EventsExecutor executor; executor.add_node(node); - auto start = node->now(); + auto start = std::chrono::steady_clock::now(); executor.spin_all(10ms); EXPECT_EQ(0u, t_runs); - EXPECT_TRUE(node->now() - start < 200ms); + EXPECT_TRUE(std::chrono::steady_clock::now() - start < 200ms); } { @@ -213,11 +214,11 @@ TEST_F(TestEventsExecutor, spin_all_max_duration) EventsExecutor executor; executor.add_node(node); - auto start = node->now(); + auto start = std::chrono::steady_clock::now(); executor.spin_all(10s); EXPECT_EQ(1u, t_runs); - EXPECT_TRUE(node->now() - start < 200ms); + EXPECT_TRUE(std::chrono::steady_clock::now() - start < 200ms); } EventsExecutor executor; @@ -225,60 +226,68 @@ TEST_F(TestEventsExecutor, spin_all_max_duration) EXPECT_THROW(executor.spin_all(-5ms), std::invalid_argument); } -TEST_F(TestEventsExecutor, cancel_while_timers_running) +TEST_F(TestEventsExecutor, cancel_while_timers_waiting) { auto node = std::make_shared("node"); size_t t1_runs = 0; auto t1 = node->create_wall_timer( - 1ms, + 100s, [&]() { t1_runs++; - std::this_thread::sleep_for(25ms); - }); - - size_t t2_runs = 0; - auto t2 = node->create_wall_timer( - 1ms, - [&]() { - t2_runs++; - std::this_thread::sleep_for(25ms); }); EventsExecutor executor; executor.add_node(node); + auto start = std::chrono::steady_clock::now(); std::thread spinner([&executor, this]() {executor.spin();}); std::this_thread::sleep_for(10ms); - // Call cancel while t1 callback is still being executed executor.cancel(); spinner.join(); - // Depending on the latency on the system, t2 may start to execute before cancel is signaled - EXPECT_GE(1u, t1_runs); - EXPECT_GE(1u, t2_runs); + EXPECT_EQ(0u, t1_runs); + EXPECT_TRUE(std::chrono::steady_clock::now() - start < 1s); } -TEST_F(TestEventsExecutor, cancel_while_timers_waiting) +TEST_F(TestEventsExecutor, destroy_entities) { - auto node = std::make_shared("node"); + // Create a publisher node and start publishing messages + auto node_pub = std::make_shared("node_pub"); + auto publisher = node_pub->create_publisher("topic", rclcpp::QoS(10)); + auto timer = node_pub->create_wall_timer( + 2ms, [&]() { publisher->publish(std::make_unique()); }); + EventsExecutor executor_pub; + executor_pub.add_node(node_pub); + std::thread spinner([&executor_pub, this]() {executor_pub.spin();}); + + // Create a node with two different subscriptions to the topic + auto node_sub = std::make_shared("node_sub"); + size_t callback_count_1 = 0; + auto subscription_1 = + node_sub->create_subscription( + "topic", rclcpp::QoS(10), [&](test_msgs::msg::Empty::SharedPtr) {callback_count_1++;}); + size_t callback_count_2 = 0; + auto subscription_2 = + node_sub->create_subscription( + "topic", rclcpp::QoS(10), [&](test_msgs::msg::Empty::SharedPtr) {callback_count_2++;}); + EventsExecutor executor_sub; + executor_sub.add_node(node_sub); + + // Wait some time while messages are published + std::this_thread::sleep_for(10ms); - size_t t1_runs = 0; - auto t1 = node->create_wall_timer( - 100s, - [&]() { - t1_runs++; - }); + // Destroy one of the two subscriptions + subscription_1.reset(); - EventsExecutor executor; - executor.add_node(node); + // Let subscriptions executor spin + executor_sub.spin_some(10ms); - std::thread spinner([&executor, this]() {executor.spin();}); + // The callback count of the destroyed subscription remained at 0 + EXPECT_EQ(0u, callback_count_1); + EXPECT_LT(0u, callback_count_2); - std::this_thread::sleep_for(10ms); - executor.cancel(); + executor_pub.cancel(); spinner.join(); - - EXPECT_EQ(0u, t1_runs); } diff --git a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp index 446bf74037..66e9a5e468 100644 --- a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp +++ b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp @@ -78,7 +78,7 @@ TEST_F(TestTimersManager, add_run_remove_timer) // Add the timer to the timers manager auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); - timers_manager->add_timer(std::move(t)); + timers_manager->add_timer(t); // Sleep for more 3 times the timer period std::this_thread::sleep_for(3ms); @@ -87,16 +87,12 @@ TEST_F(TestTimersManager, add_run_remove_timer) timers_manager->execute_ready_timers(); EXPECT_EQ(1u, t_runs); - // The timer is still valid after execution - auto t_tmp = t_weak.lock(); - EXPECT_TRUE(t_tmp != nullptr); - // Remove the timer from the manager - timers_manager->remove_timer(std::move(t_tmp)); + timers_manager->remove_timer(t); + t.reset(); // The timer is now not valid anymore - t_tmp = t_weak.lock(); - EXPECT_FALSE(t_tmp != nullptr); + EXPECT_FALSE(t_weak.lock() != nullptr); } TEST_F(TestTimersManager, clear_all) @@ -109,14 +105,17 @@ TEST_F(TestTimersManager, clear_all) auto t2 = TimerT::make_shared(1ms, CallbackT(), rclcpp::contexts::get_global_default_context()); std::weak_ptr t2_weak = t2; - timers_manager->add_timer(std::move(t1)); - timers_manager->add_timer(std::move(t2)); + timers_manager->add_timer(t1); + timers_manager->add_timer(t2); EXPECT_TRUE(t1_weak.lock() != nullptr); EXPECT_TRUE(t2_weak.lock() != nullptr); timers_manager->clear_all(); + t1.reset(); + t2.reset(); + EXPECT_FALSE(t1_weak.lock() != nullptr); EXPECT_FALSE(t2_weak.lock() != nullptr); } @@ -325,7 +324,7 @@ TEST_F(TestTimersManager, destructor) auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); - timers_manager->add_timer(std::move(t)); + timers_manager->add_timer(t); timers_manager->start(); std::this_thread::sleep_for(3ms); @@ -337,6 +336,7 @@ TEST_F(TestTimersManager, destructor) size_t runs = t_runs; std::this_thread::sleep_for(3ms); EXPECT_EQ(runs, t_runs); + t.reset(); EXPECT_FALSE(t_weak.lock() != nullptr); } From 37f32ef65af288220c014a1250a7d19a12fc2622 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Fri, 16 Oct 2020 09:25:26 +0100 Subject: [PATCH 056/168] remove prints Signed-off-by: Soragna, Alberto --- .../executors/events_executor_entities_collector.cpp | 7 ------- rclcpp/src/rclcpp/subscription_base.cpp | 4 ---- 2 files changed, 11 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 84fb0a1195..661674ca24 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -29,18 +29,13 @@ EventsExecutorEntitiesCollector::EventsExecutorEntitiesCollector( EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() { - std::cout<<"entities collector destructor"<get_associated_with_executor_atomic(); has_executor.store(false); this->unset_entities_callbacks(node); - } else { - std::cout<<"skipping node in entities collector destructor"<set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - std::cout<<"Setting destruction callback on "<< subscription.get()<< " with count "<< subscription.use_count()<set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); } return false; @@ -213,7 +207,6 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { subscription->set_events_executor_callback(nullptr, nullptr); - std::cout<<"Removing destruction callback on "<< subscription.get()<set_on_destruction_callback(nullptr); } return false; diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 2a78fd6448..d7ba313531 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -79,15 +79,11 @@ SubscriptionBase::SubscriptionBase( rclcpp::exceptions::throw_from_rcl_error(ret, "could not create subscription"); } - - std::cout<<"Construction subscription "<< this< Date: Fri, 16 Oct 2020 09:38:20 +0100 Subject: [PATCH 057/168] undo whitespace change Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 163188dca7..a47d64a74c 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -321,7 +321,7 @@ class SubscriptionBase : public std::enable_shared_from_this std::atomic subscription_in_use_by_wait_set_{false}; std::atomic intra_process_subscription_waitable_in_use_by_wait_set_{false}; std::unordered_map> qos_events_in_use_by_wait_set_; + std::atomic> qos_events_in_use_by_wait_set_; }; } // namespace rclcpp From 83c565979765ecd494ea4b2f51b2154fb150c31f Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Fri, 16 Oct 2020 13:57:52 +0100 Subject: [PATCH 058/168] use std:: functions for heap and minor fixes Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/timers_manager.hpp | 149 ++++++++++-------- .../src/rclcpp/executors/events_executor.cpp | 12 +- .../events_executor_entities_collector.cpp | 29 +++- .../src/rclcpp/executors/timers_manager.cpp | 54 ++----- .../rclcpp/executors/test_timers_manager.cpp | 10 +- 5 files changed, 130 insertions(+), 124 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 07331626c4..ec5b77ee3b 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -15,10 +15,10 @@ #ifndef RCLCPP__EXECUTORS__TIMERS_MANAGER_HPP_ #define RCLCPP__EXECUTORS__TIMERS_MANAGER_HPP_ +#include #include #include #include -#include #include #include #include @@ -113,14 +113,18 @@ class TimersManager /** * @brief Remove all the timers stored in the object. */ - void clear_all(); + void clear(); /** - * @brief Remove a single timer stored in the object. + * @brief Remove a single timer stored in the object, passed as a shared_ptr. * @param timer the timer to remove. */ void remove_timer(rclcpp::TimerBase::SharedPtr timer); + /** + * @brief Remove a single timer stored in the object, passed as a raw ptr. + * @param timer the timer to remove. + */ void remove_timer_raw(rclcpp::TimerBase* timer); // This is what the TimersManager uses to denote a duration forever. @@ -133,6 +137,78 @@ class TimersManager using TimerPtr = rclcpp::TimerBase *; + /** + * @brief This struct provides convenient access to a MinHeap of timers. + * The root of the heap is the timer that expires first. + * This struct is not thread safe and requires external mutexes to protect its usage. + */ + struct TimersHeap + { + void add_timer(TimerPtr timer) + { + // Nothing to do if the timer is already stored here + auto it = std::find(timers_.begin(), timers_.end(), timer); + if (it != timers_.end()) { + return; + } + timers_.push_back(timer); + std::make_heap(timers_.begin(), timers_.end(), timer_greater); + } + + /** + * @brief Restore a valid heap after the root value is replaced. + */ + void heapify_root() + { + // Push the modified element (i.e. the current root) at the bottom of the heap + timers_.push_back(timers_[0]); + // Exchange first and last elements and reheapify + std::pop_heap(timers_.begin(), timers_.end(), timer_greater); + // Remove last element + timers_.pop_back(); + } + + void remove_timer(TimerPtr timer) + { + // Nothing to do if the timer is not stored here + auto it = std::find(timers_.begin(), timers_.end(), timer); + if (it == timers_.end()) { + return; + } + timers_.erase(it); + std::make_heap(timers_.begin(), timers_.end(), timer_greater); + } + + TimerPtr& front() + { + return timers_.front(); + } + + const TimerPtr& front() const + { + return timers_.front(); + } + + bool empty() + { + return timers_.empty(); + } + + void clear() + { + timers_.clear(); + } + + private: + static bool timer_greater(TimerPtr a, TimerPtr b) + { + return a->time_until_trigger() > b->time_until_trigger(); + } + + // Vector of pointers to timers used to implement the priority queue + std::vector timers_; + }; + /** * @brief Implements a loop that keeps executing ready timers. * This function is executed in the timers thread. @@ -152,7 +228,7 @@ class TimersManager if (heap_.empty()) { return MAX_TIME; } - return (heap_[0])->time_until_trigger(); + return (heap_.front())->time_until_trigger(); } /** @@ -178,65 +254,6 @@ class TimersManager return time_ready < tp; } - /** - * @brief Add a new timer to the heap and sort it. - * @param x pointer to a timer owned by this object. - */ - void add_timer_to_heap(TimerPtr x) - { - size_t i = heap_.size(); // Position where we are going to add timer - heap_.push_back(x); - - size_t parent = (i - 1) / 2; - while (i > 0 && ((x)->time_until_trigger() < (heap_[parent])->time_until_trigger())) { - heap_[i] = heap_[parent]; - heap_[parent] = x; - i = parent; - parent = (i - 1) / 2; - } - } - - /** - * @brief Restore a valid heap after the root value is replaced. - * This function assumes that the value is increased, because if it was decreased - * then there is nothing to do. - */ - void restore_heap_root() - { - constexpr size_t start = 0; - TimerPtr updated_timer = heap_[0]; - - // Initialize to root and first left child - size_t i = 0; - size_t left_child = 1; - - // Swim up - while (left_child < heap_.size()) { - size_t right_child = left_child + 1; - if (right_child < heap_.size() && - (heap_[left_child])->time_until_trigger() >= (heap_[right_child])->time_until_trigger()) - { - left_child = right_child; - } - heap_[i] = heap_[left_child]; - i = left_child; - left_child = 2 * i + 1; - } - - // Swim down - while (i > start) { - size_t parent = (i - 1) / 2; - if ((updated_timer)->time_until_trigger() < (heap_[parent])->time_until_trigger()) { - heap_[i] = heap_[parent]; - i = parent; - continue; - } - break; - } - - heap_[i] = updated_timer; - } - // Thread used to run the timers monitoring and execution task std::thread timers_thread_; // Protects access to timers @@ -249,10 +266,8 @@ class TimersManager bool running_ {false}; // Context of the parent executor std::shared_ptr context_; - // Container to keep ownership of the timers - std::list timers_storage_; - // Vector of pointers to stored timers used to implement the priority queue - std::vector heap_; + // MinHeap of timers + TimersHeap heap_; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 02de1f6b03..168504d2b7 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -78,10 +78,10 @@ EventsExecutor::spin() // We wait here until something has been pushed to the event queue event_queue_cv_.wait(push_lock, has_event_predicate); std::unique_lock execution_lock(execution_mutex_); - // We got an event! Swap queues and execute events while we hold both mutexes + // We got an event! Swap queues while we hold both mutexes std::swap(local_event_queue_, event_queue_); // After swapping the queues, we don't need the push lock anymore - push_lock.~unique_lock(); + push_lock.unlock(); // Consume events while under the execution lock only this->consume_all_events(local_event_queue_); } @@ -121,9 +121,9 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); std::unique_lock execution_lock(execution_mutex_); std::swap(local_event_queue_, event_queue_); - push_lock.~unique_lock(); + push_lock.unlock(); this->consume_all_events(local_event_queue_); - execution_lock.~unique_lock(); + execution_lock.unlock(); timers_manager_->execute_ready_timers(); } @@ -168,7 +168,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) std::unique_lock push_lock(push_mutex_); std::unique_lock execution_lock(execution_mutex_); std::swap(local_event_queue_, event_queue_); - push_lock.~unique_lock(); + push_lock.unlock(); bool ready_timer = timers_manager_->get_head_timeout() < 0ns; bool has_events = !local_event_queue_.empty(); @@ -179,7 +179,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Execute all ready work this->consume_all_events(local_event_queue_); - execution_lock.~unique_lock(); + execution_lock.unlock(); timers_manager_->execute_ready_timers(); } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 661674ca24..acf3b4b7f3 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -49,7 +49,7 @@ EventsExecutorEntitiesCollector::execute() // This function is called when the associated executor is notified that something changed. // We do not know if an entity has been added or remode so we have to rebuild everything. - timers_manager_->clear_all(); + timers_manager_->clear(); for (auto & weak_node : weak_nodes_) { auto node = weak_node.lock(); @@ -118,7 +118,8 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { timers_manager_->add_timer(timer); - timer->set_on_destruction_callback(std::bind(&TimersManager::remove_timer_raw, timers_manager_, std::placeholders::_1)); + timer->set_on_destruction_callback( + std::bind(&TimersManager::remove_timer_raw, timers_manager_, std::placeholders::_1)); } return false; }); @@ -130,7 +131,11 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( subscription->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - subscription->set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); + subscription->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); } return false; }); @@ -140,7 +145,11 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( service->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - service->set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); + service->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); } return false; }); @@ -150,7 +159,11 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( client->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - client->set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); + client->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); } return false; }); @@ -160,7 +173,11 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( waitable->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - waitable->set_on_destruction_callback(std::bind(&EventsExecutor::remove_entity, associated_executor_, std::placeholders::_1)); + waitable->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); } return false; }); diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 5b1b8aeb74..f9e93f9a00 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -32,7 +32,7 @@ TimersManager::~TimersManager() this->stop(); // Remove all timers - this->clear_all(); + this->clear(); } void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) @@ -43,18 +43,7 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) { std::unique_lock timers_lock(timers_mutex_); - - // Make sure that the provided timer is not already in the timers storage - for (auto t : timers_storage_) { - if (t.lock() == timer) { - return; - } - } - - // Store ownership of timer and add it to heap - timers_storage_.emplace_back(timer); - this->add_timer_to_heap(timer.get()); - + heap_.add_timer(timer.get()); timers_updated_ = true; } @@ -139,10 +128,10 @@ bool TimersManager::execute_head_timer() } TimerPtr head = heap_.front(); - if ((head)->is_ready()) { + if (head->is_ready()) { // Head timer is ready, execute and re-heapify - (head)->execute_callback(); - this->restore_heap_root(); + head->execute_callback(); + heap_.heapify_root(); return true; } else { // Head timer was not ready yet @@ -163,11 +152,11 @@ void TimersManager::execute_ready_timers_unsafe() auto start = std::chrono::steady_clock::now(); TimerPtr head = heap_.front(); - while ((head)->is_ready() && this->timer_was_ready_at_tp(head, start)) { + while (head->is_ready() && this->timer_was_ready_at_tp(head, start)) { // Execute head timer - (head)->execute_callback(); + head->execute_callback(); // Executing a timer will result in updating its time_until_trigger, so re-heapify - this->restore_heap_root(); + heap_.heapify_root(); // Get new head timer head = heap_.front(); } @@ -199,18 +188,17 @@ void TimersManager::run_timers() running_ = false; } -void TimersManager::clear_all() +void TimersManager::clear() { { // Lock mutex and then clear all data structures std::unique_lock timers_lock(timers_mutex_); heap_.clear(); - timers_storage_.clear(); timers_updated_ = true; } - // Notify timers thead such that it can re-compute its timeout + // Notify timers thread such that it can re-compute its timeout timers_cv_.notify_one(); } @@ -223,28 +211,10 @@ void TimersManager::remove_timer_raw(rclcpp::TimerBase* timer) { { std::unique_lock timers_lock(timers_mutex_); - - // Make sure that we are currently storing the provided timer before proceeding - std::list::iterator it; - for (it = timers_storage_.begin(); it != timers_storage_.end(); ++it){ - if ((*it).lock().get() == timer) { - break; - } - } - if (it == timers_storage_.end()) { - return; - } - - // Remove timer from the storage and rebuild heap - timers_storage_.erase(it); - heap_.clear(); - for (auto & t : timers_storage_) { - this->add_timer_to_heap(t.lock().get()); - } - + heap_.remove_timer(timer); timers_updated_ = true; } - // Notify timers thead such that it can re-compute its timeout + // Notify timers thread such that it can re-compute its timeout timers_cv_.notify_one(); } diff --git a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp index 66e9a5e468..c8407186a3 100644 --- a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp +++ b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp @@ -50,7 +50,7 @@ TEST_F(TestTimersManager, empty_manager) EXPECT_EQ(TimersManager::MAX_TIME, timers_manager->get_head_timeout()); EXPECT_FALSE(timers_manager->execute_head_timer()); EXPECT_NO_THROW(timers_manager->execute_ready_timers()); - EXPECT_NO_THROW(timers_manager->clear_all()); + EXPECT_NO_THROW(timers_manager->clear()); EXPECT_NO_THROW(timers_manager->start()); EXPECT_NO_THROW(timers_manager->stop()); } @@ -95,7 +95,7 @@ TEST_F(TestTimersManager, add_run_remove_timer) EXPECT_FALSE(t_weak.lock() != nullptr); } -TEST_F(TestTimersManager, clear_all) +TEST_F(TestTimersManager, clear) { auto timers_manager = std::make_shared( rclcpp::contexts::get_global_default_context()); @@ -111,7 +111,7 @@ TEST_F(TestTimersManager, clear_all) EXPECT_TRUE(t1_weak.lock() != nullptr); EXPECT_TRUE(t2_weak.lock() != nullptr); - timers_manager->clear_all(); + timers_manager->clear(); t1.reset(); t2.reset(); @@ -423,9 +423,13 @@ TEST_F(TestTimersManager, infinite_loop) EXPECT_TRUE(ret); EXPECT_EQ(3u, t1_runs + t2_runs); + std::cout<<"START THREAD-------------------|||||||||||||||||||||||||||"<start(); std::this_thread::sleep_for(10ms); + std::cout<<"STOP THREAD-------------------|||||||||||||||||||||||||||"<stop(); EXPECT_LT(3u, t1_runs + t2_runs); From f96f12a228e5a83aeaa1a277eae36316b7d89a7f Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Fri, 16 Oct 2020 14:26:52 +0100 Subject: [PATCH 059/168] remove logs Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 2 +- rclcpp/test/rclcpp/executors/test_timers_manager.cpp | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index ec5b77ee3b..e0ce781573 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -250,7 +250,7 @@ class TimersManager std::chrono::time_point tp) { // A ready timer will return a negative duration when calling time_until_trigger - auto time_ready = std::chrono::steady_clock::now() + (timer)->time_until_trigger(); + auto time_ready = std::chrono::steady_clock::now() + timer->time_until_trigger(); return time_ready < tp; } diff --git a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp index c8407186a3..0a3b8e44d7 100644 --- a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp +++ b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp @@ -423,13 +423,9 @@ TEST_F(TestTimersManager, infinite_loop) EXPECT_TRUE(ret); EXPECT_EQ(3u, t1_runs + t2_runs); - std::cout<<"START THREAD-------------------|||||||||||||||||||||||||||"<start(); std::this_thread::sleep_for(10ms); - std::cout<<"STOP THREAD-------------------|||||||||||||||||||||||||||"<stop(); EXPECT_LT(3u, t1_runs + t2_runs); From 773f8f5476860ab0ec6e8747f6040d5f57ca3710 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 17:12:23 +0100 Subject: [PATCH 060/168] Add callback groups support --- .../rclcpp/executors/events_executor.hpp | 36 ++ .../events_executor_entities_collector.hpp | 105 +++- .../src/rclcpp/executors/events_executor.cpp | 40 ++ .../events_executor_entities_collector.cpp | 541 +++++++++++++----- 4 files changed, 572 insertions(+), 150 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 7ca0bdbc86..75c82d963a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -172,6 +172,42 @@ class EventsExecutor : public rclcpp::Executor } } + void + add_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + bool notify) override; + + /// Remove callback group from the executor + /** + * \sa rclcpp::Executor::remove_callback_group + */ + RCLCPP_PUBLIC + void + remove_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + bool notify) override; + + RCLCPP_PUBLIC + std::vector + get_all_callback_groups() override; + + /// Get callback groups that belong to executor. + /** + * \sa rclcpp::Executor::get_manually_added_callback_groups() + */ + RCLCPP_PUBLIC + std::vector + get_manually_added_callback_groups() override; + + /// Get callback groups that belong to executor. + /** + * \sa rclcpp::Executor::get_automatically_added_callback_groups_from_nodes() + */ + RCLCPP_PUBLIC + std::vector + get_automatically_added_callback_groups_from_nodes() override; + protected: RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index d34e709b35..666b6f584d 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -26,6 +26,9 @@ namespace rclcpp { namespace executors { +typedef std::map> WeakCallbackGroupsToNodesMap; // forward declaration of EventsExecutor to avoid circular dependency class EventsExecutor; @@ -94,12 +97,110 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable return false; } + /// Add a callback group to the entities collector + /** + * \see rclcpp::Executor::add_callback_group + */ + RCLCPP_PUBLIC + void + add_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr); + + /// Add a callback group to the entities collector + /** + * \see rclcpp::Executor::add_callback_group + * \return boolean whether the node from the callback group is new + */ + RCLCPP_PUBLIC + void + add_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + WeakCallbackGroupsToNodesMap & weak_groups_to_nodes); + + /// Remove a callback group from the entities collector + /** + * \see rclcpp::Executor::remove_callback_group + */ + RCLCPP_PUBLIC + void + remove_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr); + + /// Remove a callback group from the entities collector + /** + * \see rclcpp::Executor::remove_callback_group_from_map + */ + RCLCPP_PUBLIC + void + remove_callback_group_from_map( + rclcpp::CallbackGroup::SharedPtr group_ptr, + WeakCallbackGroupsToNodesMap & weak_groups_to_nodes); + + RCLCPP_PUBLIC + std::vector + get_all_callback_groups(); + + /// Get manually added callback groups that belong to the entities collector + /** + * \see rclcpp::Executor::get_manually_added_callback_groups() + */ + RCLCPP_PUBLIC + std::vector + get_manually_added_callback_groups(); + + /// Get autmatically added callback groups that belong to the entities collector + /** + * \see rclcpp::Executor::get_automatically_added_callback_groups_from_nodes() + */ + RCLCPP_PUBLIC + std::vector + get_automatically_added_callback_groups_from_nodes(); + private: void - set_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); + set_node_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); + + void + unset_node_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); + + void + set_callback_group_entities_callbacks(rclcpp::CallbackGroup::SharedPtr group); void - unset_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); + unset_callback_group_entities_callbacks(rclcpp::CallbackGroup::SharedPtr group); + + /// Return true if the node belongs to the collector + /** + * \param[in] group_ptr a node base interface shared pointer + * \return boolean whether a node belongs the collector + */ + bool + has_node( + const rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + const WeakCallbackGroupsToNodesMap & weak_groups_to_nodes) const; + + /// Add all callback groups that can be automatically added by any executor + /// and is not already associated with an executor from nodes + /// that are associated with executor + /** + * \see rclcpp::Executor::add_callback_groups_from_nodes_associated_to_executor() + */ + void + add_callback_groups_from_nodes_associated_to_executor(); + + void + set_entities_event_callbacks_from_map( + const WeakCallbackGroupsToNodesMap & weak_groups_to_nodes); + + /// Memory strategy: an interface for handling user-defined memory allocation strategies. + rclcpp::memory_strategy::MemoryStrategy::SharedPtr memory_strategy_; + + // maps callback groups to nodes. + WeakCallbackGroupsToNodesMap weak_groups_associated_with_executor_to_nodes_; + // maps callback groups to nodes. + WeakCallbackGroupsToNodesMap weak_groups_to_nodes_associated_with_executor_; /// List of weak nodes registered in the events executor std::list weak_nodes_; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 168504d2b7..3615c2475a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -326,3 +326,43 @@ EventsExecutor::execute_event(const ExecutorEvent & event) } } } + +void +EventsExecutor::add_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + bool notify) +{ + // This field is unused because we don't have to wake up + // the executor when a callback group is added. + (void)notify; + entities_collector_->add_callback_group(group_ptr, node_ptr); +} + +void +EventsExecutor::remove_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, bool notify) +{ + // This field is unused because we don't have to wake up + // the executor when a callback group is removed. + (void)notify; + entities_collector_->remove_callback_group(group_ptr); +} + +std::vector +EventsExecutor::get_all_callback_groups() +{ + return entities_collector_->get_all_callback_groups(); +} + +std::vector +EventsExecutor::get_manually_added_callback_groups() +{ + return entities_collector_->get_manually_added_callback_groups(); +} + +std::vector +EventsExecutor::get_automatically_added_callback_groups_from_nodes() +{ + return entities_collector_->get_automatically_added_callback_groups_from_nodes(); +} \ No newline at end of file diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index acf3b4b7f3..aeed985816 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -29,37 +29,37 @@ EventsExecutorEntitiesCollector::EventsExecutorEntitiesCollector( EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() { + // Disassociate all callback groups + for (const auto & pair : weak_groups_associated_with_executor_to_nodes_) { + auto group = pair.first.lock(); + if (group) { + std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); + has_executor.store(false); + } + } + for (const auto & pair : weak_groups_to_nodes_associated_with_executor_) { + auto group = pair.first.lock(); + if (group) { + std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); + has_executor.store(false); + } + } // Disassociate all nodes for (const auto & weak_node : weak_nodes_) { auto node = weak_node.lock(); if (node) { std::atomic_bool & has_executor = node->get_associated_with_executor_atomic(); has_executor.store(false); - this->unset_entities_callbacks(node); + this->unset_node_entities_callbacks(node); } } - // Make sure that the list is empty + // Clear all lists + weak_groups_associated_with_executor_to_nodes_.clear(); + weak_groups_to_nodes_associated_with_executor_.clear(); weak_nodes_.clear(); } -void -EventsExecutorEntitiesCollector::execute() -{ - // This function is called when the associated executor is notified that something changed. - // We do not know if an entity has been added or remode so we have to rebuild everything. - - timers_manager_->clear(); - - for (auto & weak_node : weak_nodes_) { - auto node = weak_node.lock(); - if (!node) { - continue; - } - set_entities_callbacks(node); - } -} - void EventsExecutorEntitiesCollector::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) @@ -71,38 +71,132 @@ EventsExecutorEntitiesCollector::add_node( throw std::runtime_error("Node has already been added to an executor."); } + // Get node callback groups, add them to weak_groups_to_nodes_associated_with_executor_ + for (const auto & weak_group : node_ptr->get_callback_groups()) + { + auto group_ptr = weak_group.lock(); + if (group_ptr != nullptr && !group_ptr->get_associated_with_executor_atomic().load() && + group_ptr->automatically_add_to_executor_with_node()) + { + add_callback_group(group_ptr, node_ptr, weak_groups_to_nodes_associated_with_executor_); + } + } + + // Add node to weak_nodes_ weak_nodes_.push_back(node_ptr); - set_entities_callbacks(node_ptr); + // Set node's entities callbacks + set_node_entities_callbacks(node_ptr); } + void -EventsExecutorEntitiesCollector::remove_node( +EventsExecutorEntitiesCollector::add_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { - // Check if this node is currently stored here - auto node_it = weak_nodes_.begin(); - while (node_it != weak_nodes_.end()) { - if (node_it->lock() == node_ptr) { - break; - } + add_callback_group(group_ptr, node_ptr, weak_groups_associated_with_executor_to_nodes_); +} + +void +EventsExecutorEntitiesCollector::add_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + WeakCallbackGroupsToNodesMap & weak_groups_to_nodes) +{ + // If the callback_group already has an executor, throw error + std::atomic_bool & has_executor = group_ptr->get_associated_with_executor_atomic(); + if (has_executor.exchange(true)) { + throw std::runtime_error("Callback group has already been added to an executor."); } - if (node_it == weak_nodes_.end()) { - // The node is not stored here, so nothing to do - return; + + // Add callback group to weak_groups_to_node + rclcpp::CallbackGroup::WeakPtr weak_group_ptr = group_ptr; + + auto insert_info = weak_groups_to_nodes.insert(std::make_pair(weak_group_ptr, node_ptr)); + //auto insert_info = weak_groups_to_nodes.emplace_back(weak_group_ptr, node_ptr); + + // Throw error if the group was already registered in the executor + bool was_inserted = insert_info.second; + if (!was_inserted) { + throw std::runtime_error("Callback group was already added to executor."); } - // Set that the node does not have an executor anymore - std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); - has_executor.store(false); + // For all entities in the callback group, set their event callback + set_callback_group_entities_callbacks(group_ptr); - weak_nodes_.erase(node_it); + // Set an event callback for the node's notify guard condition, so if new entities are added + // or removed to this node we will receive an event. + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event, + this, + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); - unset_entities_callbacks(node_ptr); + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set node guard condition callback"); + } } void -EventsExecutorEntitiesCollector::set_entities_callbacks( +EventsExecutorEntitiesCollector::execute() +{ + // This function is called when the associated executor is notified that something changed. + // We do not know if an entity has been added or remode so we have to rebuild everything. + + timers_manager_->clear(); + + // If a registered node has a new callback group, register the group. + add_callback_groups_from_nodes_associated_to_executor(); + + // For all groups registered in the executor, set their event callbacks. + set_entities_event_callbacks_from_map(weak_groups_associated_with_executor_to_nodes_); + set_entities_event_callbacks_from_map(weak_groups_to_nodes_associated_with_executor_); +} + +void +EventsExecutorEntitiesCollector::add_callback_groups_from_nodes_associated_to_executor() +{ + // Register new callback groups added to a node while already spinning + for (const auto & weak_node : weak_nodes_) { + auto node = weak_node.lock(); + if (node) { + auto group_ptrs = node->get_callback_groups(); + std::for_each( + group_ptrs.begin(), group_ptrs.end(), + [this, node](rclcpp::CallbackGroup::WeakPtr group_ptr) + { + auto shared_group_ptr = group_ptr.lock(); + if (shared_group_ptr && shared_group_ptr->automatically_add_to_executor_with_node() && + !shared_group_ptr->get_associated_with_executor_atomic().load()) + { + add_callback_group( + shared_group_ptr, + node, + weak_groups_to_nodes_associated_with_executor_); + } + }); + } + } +} + +void +EventsExecutorEntitiesCollector::set_entities_event_callbacks_from_map( + const WeakCallbackGroupsToNodesMap & weak_groups_to_nodes) +{ + for (const auto & pair : weak_groups_to_nodes) { + auto group = pair.first.lock(); + auto node = pair.second.lock(); + if (!node || !group || !group->can_be_taken_from().load()) { + continue; + } + set_callback_group_entities_callbacks(group); + } +} + +void +EventsExecutorEntitiesCollector::set_node_entities_callbacks( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node) { // Set event callbacks for all entities in this node @@ -113,74 +207,7 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( continue; } - // Timers are handled by the timers manager - group->find_timer_ptrs_if( - [this](const rclcpp::TimerBase::SharedPtr & timer) { - if (timer) { - timers_manager_->add_timer(timer); - timer->set_on_destruction_callback( - std::bind(&TimersManager::remove_timer_raw, timers_manager_, std::placeholders::_1)); - } - return false; - }); - - // Set callbacks for all other entity types - group->find_subscription_ptrs_if( - [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { - if (subscription) { - subscription->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); - subscription->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); - } - return false; - }); - group->find_service_ptrs_if( - [this](const rclcpp::ServiceBase::SharedPtr & service) { - if (service) { - service->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); - service->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); - } - return false; - }); - group->find_client_ptrs_if( - [this](const rclcpp::ClientBase::SharedPtr & client) { - if (client) { - client->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); - client->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); - } - return false; - }); - group->find_waitable_ptrs_if( - [this](const rclcpp::Waitable::SharedPtr & waitable) { - if (waitable) { - waitable->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); - waitable->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); - } - return false; - }); + set_callback_group_entities_callbacks(group); } // Set an event callback for the node's notify guard condition, so if new entities are added @@ -198,7 +225,130 @@ EventsExecutorEntitiesCollector::set_entities_callbacks( } void -EventsExecutorEntitiesCollector::unset_entities_callbacks( +EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( + rclcpp::CallbackGroup::SharedPtr group) +{ + // Timers are handled by the timers manager + group->find_timer_ptrs_if( + [this](const rclcpp::TimerBase::SharedPtr & timer) { + if (timer) { + timers_manager_->add_timer(timer); + timer->set_on_destruction_callback( + std::bind(&TimersManager::remove_timer_raw, timers_manager_, std::placeholders::_1)); + } + return false; + }); + + // Set callbacks for all other entity types + group->find_subscription_ptrs_if( + [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { + if (subscription) { + subscription->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); + subscription->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); + } + return false; + }); + group->find_service_ptrs_if( + [this](const rclcpp::ServiceBase::SharedPtr & service) { + if (service) { + service->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); + service->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); + } + return false; + }); + group->find_client_ptrs_if( + [this](const rclcpp::ClientBase::SharedPtr & client) { + if (client) { + client->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); + client->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); + } + return false; + }); + group->find_waitable_ptrs_if( + [this](const rclcpp::Waitable::SharedPtr & waitable) { + if (waitable) { + waitable->set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event); + waitable->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + associated_executor_, + std::placeholders::_1)); + } + return false; + }); +} + +void +EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( + rclcpp::CallbackGroup::SharedPtr group) +{ + // Timers are handled by the timers manager + group->find_timer_ptrs_if( + [this](const rclcpp::TimerBase::SharedPtr & timer) { + if (timer) { + timers_manager_->remove_timer(timer); + timer->set_on_destruction_callback(nullptr); + } + return false; + }); + + // Unset callbacks for all other entity types + group->find_subscription_ptrs_if( + [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { + if (subscription) { + subscription->set_events_executor_callback(nullptr, nullptr); + subscription->set_on_destruction_callback(nullptr); + } + return false; + }); + group->find_service_ptrs_if( + [this](const rclcpp::ServiceBase::SharedPtr & service) { + if (service) { + service->set_events_executor_callback(nullptr, nullptr); + service->set_on_destruction_callback(nullptr); + } + return false; + }); + group->find_client_ptrs_if( + [this](const rclcpp::ClientBase::SharedPtr & client) { + if (client) { + client->set_events_executor_callback(nullptr, nullptr); + client->set_on_destruction_callback(nullptr); + } + return false; + }); + group->find_waitable_ptrs_if( + [this](const rclcpp::Waitable::SharedPtr & waitable) { + if (waitable) { + waitable->set_events_executor_callback(nullptr, nullptr); + waitable->set_on_destruction_callback(nullptr); + } + return false; + }); +} + +void +EventsExecutorEntitiesCollector::unset_node_entities_callbacks( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node) { // Unset event callbacks for all entities in this node @@ -209,49 +359,7 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( continue; } - // Timers are handled by the timers manager - group->find_timer_ptrs_if( - [this](const rclcpp::TimerBase::SharedPtr & timer) { - if (timer) { - timers_manager_->remove_timer(timer); - timer->set_on_destruction_callback(nullptr); - } - return false; - }); - - // Unset callbacks for all other entity types - group->find_subscription_ptrs_if( - [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { - if (subscription) { - subscription->set_events_executor_callback(nullptr, nullptr); - subscription->set_on_destruction_callback(nullptr); - } - return false; - }); - group->find_service_ptrs_if( - [this](const rclcpp::ServiceBase::SharedPtr & service) { - if (service) { - service->set_events_executor_callback(nullptr, nullptr); - service->set_on_destruction_callback(nullptr); - } - return false; - }); - group->find_client_ptrs_if( - [this](const rclcpp::ClientBase::SharedPtr & client) { - if (client) { - client->set_events_executor_callback(nullptr, nullptr); - client->set_on_destruction_callback(nullptr); - } - return false; - }); - group->find_waitable_ptrs_if( - [this](const rclcpp::Waitable::SharedPtr & waitable) { - if (waitable) { - waitable->set_events_executor_callback(nullptr, nullptr); - waitable->set_on_destruction_callback(nullptr); - } - return false; - }); + unset_callback_group_entities_callbacks(group); } // Unset the event callback for the node's notify guard condition, to stop receiving events @@ -265,3 +373,140 @@ EventsExecutorEntitiesCollector::unset_entities_callbacks( throw std::runtime_error("Couldn't set node guard condition callback"); } } + +void +EventsExecutorEntitiesCollector::remove_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr) +{ + this->remove_callback_group_from_map( + group_ptr, + weak_groups_associated_with_executor_to_nodes_); +} + +void +EventsExecutorEntitiesCollector::remove_callback_group_from_map( + rclcpp::CallbackGroup::SharedPtr group_ptr, + WeakCallbackGroupsToNodesMap & weak_groups_to_nodes) +{ + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr; + rclcpp::CallbackGroup::WeakPtr weak_group_ptr = group_ptr; + + // Look for the group to remove in the map + auto iter = weak_groups_to_nodes.find(weak_group_ptr); + if (iter != weak_groups_to_nodes.end()){ + node_ptr = iter->second.lock(); + if (node_ptr == nullptr) { + throw std::runtime_error("Node must not be deleted before its callback group(s)."); + } + // Remove group from map + weak_groups_to_nodes.erase(iter); + } else { + throw std::runtime_error("Callback group needs to be associated with executor."); + } + + // For all the entities in the group, unset their callbacks + unset_callback_group_entities_callbacks(group_ptr); +} + +void +EventsExecutorEntitiesCollector::remove_node( + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) +{ + if (!node_ptr->get_associated_with_executor_atomic().load()) { + throw std::runtime_error("Node needs to be associated with this executor."); + return; + } + // Check if this node is currently stored here + auto node_it = weak_nodes_.begin(); + while (node_it != weak_nodes_.end()) { + if (node_it->lock() == node_ptr) { + weak_nodes_.erase(node_it); + break; + } + } + if (node_it == weak_nodes_.end()) { + // The node is not stored here, so nothing to do + throw std::runtime_error("Tried to remove node not stored in executor."); + return; + } + + // Find callback groups belonging to the node to remove + std::vector found_group_ptrs; + + std::for_each( + weak_groups_to_nodes_associated_with_executor_.begin(), + weak_groups_to_nodes_associated_with_executor_.end(), + [&found_group_ptrs, node_ptr](std::pair key_value_pair) { + auto & weak_node_ptr = key_value_pair.second; + auto shared_node_ptr = weak_node_ptr.lock(); + auto group_ptr = key_value_pair.first.lock(); + if (shared_node_ptr == node_ptr) { + found_group_ptrs.push_back(group_ptr); + } + }); + + // Remove those callback groups + std::for_each( + found_group_ptrs.begin(), found_group_ptrs.end(), [this] + (rclcpp::CallbackGroup::SharedPtr group_ptr) { + this->remove_callback_group_from_map( + group_ptr, + weak_groups_to_nodes_associated_with_executor_); + }); + + // Set that the node does not have an executor anymore + std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); + has_executor.store(false); + + // Unset the node entities callbacks + unset_node_entities_callbacks(node_ptr); +} + +// Returns true if the map has the node_ptr +bool +EventsExecutorEntitiesCollector::has_node( + const rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + const WeakCallbackGroupsToNodesMap & weak_groups_to_nodes) const +{ + return std::find_if( + weak_groups_to_nodes.begin(), + weak_groups_to_nodes.end(), + [&](const WeakCallbackGroupsToNodesMap::value_type & other) -> bool { + auto other_ptr = other.second.lock(); + return other_ptr == node_ptr; + }) != weak_groups_to_nodes.end(); +} + +std::vector +EventsExecutorEntitiesCollector::get_all_callback_groups() +{ + std::vector groups; + for (const auto & group_node_ptr : weak_groups_associated_with_executor_to_nodes_) { + groups.push_back(group_node_ptr.first); + } + for (const auto & group_node_ptr : weak_groups_to_nodes_associated_with_executor_) { + groups.push_back(group_node_ptr.first); + } + return groups; +} + +std::vector +EventsExecutorEntitiesCollector::get_manually_added_callback_groups() +{ + std::vector groups; + for (const auto & group_node_ptr : weak_groups_associated_with_executor_to_nodes_) { + groups.push_back(group_node_ptr.first); + } + return groups; +} + +std::vector +EventsExecutorEntitiesCollector::get_automatically_added_callback_groups_from_nodes() +{ + std::vector groups; + for (const auto & group_node_ptr : weak_groups_to_nodes_associated_with_executor_) { + groups.push_back(group_node_ptr.first); + } + return groups; +} From 78404334fc3560e382af1c0e6d655d5ffe689308 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 17:21:10 +0100 Subject: [PATCH 061/168] copy/paste fix --- .../rclcpp/executors/events_executor_entities_collector.hpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 666b6f584d..4596fffeaa 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -194,9 +194,6 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable set_entities_event_callbacks_from_map( const WeakCallbackGroupsToNodesMap & weak_groups_to_nodes); - /// Memory strategy: an interface for handling user-defined memory allocation strategies. - rclcpp::memory_strategy::MemoryStrategy::SharedPtr memory_strategy_; - // maps callback groups to nodes. WeakCallbackGroupsToNodesMap weak_groups_associated_with_executor_to_nodes_; // maps callback groups to nodes. From 440b3f0ed31816d745e7ea795a30d7824b088274 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 17:25:58 +0100 Subject: [PATCH 062/168] Add missing unsets --- .../src/rclcpp/executors/events_executor_entities_collector.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index aeed985816..758ac27aed 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -35,6 +35,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() if (group) { std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); has_executor.store(false); + unset_callback_group_entities_callbacks(group); } } for (const auto & pair : weak_groups_to_nodes_associated_with_executor_) { @@ -42,6 +43,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() if (group) { std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); has_executor.store(false); + unset_callback_group_entities_callbacks(group); } } // Disassociate all nodes From a1e9f00fc303b23b190f693d21b80945037f99ef Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 17:32:46 +0100 Subject: [PATCH 063/168] use emplace --- .../rclcpp/executors/events_executor_entities_collector.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 758ac27aed..403d026e59 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -115,11 +115,10 @@ EventsExecutorEntitiesCollector::add_callback_group( // Add callback group to weak_groups_to_node rclcpp::CallbackGroup::WeakPtr weak_group_ptr = group_ptr; - auto insert_info = weak_groups_to_nodes.insert(std::make_pair(weak_group_ptr, node_ptr)); - //auto insert_info = weak_groups_to_nodes.emplace_back(weak_group_ptr, node_ptr); + auto emplace_info = weak_groups_to_nodes.emplace(weak_group_ptr, node_ptr); // Throw error if the group was already registered in the executor - bool was_inserted = insert_info.second; + bool was_inserted = emplace_info.second; if (!was_inserted) { throw std::runtime_error("Callback group was already added to executor."); } From 236a473ca3606e3b4971d68820c3655d0bf41dfe Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 17:38:18 +0100 Subject: [PATCH 064/168] Remove not needed unset_node_entities_callbacks --- .../src/rclcpp/executors/events_executor_entities_collector.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 403d026e59..ac8bb8ac1a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -52,7 +52,6 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() if (node) { std::atomic_bool & has_executor = node->get_associated_with_executor_atomic(); has_executor.store(false); - this->unset_node_entities_callbacks(node); } } From 7772beb05a6c8f769290fa0ca70fafcb0185143b Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 17:46:36 +0100 Subject: [PATCH 065/168] use insert --- .../rclcpp/executors/events_executor_entities_collector.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index ac8bb8ac1a..d6b041ad5a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -114,10 +114,10 @@ EventsExecutorEntitiesCollector::add_callback_group( // Add callback group to weak_groups_to_node rclcpp::CallbackGroup::WeakPtr weak_group_ptr = group_ptr; - auto emplace_info = weak_groups_to_nodes.emplace(weak_group_ptr, node_ptr); + auto insert_info = weak_groups_to_nodes.insert(std::make_pair(weak_group_ptr, node_ptr)); // Throw error if the group was already registered in the executor - bool was_inserted = emplace_info.second; + bool was_inserted = insert_info.second; if (!was_inserted) { throw std::runtime_error("Callback group was already added to executor."); } From cf4d2e0f9f40c5ef2a62bfd5128d21c5df9eb84e Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Fri, 16 Oct 2020 17:59:32 +0100 Subject: [PATCH 066/168] fix unit tests Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/client.hpp | 11 +++- .../rclcpp/executors/events_executor.hpp | 18 ++++-- .../rclcpp/executors/timers_manager.hpp | 9 ++- .../subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 9 ++- rclcpp/include/rclcpp/service.hpp | 11 +++- rclcpp/include/rclcpp/subscription_base.hpp | 11 +++- rclcpp/include/rclcpp/timer.hpp | 4 +- rclcpp/include/rclcpp/waitable.hpp | 11 +++- rclcpp/src/rclcpp/client.cpp | 8 ++- .../src/rclcpp/executors/events_executor.cpp | 1 - .../src/rclcpp/executors/timers_manager.cpp | 43 ++++++--------- rclcpp/src/rclcpp/service.cpp | 8 ++- rclcpp/src/rclcpp/subscription_base.cpp | 7 ++- .../subscription_intra_process_base.cpp | 4 +- rclcpp/src/rclcpp/timer.cpp | 4 +- rclcpp/src/rclcpp/waitable.cpp | 7 ++- .../rclcpp/executors/test_events_executor.cpp | 55 ++++++++++++++++++- .../test/rclcpp/executors/test_executors.cpp | 4 +- 19 files changed, 153 insertions(+), 74 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 8045f60d05..5036005219 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -51,6 +51,11 @@ namespace node_interfaces class NodeBaseInterface; } // namespace node_interfaces +namespace executors +{ +class EventsExecutor; +} // namespace executors + class ClientBase { public: @@ -154,12 +159,12 @@ class ClientBase RCLCPP_PUBLIC void set_events_executor_callback( - const void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const; RCLCPP_PUBLIC void - set_on_destruction_callback(std::function on_destruction_callback); + set_on_destruction_callback(std::function on_destruction_callback); protected: RCLCPP_DISABLE_COPY(ClientBase) @@ -176,7 +181,7 @@ class ClientBase const rcl_node_t * get_rcl_node_handle() const; - std::function on_destruction_callback_; + std::function on_destruction_callback_; rclcpp::node_interfaces::NodeGraphInterface::WeakPtr node_graph_; std::shared_ptr node_handle_; diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 7ca0bdbc86..4c705b9a2d 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -149,9 +149,9 @@ class EventsExecutor : public rclcpp::Executor this_executor->event_queue_cv_.notify_one(); } - template + template void - remove_entity(T* entity) + remove_entity(T * entity) { // We need to unset the callbacks to make sure that after removing events from the // queues, this entity will not push anymore before being completely destroyed. @@ -160,15 +160,21 @@ class EventsExecutor : public rclcpp::Executor // Remove events associated with this entity from the event queue { std::unique_lock lock(push_mutex_); - event_queue_.erase(std::remove_if(event_queue_.begin(), event_queue_.end(), - [&entity](ExecutorEvent event) { return event.entity == entity; }), event_queue_.end()); + event_queue_.erase( + std::remove_if( + event_queue_.begin(), event_queue_.end(), + [&entity](ExecutorEvent event) {return event.entity == entity;}), event_queue_.end()); } // Remove events associated with this entity from the local event queue { std::unique_lock lock(execution_mutex_); - local_event_queue_.erase(std::remove_if(local_event_queue_.begin(), local_event_queue_.end(), - [&entity](ExecutorEvent event) { return event.entity == entity; }), local_event_queue_.end()); + local_event_queue_.erase( + std::remove_if( + local_event_queue_.begin(), local_event_queue_.end(), + [&entity](ExecutorEvent event) { + return event.entity == entity; + }), local_event_queue_.end()); } } diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index e0ce781573..7b75fc2e94 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -125,7 +125,7 @@ class TimersManager * @brief Remove a single timer stored in the object, passed as a raw ptr. * @param timer the timer to remove. */ - void remove_timer_raw(rclcpp::TimerBase* timer); + void remove_timer_raw(rclcpp::TimerBase * timer); // This is what the TimersManager uses to denote a duration forever. // We don't use std::chrono::nanoseconds::max because it will overflow. @@ -179,12 +179,12 @@ class TimersManager std::make_heap(timers_.begin(), timers_.end(), timer_greater); } - TimerPtr& front() + TimerPtr & front() { return timers_.front(); } - const TimerPtr& front() const + const TimerPtr & front() const { return timers_.front(); } @@ -199,7 +199,6 @@ class TimersManager timers_.clear(); } - private: static bool timer_greater(TimerPtr a, TimerPtr b) { return a->time_until_trigger() > b->time_until_trigger(); @@ -263,7 +262,7 @@ class TimersManager // Flag used as predicate by timers_cv bool timers_updated_ {false}; // Indicates whether the timers thread is currently running or requested to stop - bool running_ {false}; + std::atomic running_ {false}; // Context of the parent executor std::shared_ptr context_; // MinHeap of timers diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index a961ef23c8..8d3a5dc2f5 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -73,7 +73,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void set_events_executor_callback( - void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const override; protected: diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 58bfa0331d..7aadf5a224 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -30,6 +30,11 @@ namespace rclcpp { +namespace executors +{ +class EventsExecutor; +} // namespace executors + using QOSDeadlineRequestedInfo = rmw_requested_deadline_missed_status_t; using QOSDeadlineOfferedInfo = rmw_offered_deadline_missed_status_t; using QOSLivelinessChangedInfo = rmw_liveliness_changed_status_t; @@ -152,11 +157,11 @@ class QOSEventHandler : public QOSEventHandlerBase RCLCPP_PUBLIC void set_events_executor_callback( - void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const override { rcl_ret_t ret = rcl_event_set_events_executor_callback( - executor_context, + executor, executor_callback, this, &event_handle_, diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 7fe3f51c77..c588f0a98a 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -39,6 +39,11 @@ namespace rclcpp { +namespace executors +{ +class EventsExecutor; +} // namespace executors + class ServiceBase { public: @@ -124,12 +129,12 @@ class ServiceBase RCLCPP_PUBLIC void set_events_executor_callback( - const void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const; RCLCPP_PUBLIC void - set_on_destruction_callback(std::function on_destruction_callback); + set_on_destruction_callback(std::function on_destruction_callback); protected: RCLCPP_DISABLE_COPY(ServiceBase) @@ -142,7 +147,7 @@ class ServiceBase const rcl_node_t * get_rcl_node_handle() const; - std::function on_destruction_callback_; + std::function on_destruction_callback_; std::shared_ptr node_handle_; diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index a47d64a74c..90df0f1cb5 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -46,6 +46,11 @@ namespace node_interfaces class NodeBaseInterface; } // namespace node_interfaces +namespace executors +{ +class EventsExecutor; +} // namespace executors + namespace experimental { /** @@ -267,12 +272,12 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void set_events_executor_callback( - const void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const; RCLCPP_PUBLIC void - set_on_destruction_callback(std::function on_destruction_callback); + set_on_destruction_callback(std::function on_destruction_callback); protected: template @@ -316,7 +321,7 @@ class SubscriptionBase : public std::enable_shared_from_this rosidl_message_type_support_t type_support_; bool is_serialized_; - std::function on_destruction_callback_; + std::function on_destruction_callback_; std::atomic subscription_in_use_by_wait_set_{false}; std::atomic intra_process_subscription_waitable_in_use_by_wait_set_{false}; diff --git a/rclcpp/include/rclcpp/timer.hpp b/rclcpp/include/rclcpp/timer.hpp index 421eb08355..9ec679b2ff 100644 --- a/rclcpp/include/rclcpp/timer.hpp +++ b/rclcpp/include/rclcpp/timer.hpp @@ -139,12 +139,12 @@ class TimerBase RCLCPP_PUBLIC void - set_on_destruction_callback(std::function on_destruction_callback); + set_on_destruction_callback(std::function on_destruction_callback); protected: Clock::SharedPtr clock_; std::shared_ptr timer_handle_; - std::function on_destruction_callback_; + std::function on_destruction_callback_; std::atomic in_use_by_wait_set_{false}; }; diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index dc4307d4ff..a9f30c68cb 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -26,6 +26,11 @@ namespace rclcpp { +namespace executors +{ +class EventsExecutor; +} // namespace executors + class Waitable { public: @@ -170,15 +175,15 @@ class Waitable virtual void set_events_executor_callback( - void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const; RCLCPP_PUBLIC void - set_on_destruction_callback(std::function on_destruction_callback); + set_on_destruction_callback(std::function on_destruction_callback); private: - std::function on_destruction_callback_; + std::function on_destruction_callback_; std::atomic in_use_by_wait_set_{false}; }; // class Waitable diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 7d8666344f..0534ba6bdc 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -204,11 +204,11 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( - const void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_client_set_events_executor_callback( - executor_context, + executor, executor_callback, this, client_handle_.get()); @@ -218,7 +218,9 @@ ClientBase::set_events_executor_callback( } } -void ClientBase::set_on_destruction_callback(std::function on_destruction_callback) +void +ClientBase::set_on_destruction_callback( + std::function on_destruction_callback) { on_destruction_callback_ = on_destruction_callback; } diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 168504d2b7..ee4f734fc6 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -164,7 +164,6 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Keep executing until work available or timeout expired while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { - std::unique_lock push_lock(push_mutex_); std::unique_lock execution_lock(execution_mutex_); std::swap(local_event_queue_, event_queue_); diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index f9e93f9a00..b8c97a96f3 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -28,11 +28,11 @@ TimersManager::TimersManager(std::shared_ptr context) TimersManager::~TimersManager() { - // Make sure timers thread is stopped before destroying this object - this->stop(); - // Remove all timers this->clear(); + + // Make sure timers thread is stopped before destroying this object + this->stop(); } void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) @@ -53,29 +53,25 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) void TimersManager::start() { - std::unique_lock timers_lock(timers_mutex_); - // Make sure that the thread is not already running - if (running_) { + if (running_.exchange(true)) { throw std::runtime_error("TimersManager::start() can't start timers thread as already running"); } - running_ = true; timers_thread_ = std::thread(&TimersManager::run_timers, this); pthread_setname_np(timers_thread_.native_handle(), "TimersManager"); } void TimersManager::stop() { + // Nothing to do if the timers thread is not running + // or if another thred already signaled to stop. + if (!running_.exchange(false)) { + return; + } + { std::unique_lock timers_lock(timers_mutex_); - // Nothing to do if the timers thread is not running - // or if another thred already signaled to stop. - if (!running_) { - return; - } - - running_ = false; timers_updated_ = true; } timers_cv_.notify_one(); @@ -88,40 +84,38 @@ void TimersManager::stop() std::chrono::nanoseconds TimersManager::get_head_timeout() { - std::unique_lock lock(timers_mutex_); - // Do not allow to interfere with the thread running if (running_) { throw std::runtime_error( "TimersManager::get_head_timeout() can't be used while timers thread is running"); } + std::unique_lock lock(timers_mutex_); return this->get_head_timeout_unsafe(); } void TimersManager::execute_ready_timers() { - std::unique_lock lock(timers_mutex_); - // Do not allow to interfere with the thread running if (running_) { throw std::runtime_error( "TimersManager::execute_ready_timers() can't be used while timers thread is running"); } + std::unique_lock lock(timers_mutex_); this->execute_ready_timers_unsafe(); } bool TimersManager::execute_head_timer() { - std::unique_lock lock(timers_mutex_); - // Do not allow to interfere with the thread running if (running_) { throw std::runtime_error( "TimersManager::execute_head_timer() can't be used while timers thread is running"); } + std::unique_lock lock(timers_mutex_); + // Nothing to do if we don't have any timer if (heap_.empty()) { return false; @@ -164,14 +158,10 @@ void TimersManager::execute_ready_timers_unsafe() void TimersManager::run_timers() { - while (rclcpp::ok(context_)) { + while (rclcpp::ok(context_) && running_) { // Lock mutex std::unique_lock timers_lock(timers_mutex_); - if (!running_) { - break; - } - // Get timeout before next timer expires auto time_to_sleep = this->get_head_timeout_unsafe(); // Wait until timeout or notification that timers have been updated @@ -184,7 +174,6 @@ void TimersManager::run_timers() // Make sure the running flag is set to false when we exit from this function // to allow restarting the timers thread. - std::unique_lock timers_lock(timers_mutex_); running_ = false; } @@ -207,7 +196,7 @@ void TimersManager::remove_timer(rclcpp::TimerBase::SharedPtr timer) this->remove_timer_raw(timer.get()); } -void TimersManager::remove_timer_raw(rclcpp::TimerBase* timer) +void TimersManager::remove_timer_raw(rclcpp::TimerBase * timer) { { std::unique_lock timers_lock(timers_mutex_); diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index b0c81d73e1..639de9858d 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -91,11 +91,11 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( - const void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_service_set_events_executor_callback( - executor_context, + executor, executor_callback, this, service_handle_.get()); @@ -105,7 +105,9 @@ ServiceBase::set_events_executor_callback( } } -void ServiceBase::set_on_destruction_callback(std::function on_destruction_callback) +void +ServiceBase::set_on_destruction_callback( + std::function on_destruction_callback) { on_destruction_callback_ = on_destruction_callback; } diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index d7ba313531..4765adbfc6 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -295,11 +295,11 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( - const void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_subscription_set_events_executor_callback( - executor_context, + executor, executor_callback, this, subscription_handle_.get()); @@ -310,7 +310,8 @@ SubscriptionBase::set_events_executor_callback( } void -SubscriptionBase::set_on_destruction_callback(std::function on_destruction_callback) +SubscriptionBase::set_on_destruction_callback( + std::function on_destruction_callback) { on_destruction_callback_ = on_destruction_callback; } diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index ae2bad93f4..754573e716 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -41,11 +41,11 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( - void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - executor_context, + executor, executor_callback, this, &gc_, diff --git a/rclcpp/src/rclcpp/timer.cpp b/rclcpp/src/rclcpp/timer.cpp index 64545b91b3..4659898055 100644 --- a/rclcpp/src/rclcpp/timer.cpp +++ b/rclcpp/src/rclcpp/timer.cpp @@ -141,7 +141,9 @@ TimerBase::exchange_in_use_by_wait_set_state(bool in_use_state) return in_use_by_wait_set_.exchange(in_use_state); } -void TimerBase::set_on_destruction_callback(std::function on_destruction_callback) +void +TimerBase::set_on_destruction_callback( + std::function on_destruction_callback) { on_destruction_callback_ = on_destruction_callback; } diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index f2726ff842..919af579d5 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -69,10 +69,10 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( - void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const { - (void)executor_context; + (void)executor; (void)executor_callback; throw std::runtime_error( @@ -80,7 +80,8 @@ Waitable::set_events_executor_callback( } void -Waitable::set_on_destruction_callback(std::function on_destruction_callback) +Waitable::set_on_destruction_callback( + std::function on_destruction_callback) { on_destruction_callback_ = on_destruction_callback; } diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index 406fd68d07..7250186e28 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -179,6 +179,24 @@ TEST_F(TestEventsExecutor, spin_some_max_duration) } } +TEST_F(TestEventsExecutor, spin_some_zero_duration) +{ + auto node = std::make_shared("node"); + + size_t t_runs = 0; + auto t = node->create_wall_timer( + 20ms, + [&]() { + t_runs++; + }); + + EventsExecutor executor; + executor.add_node(node); + executor.spin_some(0ms); + + EXPECT_EQ(1u, t_runs); +} + TEST_F(TestEventsExecutor, spin_all_max_duration) { { @@ -226,6 +244,41 @@ TEST_F(TestEventsExecutor, spin_all_max_duration) EXPECT_THROW(executor.spin_all(-5ms), std::invalid_argument); } +TEST_F(TestEventsExecutor, cancel_while_timers_running) +{ + auto node = std::make_shared("node"); + + size_t t1_runs = 0; + auto t1 = node->create_wall_timer( + 1ms, + [&]() { + t1_runs++; + std::this_thread::sleep_for(25ms); + }); + + size_t t2_runs = 0; + auto t2 = node->create_wall_timer( + 1ms, + [&]() { + t2_runs++; + std::this_thread::sleep_for(25ms); + }); + + EventsExecutor executor; + executor.add_node(node); + + std::thread spinner([&executor, this]() {executor.spin();}); + + std::this_thread::sleep_for(10ms); + // Call cancel while t1 callback is still being executed + executor.cancel(); + spinner.join(); + + // Depending on the latency on the system, t2 may start to execute before cancel is signaled + EXPECT_GE(1u, t1_runs); + EXPECT_GE(1u, t2_runs); +} + TEST_F(TestEventsExecutor, cancel_while_timers_waiting) { auto node = std::make_shared("node"); @@ -257,7 +310,7 @@ TEST_F(TestEventsExecutor, destroy_entities) auto node_pub = std::make_shared("node_pub"); auto publisher = node_pub->create_publisher("topic", rclcpp::QoS(10)); auto timer = node_pub->create_wall_timer( - 2ms, [&]() { publisher->publish(std::make_unique()); }); + 2ms, [&]() {publisher->publish(std::make_unique());}); EventsExecutor executor_pub; executor_pub.add_node(node_pub); std::thread spinner([&executor_pub, this]() {executor_pub.spin();}); diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index b05ca0db6d..e65064f696 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -454,11 +454,11 @@ class TestWaitable : public rclcpp::Waitable void set_events_executor_callback( - void * executor_context, + const rclcpp::executors::EventsExecutor * executor, ExecutorEventCallback executor_callback) const override { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - executor_context, + executor, executor_callback, this, &gc_, From 69b244e8e11b0f419135363e86c095191051e8a1 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 18:02:31 +0100 Subject: [PATCH 067/168] Adress comments --- .../events_executor_entities_collector.hpp | 3 -- .../events_executor_entities_collector.cpp | 32 +++++++++++-------- 2 files changed, 18 insertions(+), 17 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 4596fffeaa..f41b0ba9ab 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -159,9 +159,6 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable get_automatically_added_callback_groups_from_nodes(); private: - void - set_node_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); - void unset_node_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index d6b041ad5a..117929ae6e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -52,6 +52,12 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() if (node) { std::atomic_bool & has_executor = node->get_associated_with_executor_atomic(); has_executor.store(false); + + // Unset node's guard condition event callback + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + nullptr, nullptr, nullptr, + node->get_notify_guard_condition(), + false); } } @@ -83,11 +89,17 @@ EventsExecutorEntitiesCollector::add_node( } } + // Set an event callback for the node's notify guard condition, so if new entities are added + // or removed to this node we will receive an event. + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event, + this, + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); + // Add node to weak_nodes_ weak_nodes_.push_back(node_ptr); - - // Set node's entities callbacks - set_node_entities_callbacks(node_ptr); } @@ -210,14 +222,6 @@ EventsExecutorEntitiesCollector::set_node_entities_callbacks( set_callback_group_entities_callbacks(group); } - // Set an event callback for the node's notify guard condition, so if new entities are added - // or removed to this node we will receive an event. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event, - this, - node->get_notify_guard_condition(), - false /* Discard previous events */); if (ret != RCL_RET_OK) { throw std::runtime_error("Couldn't set node guard condition callback"); @@ -400,12 +404,12 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( } // Remove group from map weak_groups_to_nodes.erase(iter); + + // For all the entities in the group, unset their callbacks + unset_callback_group_entities_callbacks(group_ptr); } else { throw std::runtime_error("Callback group needs to be associated with executor."); } - - // For all the entities in the group, unset their callbacks - unset_callback_group_entities_callbacks(group_ptr); } void From 9a8e4513fa7cf51f5c559b00ce4e03a3909682f4 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 18:07:33 +0100 Subject: [PATCH 068/168] Remove not needed unset_node_entities_callbacks --- .../events_executor_entities_collector.hpp | 3 -- .../events_executor_entities_collector.cpp | 35 ++++--------------- 2 files changed, 6 insertions(+), 32 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index f41b0ba9ab..4617b29c67 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -159,9 +159,6 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable get_automatically_added_callback_groups_from_nodes(); private: - void - unset_node_entities_callbacks(rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node); - void set_callback_group_entities_callbacks(rclcpp::CallbackGroup::SharedPtr group); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 117929ae6e..af8bf3b78f 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -351,33 +351,6 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( }); } -void -EventsExecutorEntitiesCollector::unset_node_entities_callbacks( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node) -{ - // Unset event callbacks for all entities in this node - // by searching them in all callback groups - for (auto & weak_group : node->get_callback_groups()) { - auto group = weak_group.lock(); - if (!group || !group->can_be_taken_from().load()) { - continue; - } - - unset_callback_group_entities_callbacks(group); - } - - // Unset the event callback for the node's notify guard condition, to stop receiving events - // if entities are added or removed to this node. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node->get_notify_guard_condition(), - false); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } -} - void EventsExecutorEntitiesCollector::remove_callback_group( rclcpp::CallbackGroup::SharedPtr group_ptr) @@ -463,8 +436,12 @@ EventsExecutorEntitiesCollector::remove_node( std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); has_executor.store(false); - // Unset the node entities callbacks - unset_node_entities_callbacks(node_ptr); + // Unset the event callback for the node's notify guard condition, to stop receiving events + // if entities are added or removed to this node. + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + nullptr, nullptr, nullptr, + node->get_notify_guard_condition(), + false); } // Returns true if the map has the node_ptr From 6f41af9a2d4d67045bb9785949103da94ddce97b Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 18:25:17 +0100 Subject: [PATCH 069/168] fixes --- .../events_executor_entities_collector.cpp | 33 +++++++------------ 1 file changed, 11 insertions(+), 22 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index af8bf3b78f..410c88b242 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -58,6 +58,9 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() nullptr, nullptr, nullptr, node->get_notify_guard_condition(), false); + + // Cant' throw exeptions in destructors + (void)ret; } } @@ -98,6 +101,9 @@ EventsExecutorEntitiesCollector::add_node( node_ptr->get_notify_guard_condition(), false /* Discard previous events */); + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set node guard condition callback"); + } // Add node to weak_nodes_ weak_nodes_.push_back(node_ptr); } @@ -207,27 +213,6 @@ EventsExecutorEntitiesCollector::set_entities_event_callbacks_from_map( } } -void -EventsExecutorEntitiesCollector::set_node_entities_callbacks( - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node) -{ - // Set event callbacks for all entities in this node - // by searching them in all callback groups - for (auto & weak_group : node->get_callback_groups()) { - auto group = weak_group.lock(); - if (!group || !group->can_be_taken_from().load()) { - continue; - } - - set_callback_group_entities_callbacks(group); - } - - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } -} - void EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( rclcpp::CallbackGroup::SharedPtr group) @@ -440,8 +425,12 @@ EventsExecutorEntitiesCollector::remove_node( // if entities are added or removed to this node. rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( nullptr, nullptr, nullptr, - node->get_notify_guard_condition(), + node_ptr->get_notify_guard_condition(), false); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set node guard condition callback"); + } } // Returns true if the map has the node_ptr From 8c4db59258104f481525d3679aa6bd99c24e2e77 Mon Sep 17 00:00:00 2001 From: Mauro Date: Fri, 16 Oct 2020 18:28:12 +0100 Subject: [PATCH 070/168] Set gc's callback only if new node --- .../events_executor_entities_collector.cpp | 31 +++++++++++-------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 410c88b242..dd86f39415 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -129,6 +129,24 @@ EventsExecutorEntitiesCollector::add_callback_group( throw std::runtime_error("Callback group has already been added to an executor."); } + bool is_new_node = !has_node(node_ptr, weak_groups_associated_with_executor_to_nodes_) && + !has_node(node_ptr, weak_groups_to_nodes_associated_with_executor_); + + if (is_new_node) { + // Set an event callback for the node's notify guard condition, so if new entities are added + // or removed to this node we will receive an event. + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event, + this, + node_ptr->get_notify_guard_condition(), + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set node guard condition callback"); + } + } + // Add callback group to weak_groups_to_node rclcpp::CallbackGroup::WeakPtr weak_group_ptr = group_ptr; @@ -142,19 +160,6 @@ EventsExecutorEntitiesCollector::add_callback_group( // For all entities in the callback group, set their event callback set_callback_group_entities_callbacks(group_ptr); - - // Set an event callback for the node's notify guard condition, so if new entities are added - // or removed to this node we will receive an event. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event, - this, - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } } void From 888ce252f6f2091991da96c56b944724326d81b6 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 19 Oct 2020 09:41:58 +0100 Subject: [PATCH 071/168] fix comments Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/executors/events_executor.hpp | 5 +++++ .../rclcpp/executors/events_executor_entities_collector.cpp | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 4c705b9a2d..23e66ebadc 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -149,12 +149,17 @@ class EventsExecutor : public rclcpp::Executor this_executor->event_queue_cv_.notify_one(); } + // This function allows to remove an entity from the EventsExecutor. + // Entities are any of SubscriptionBase, PublisherBase, ClientBase, ServerBase, Waitable. + // After an entity has been removed using this API, it can be safely destroyed without the risk + // that the executor would try to access it again. template void remove_entity(T * entity) { // We need to unset the callbacks to make sure that after removing events from the // queues, this entity will not push anymore before being completely destroyed. + // This assumes that all supported entities implement this function. entity->set_events_executor_callback(nullptr, nullptr); // Remove events associated with this entity from the event queue diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index acf3b4b7f3..4eb8f778e1 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -47,7 +47,7 @@ void EventsExecutorEntitiesCollector::execute() { // This function is called when the associated executor is notified that something changed. - // We do not know if an entity has been added or remode so we have to rebuild everything. + // We do not know if an entity has been added or removed so we have to rebuild everything. timers_manager_->clear(); From d88d05b6ab4ed920d9823ba89dd32ebd9482216d Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 19 Oct 2020 11:01:01 +0100 Subject: [PATCH 072/168] Change set/unset node's notify guard condition location --- .../events_executor_entities_collector.cpp | 43 +++++++++---------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index dd86f39415..30b0aa12af 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -92,18 +92,6 @@ EventsExecutorEntitiesCollector::add_node( } } - // Set an event callback for the node's notify guard condition, so if new entities are added - // or removed to this node we will receive an event. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event, - this, - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } // Add node to weak_nodes_ weak_nodes_.push_back(node_ptr); } @@ -361,6 +349,7 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( // Look for the group to remove in the map auto iter = weak_groups_to_nodes.find(weak_group_ptr); if (iter != weak_groups_to_nodes.end()){ + // Group found, get its associated node. node_ptr = iter->second.lock(); if (node_ptr == nullptr) { throw std::runtime_error("Node must not be deleted before its callback group(s)."); @@ -370,6 +359,25 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( // For all the entities in the group, unset their callbacks unset_callback_group_entities_callbacks(group_ptr); + + // Check if this node still has other callback groups associated with the executor + bool node_has_associated_callback_groups = + !has_node(node_ptr, weak_groups_associated_with_executor_to_nodes_) && + !has_node(node_ptr, weak_groups_to_nodes_associated_with_executor_); + + if(!node_has_associated_callback_groups) { + // Node doesn't have more callback groups associated to the executor. + // Unset the event callback for the node's notify guard condition, to stop + // receiving events if entities are added or removed to this node. + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + nullptr, nullptr, nullptr, + node_ptr->get_notify_guard_condition(), + false); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set node guard condition callback"); + } + } } else { throw std::runtime_error("Callback group needs to be associated with executor."); } @@ -425,17 +433,6 @@ EventsExecutorEntitiesCollector::remove_node( // Set that the node does not have an executor anymore std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); has_executor.store(false); - - // Unset the event callback for the node's notify guard condition, to stop receiving events - // if entities are added or removed to this node. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node_ptr->get_notify_guard_condition(), - false); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } } // Returns true if the map has the node_ptr From 2b9eb8dfd121d917fddbe36eb923521618b5e583 Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 19 Oct 2020 11:21:35 +0100 Subject: [PATCH 073/168] Don't use weak_nodes list to unset guard conditions --- .../executors/events_executor_entities_collector.cpp | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 30b0aa12af..f45e7277ac 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -36,6 +36,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); has_executor.store(false); unset_callback_group_entities_callbacks(group); + remove_callback_group_from_map(group, weak_groups_associated_with_executor_to_nodes_); } } for (const auto & pair : weak_groups_to_nodes_associated_with_executor_) { @@ -44,6 +45,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); has_executor.store(false); unset_callback_group_entities_callbacks(group); + remove_callback_group_from_map(group, weak_groups_to_nodes_associated_with_executor_); } } // Disassociate all nodes @@ -52,15 +54,6 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() if (node) { std::atomic_bool & has_executor = node->get_associated_with_executor_atomic(); has_executor.store(false); - - // Unset node's guard condition event callback - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node->get_notify_guard_condition(), - false); - - // Cant' throw exeptions in destructors - (void)ret; } } From 9b4856f4e9e39e269cb5a126766550f8c48a8e1d Mon Sep 17 00:00:00 2001 From: Mauro Date: Mon, 19 Oct 2020 12:28:10 +0100 Subject: [PATCH 074/168] Add nodes_notify_guard_conditions_ list --- .../events_executor_entities_collector.hpp | 2 ++ .../events_executor_entities_collector.cpp | 26 +++++++++++++++++-- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 4617b29c67..b662266be5 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -193,6 +193,8 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable // maps callback groups to nodes. WeakCallbackGroupsToNodesMap weak_groups_to_nodes_associated_with_executor_; + /// List of nodes notify guard conditions + std::list nodes_notify_guard_conditions_; /// List of weak nodes registered in the events executor std::list weak_nodes_; /// Executor using this entities collector object diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index f45e7277ac..a81dff987d 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -36,7 +36,6 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); has_executor.store(false); unset_callback_group_entities_callbacks(group); - remove_callback_group_from_map(group, weak_groups_associated_with_executor_to_nodes_); } } for (const auto & pair : weak_groups_to_nodes_associated_with_executor_) { @@ -45,9 +44,9 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() std::atomic_bool & has_executor = group->get_associated_with_executor_atomic(); has_executor.store(false); unset_callback_group_entities_callbacks(group); - remove_callback_group_from_map(group, weak_groups_to_nodes_associated_with_executor_); } } + // Disassociate all nodes for (const auto & weak_node : weak_nodes_) { auto node = weak_node.lock(); @@ -57,9 +56,20 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() } } + // Unset nodes notify guard condition executor callback + for (const auto & node_gc : nodes_notify_guard_conditions_) { + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + nullptr, nullptr, nullptr, + node_gc, + false); + + (void)ret; // Can't throw on destructors + } + // Clear all lists weak_groups_associated_with_executor_to_nodes_.clear(); weak_groups_to_nodes_associated_with_executor_.clear(); + nodes_notify_guard_conditions_.clear(); weak_nodes_.clear(); } @@ -126,6 +136,9 @@ EventsExecutorEntitiesCollector::add_callback_group( if (ret != RCL_RET_OK) { throw std::runtime_error("Couldn't set node guard condition callback"); } + + // Store node's notify guard condition + nodes_notify_guard_conditions_.push_back(node_ptr->get_notify_guard_condition()); } // Add callback group to weak_groups_to_node @@ -370,6 +383,15 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( if (ret != RCL_RET_OK) { throw std::runtime_error("Couldn't set node guard condition callback"); } + + // Remove guard condition from list + auto gc_it = nodes_notify_guard_conditions_.begin(); + while (gc_it != nodes_notify_guard_conditions_.end()) { + if (*gc_it == node_ptr->get_notify_guard_condition()) { + nodes_notify_guard_conditions_.erase(gc_it); + break; + } + } } } else { throw std::runtime_error("Callback group needs to be associated with executor."); From d541892754348cf77b7ec01fa966cf08fddca811 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 19 Oct 2020 15:16:03 +0100 Subject: [PATCH 075/168] make sure that node does not get destroyed while unsetting guard condition Signed-off-by: Soragna, Alberto --- .../events_executor_entities_collector.hpp | 8 +++-- .../events_executor_entities_collector.cpp | 30 +++++++++---------- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index b662266be5..f0053e167c 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -193,8 +193,12 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable // maps callback groups to nodes. WeakCallbackGroupsToNodesMap weak_groups_to_nodes_associated_with_executor_; - /// List of nodes notify guard conditions - std::list nodes_notify_guard_conditions_; + typedef std::map> + WeakNodesToGuardConditionsMap; + WeakNodesToGuardConditionsMap weak_nodes_to_guard_conditions_; + /// List of weak nodes registered in the events executor std::list weak_nodes_; /// Executor using this entities collector object diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 8f6365fb32..08d4904a98 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -57,19 +57,23 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() } // Unset nodes notify guard condition executor callback - for (const auto & node_gc : nodes_notify_guard_conditions_) { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node_gc, - false); + for (const auto & pair : weak_nodes_to_guard_conditions_) { + auto node = pair.first.lock(); + if (node) { + auto node_gc = pair.second; + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + nullptr, nullptr, nullptr, + node_gc, + false); - (void)ret; // Can't throw on destructors + (void)ret; // Can't throw on destructors + } } // Clear all lists weak_groups_associated_with_executor_to_nodes_.clear(); weak_groups_to_nodes_associated_with_executor_.clear(); - nodes_notify_guard_conditions_.clear(); + weak_nodes_to_guard_conditions_.clear(); weak_nodes_.clear(); } @@ -138,7 +142,8 @@ EventsExecutorEntitiesCollector::add_callback_group( } // Store node's notify guard condition - nodes_notify_guard_conditions_.push_back(node_ptr->get_notify_guard_condition()); + rclcpp::node_interfaces::NodeBaseInterface::WeakPtr node_weak_ptr(node_ptr); + weak_nodes_to_guard_conditions_[node_weak_ptr] = node_ptr->get_notify_guard_condition(); } // Add callback group to weak_groups_to_node @@ -385,13 +390,8 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( } // Remove guard condition from list - auto gc_it = nodes_notify_guard_conditions_.begin(); - while (gc_it != nodes_notify_guard_conditions_.end()) { - if (*gc_it == node_ptr->get_notify_guard_condition()) { - nodes_notify_guard_conditions_.erase(gc_it); - break; - } - } + rclcpp::node_interfaces::NodeBaseInterface::WeakPtr weak_node_ptr(node_ptr); + weak_nodes_to_guard_conditions_.erase(weak_node_ptr); } } else { throw std::runtime_error("Callback group needs to be associated with executor."); From 1e0c24e4ac79dc96c809a3f904007e47d00e1d37 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 19 Oct 2020 14:06:18 +0100 Subject: [PATCH 076/168] fix usage of destruction callback in waitables, add executor notifier Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/event_waitable.hpp | 75 ++++++++++++++++ .../rclcpp/executors/events_executor.hpp | 9 +- .../events_executor_entities_collector.hpp | 24 +----- .../events_executor_notify_waitable.hpp | 86 +++++++++++++++++++ .../subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 26 ++---- rclcpp/include/rclcpp/waitable.hpp | 6 +- .../src/rclcpp/executors/events_executor.cpp | 46 ++++------ .../events_executor_entities_collector.cpp | 11 +-- rclcpp/src/rclcpp/qos_event.cpp | 21 +++++ .../subscription_intra_process_base.cpp | 11 ++- rclcpp/src/rclcpp/waitable.cpp | 7 -- .../test/rclcpp/executors/test_executors.cpp | 7 ++ 13 files changed, 241 insertions(+), 90 deletions(-) create mode 100644 rclcpp/include/rclcpp/executors/event_waitable.hpp create mode 100644 rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp new file mode 100644 index 0000000000..895bf13f50 --- /dev/null +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -0,0 +1,75 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXECUTORS__EVENT_WAITABLE_HPP_ +#define RCLCPP__EXECUTORS__EVENT_WAITABLE_HPP_ + +#include "rclcpp/waitable.hpp" + +namespace rclcpp +{ +namespace executors +{ + +/** + * @brief This class provides a wrapper around the waitable object, that is + * meant to be used with the EventsExecutor. + * The waitset related methods are stubbed out as they should not be called. + * Nodes who want to implement a custom EventWaitable, can derive from this class and override + * the execute function. + */ +class EventWaitable : public rclcpp::Waitable +{ +public: + RCLCPP_SMART_PTR_DEFINITIONS(EventWaitable) + + // Constructor + RCLCPP_PUBLIC + EventWaitable() = default; + + // Destructor + RCLCPP_PUBLIC + virtual ~EventWaitable() = default; + + // Executing an EventWaitable is a no-op. + // Derive from this class to implement execute function. + RCLCPP_PUBLIC + virtual void + execute() = 0; + + // Stub API: not used by EventsExecutor + RCLCPP_PUBLIC + bool + is_ready(rcl_wait_set_t * wait_set) final + { + (void)wait_set; + throw std::runtime_error("EventWaitable can't be checked if it's ready"); + return false; + } + + // Stub API: not used by EventsExecutor + RCLCPP_PUBLIC + bool + add_to_wait_set(rcl_wait_set_t * wait_set) final + { + (void)wait_set; + throw std::runtime_error("EventWaitable can't be added to a wait_set"); + return false; + } +}; + +} // namespace executors +} // namespace rclcpp + +#endif // RCLCPP__EXECUTORS__EVENT_WAITABLE_HPP_ diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index d784b6842e..ead774011b 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -18,12 +18,13 @@ #include #include #include -#include +#include #include "rcutils/executor_event_types.h" #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" +#include "rclcpp/executors/events_executor_notify_waitable.hpp" #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node.hpp" @@ -57,7 +58,7 @@ class EventsExecutor : public rclcpp::Executor /// Default destrcutor. RCLCPP_PUBLIC - virtual ~EventsExecutor(); + virtual ~EventsExecutor() = default; /// Events executor implementation of spin. /** @@ -231,8 +232,6 @@ class EventsExecutor : public rclcpp::Executor // facilitate the removal of events from expired entities. using EventQueue = std::deque; - EventsExecutorEntitiesCollector::SharedPtr entities_collector_; - /// Extract and execute events from the queue until it is empty RCLCPP_PUBLIC void @@ -247,6 +246,8 @@ class EventsExecutor : public rclcpp::Executor EventQueue event_queue_; EventQueue local_event_queue_; + EventsExecutorEntitiesCollector::SharedPtr entities_collector_; + EventsExecutorNotifyWaitable::SharedPtr executor_notifier_; // Mutex to protect the insertion of events in the queue std::mutex push_mutex_; // Mutex to protect the execution of events diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index f0053e167c..6f69ebcaab 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -16,11 +16,13 @@ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_ENTITIES_COLLECTOR_HPP_ #include +#include #include +#include +#include "rclcpp/executors/event_waitable.hpp" #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node_interfaces/node_base_interface.hpp" -#include "rclcpp/waitable.hpp" namespace rclcpp { @@ -46,7 +48,7 @@ class EventsExecutor; * When this occurs, the execute API takes care of handling changes * in the entities currently used by the executor. */ -class EventsExecutorEntitiesCollector final : public rclcpp::Waitable +class EventsExecutorEntitiesCollector final : public EventWaitable { public: RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorEntitiesCollector) @@ -79,24 +81,6 @@ class EventsExecutorEntitiesCollector final : public rclcpp::Waitable remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr); - // Stub API: not used by EventsExecutor - RCLCPP_PUBLIC - bool - is_ready(rcl_wait_set_t * wait_set) override - { - (void)wait_set; - return false; - } - - // Stub API: not used by EventsExecutor - RCLCPP_PUBLIC - bool - add_to_wait_set(rcl_wait_set_t * wait_set) override - { - (void)wait_set; - return false; - } - /// Add a callback group to the entities collector /** * \see rclcpp::Executor::add_callback_group diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp new file mode 100644 index 0000000000..5f937078c6 --- /dev/null +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -0,0 +1,86 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXECUTORS__EVENTS_EXECUTOR_NOTIFY_WAITABLE_HPP_ +#define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_NOTIFY_WAITABLE_HPP_ + +#include + +#include "rcl/guard_condition.h" +#include "rclcpp/executors/event_waitable.hpp" + +namespace rclcpp +{ +namespace executors +{ + +class EventsExecutorNotifyWaitable final : public EventWaitable +{ +public: + RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorNotifyWaitable) + + // Constructor + RCLCPP_PUBLIC + EventsExecutorNotifyWaitable() = default; + + // Destructor + RCLCPP_PUBLIC + virtual ~EventsExecutorNotifyWaitable() + { + if (on_destruction_callback_) { + on_destruction_callback_(this); + } + } + + // The function is a no-op, since we only care of waking up the executor + RCLCPP_PUBLIC + void + execute() override + {} + + RCLCPP_PUBLIC + void + add_guard_condition(rcl_guard_condition_t * guard_condition) + { + notify_guard_conditions_.push_back(guard_condition); + } + + RCLCPP_PUBLIC + void + set_events_executor_callback( + const rclcpp::executors::EventsExecutor * executor, + ExecutorEventCallback executor_callback) const override + { + for (auto gc : notify_guard_conditions_) { + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + executor, + executor_callback, + this, + gc, + false); + + if (RCL_RET_OK != ret) { + throw std::runtime_error("Couldn't set guard condition callback"); + } + } + } + +private: + std::list notify_guard_conditions_; +}; + +} // namespace executors +} // namespace rclcpp + +#endif // RCLCPP__EXECUTORS__EVENTS_EXECUTOR_NOTIFY_WAITABLE_HPP_ diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 8d3a5dc2f5..9d5af14b8d 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -43,7 +43,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable : topic_name_(topic_name), qos_profile_(qos_profile) {} - virtual ~SubscriptionIntraProcessBase() = default; + virtual ~SubscriptionIntraProcessBase(); RCLCPP_PUBLIC size_t diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 7aadf5a224..8ff2cfc2fa 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -105,6 +105,13 @@ class QOSEventHandlerBase : public Waitable bool is_ready(rcl_wait_set_t * wait_set) override; + /// Set EventsExecutor's callback + RCLCPP_PUBLIC + void + set_events_executor_callback( + const rclcpp::executors::EventsExecutor * executor, + ExecutorEventCallback executor_callback) const override; + protected: rcl_event_t event_handle_; size_t wait_set_event_index_; @@ -153,25 +160,6 @@ class QOSEventHandler : public QOSEventHandlerBase event_callback_(callback_info); } - /// Set EventsExecutor's callback - RCLCPP_PUBLIC - void - set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const override - { - rcl_ret_t ret = rcl_event_set_events_executor_callback( - executor, - executor_callback, - this, - &event_handle_, - false /* Discard previous events */); - - if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set EventsExecutor's callback to event"); - } - } - private: using EventCallbackInfoT = typename std::remove_reference::template argument_type<0>>::type; diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index a9f30c68cb..3bc6f84616 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -37,7 +37,7 @@ class Waitable RCLCPP_SMART_PTR_DEFINITIONS_NOT_COPYABLE(Waitable) RCLCPP_PUBLIC - virtual ~Waitable(); + virtual ~Waitable() = default; /// Get the number of ready subscriptions /** @@ -182,8 +182,10 @@ class Waitable void set_on_destruction_callback(std::function on_destruction_callback); -private: +protected: std::function on_destruction_callback_; + +private: std::atomic in_use_by_wait_set_{false}; }; // class Waitable diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 7086fef9f4..4afab2395a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include "rclcpp/exceptions/exceptions.hpp" #include "rclcpp/executors/events_executor.hpp" @@ -31,35 +32,22 @@ EventsExecutor::EventsExecutor( timers_manager_ = std::make_shared(context_); entities_collector_ = std::make_shared(this, timers_manager_); - rcl_ret_t ret; - - // Set the global ctrl-c guard condition callback - ret = rcl_guard_condition_set_events_executor_callback( - this, - &EventsExecutor::push_event, - entities_collector_.get(), - options.context->get_interrupt_guard_condition(&wait_set_), - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set ctrl-c guard condition callback"); - } - - // Set the executor interrupt guard condition callback - ret = rcl_guard_condition_set_events_executor_callback( - this, - &EventsExecutor::push_event, - entities_collector_.get(), - &interrupt_guard_condition_, - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set interrupt guard condition callback"); - } + // This API uses the wait_set only as a token to identify different executors. + auto context_interrupt_gc = options.context->get_interrupt_guard_condition(&wait_set_); + + // Setup the executor notifier to wake up the executor when some guard conditions are tiggered. + // The added guard conditions are guaranteed to not go out of scope before the executor itself. + executor_notifier_ = std::make_shared(); + executor_notifier_->add_guard_condition(context_interrupt_gc); + executor_notifier_->add_guard_condition(&interrupt_guard_condition_); + executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); + executor_notifier_->set_on_destruction_callback( + std::bind( + &EventsExecutor::remove_entity, + this, + std::placeholders::_1)); } -EventsExecutor::~EventsExecutor() {} - void EventsExecutor::spin() { @@ -345,7 +333,7 @@ EventsExecutor::remove_callback_group( // This field is unused because we don't have to wake up // the executor when a callback group is removed. (void)notify; - entities_collector_->remove_callback_group(group_ptr); + entities_collector_->remove_callback_group(group_ptr); } std::vector @@ -364,4 +352,4 @@ std::vector EventsExecutor::get_automatically_added_callback_groups_from_nodes() { return entities_collector_->get_automatically_added_callback_groups_from_nodes(); -} \ No newline at end of file +} diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 08d4904a98..8eb7320f5a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -13,6 +13,8 @@ // limitations under the License. #include +#include +#include #include "rclcpp/executors/events_executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" @@ -66,7 +68,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() node_gc, false); - (void)ret; // Can't throw on destructors + (void)ret; // Can't throw on destructors } } @@ -89,8 +91,7 @@ EventsExecutorEntitiesCollector::add_node( } // Get node callback groups, add them to weak_groups_to_nodes_associated_with_executor_ - for (const auto & weak_group : node_ptr->get_callback_groups()) - { + for (const auto & weak_group : node_ptr->get_callback_groups()) { auto group_ptr = weak_group.lock(); if (group_ptr != nullptr && !group_ptr->get_associated_with_executor_atomic().load() && group_ptr->automatically_add_to_executor_with_node()) @@ -359,7 +360,7 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( // Look for the group to remove in the map auto iter = weak_groups_to_nodes.find(weak_group_ptr); - if (iter != weak_groups_to_nodes.end()){ + if (iter != weak_groups_to_nodes.end()) { // Group found, get its associated node. node_ptr = iter->second.lock(); if (node_ptr == nullptr) { @@ -376,7 +377,7 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( !has_node(node_ptr, weak_groups_associated_with_executor_to_nodes_) && !has_node(node_ptr, weak_groups_to_nodes_associated_with_executor_); - if(!node_has_associated_callback_groups) { + if (!node_has_associated_callback_groups) { // Node doesn't have more callback groups associated to the executor. // Unset the event callback for the node's notify guard condition, to stop // receiving events if entities are added or removed to this node. diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 8af3918c48..c4837c09a9 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -35,6 +35,10 @@ UnsupportedEventTypeException::UnsupportedEventTypeException( QOSEventHandlerBase::~QOSEventHandlerBase() { + if (on_destruction_callback_) { + on_destruction_callback_(this); + } + if (rcl_event_fini(&event_handle_) != RCL_RET_OK) { RCUTILS_LOG_ERROR_NAMED( "rclcpp", @@ -68,4 +72,21 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) return wait_set->events[wait_set_event_index_] == &event_handle_; } +void +QOSEventHandlerBase::set_events_executor_callback( + const rclcpp::executors::EventsExecutor * executor, + ExecutorEventCallback executor_callback) const +{ + rcl_ret_t ret = rcl_event_set_events_executor_callback( + executor, + executor_callback, + this, + &event_handle_, + false /* Discard previous events */); + + if (RCL_RET_OK != ret) { + throw std::runtime_error("Couldn't set EventsExecutor's callback in QOSEventHandlerBase"); + } +} + } // namespace rclcpp diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 754573e716..56d91f3283 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -12,12 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include - #include "rclcpp/experimental/subscription_intra_process_base.hpp" using rclcpp::experimental::SubscriptionIntraProcessBase; +SubscriptionIntraProcessBase::~SubscriptionIntraProcessBase() +{ + if (on_destruction_callback_) { + on_destruction_callback_(this); + } +} + bool SubscriptionIntraProcessBase::add_to_wait_set(rcl_wait_set_t * wait_set) { @@ -52,6 +57,6 @@ SubscriptionIntraProcessBase::set_events_executor_callback( true /*Use previous events*/); if (RCL_RET_OK != ret) { - throw std::runtime_error(std::string("Couldn't set guard condition callback")); + throw std::runtime_error("Couldn't set guard condition callback"); } } diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 919af579d5..c1f4d4c66c 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -18,13 +18,6 @@ using rclcpp::Waitable; -Waitable::~Waitable() -{ - if (on_destruction_callback_) { - on_destruction_callback_(this); - } -} - size_t Waitable::get_number_of_ready_subscriptions() { diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index e65064f696..809f2ea2a7 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -416,6 +416,13 @@ class TestWaitable : public rclcpp::Waitable } } + ~TestWaitable() + { + if (on_destruction_callback_) { + on_destruction_callback_(this); + } + } + bool add_to_wait_set(rcl_wait_set_t * wait_set) override { From faf9d1e13fa54bb3333c2d008f1f379d9070b975 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 19 Oct 2020 17:54:46 +0100 Subject: [PATCH 077/168] add comment Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/events_executor_notify_waitable.hpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 5f937078c6..bf4b675065 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -25,6 +25,10 @@ namespace rclcpp namespace executors { +/** + * @brief This class provides an EventWaitable that allows to + * wake up an EventsExecutor when a guard condition is notified. + */ class EventsExecutorNotifyWaitable final : public EventWaitable { public: From e9e237ac3648a73fc5725de3b53869b0a49b6330 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Mon, 19 Oct 2020 18:12:46 +0100 Subject: [PATCH 078/168] add functions for setting/unsetting guard conditions Signed-off-by: Soragna, Alberto --- .../events_executor_entities_collector.hpp | 6 ++ .../events_executor_entities_collector.cpp | 59 +++++++++++-------- 2 files changed, 41 insertions(+), 24 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 6f69ebcaab..a3ffe6237c 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -149,6 +149,12 @@ class EventsExecutorEntitiesCollector final : public EventWaitable void unset_callback_group_entities_callbacks(rclcpp::CallbackGroup::SharedPtr group); + void + set_guard_condition_callback(const rcl_guard_condition_t * guard_condition); + + void + unset_guard_condition_callback(const rcl_guard_condition_t * guard_condition); + /// Return true if the node belongs to the collector /** * \param[in] group_ptr a node base interface shared pointer diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 8eb7320f5a..41d2befa0d 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -63,12 +63,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() auto node = pair.first.lock(); if (node) { auto node_gc = pair.second; - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node_gc, - false); - - (void)ret; // Can't throw on destructors + unset_guard_condition_callback(node_gc); } } @@ -131,16 +126,7 @@ EventsExecutorEntitiesCollector::add_callback_group( if (is_new_node) { // Set an event callback for the node's notify guard condition, so if new entities are added // or removed to this node we will receive an event. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event, - this, - node_ptr->get_notify_guard_condition(), - false /* Discard previous events */); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } + set_guard_condition_callback(node_ptr->get_notify_guard_condition()); // Store node's notify guard condition rclcpp::node_interfaces::NodeBaseInterface::WeakPtr node_weak_ptr(node_ptr); @@ -381,14 +367,7 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( // Node doesn't have more callback groups associated to the executor. // Unset the event callback for the node's notify guard condition, to stop // receiving events if entities are added or removed to this node. - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( - nullptr, nullptr, nullptr, - node_ptr->get_notify_guard_condition(), - false); - - if (ret != RCL_RET_OK) { - throw std::runtime_error("Couldn't set node guard condition callback"); - } + unset_guard_condition_callback(node_ptr->get_notify_guard_condition()); // Remove guard condition from list rclcpp::node_interfaces::NodeBaseInterface::WeakPtr weak_node_ptr(node_ptr); @@ -498,3 +477,35 @@ EventsExecutorEntitiesCollector::get_automatically_added_callback_groups_from_no } return groups; } + +void +EventsExecutorEntitiesCollector::set_guard_condition_callback( + const rcl_guard_condition_t * guard_condition) +{ + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + associated_executor_, + &EventsExecutor::push_event, + this, + guard_condition, + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't set guard condition event callback"); + } +} + +void +EventsExecutorEntitiesCollector::unset_guard_condition_callback( + const rcl_guard_condition_t * guard_condition) +{ + rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + nullptr, + nullptr, + nullptr, + guard_condition, + false /* Discard previous events */); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("Couldn't unset guard condition event callback"); + } +} From 91f1a3bb7041b7fd12dae6299055528ff77f548b Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Tue, 20 Oct 2020 14:33:21 +0100 Subject: [PATCH 079/168] add unit tests for callback groups and fix some issues Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/executor.hpp | 2 +- .../rclcpp/executors/event_waitable.hpp | 9 +- .../rclcpp/executors/events_executor.hpp | 19 +- .../events_executor_entities_collector.hpp | 3 +- .../src/rclcpp/executors/events_executor.cpp | 16 +- .../events_executor_entities_collector.cpp | 84 +++++---- rclcpp/test/CMakeLists.txt | 9 + .../rclcpp/executors/test_events_executor.cpp | 26 +++ ...est_events_executor_entities_collector.cpp | 178 ++++++++++++++++++ .../test/rclcpp/executors/test_executors.cpp | 8 +- .../test_add_callback_groups_to_executor.cpp | 23 ++- 11 files changed, 289 insertions(+), 88 deletions(-) create mode 100644 rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp diff --git a/rclcpp/include/rclcpp/executor.hpp b/rclcpp/include/rclcpp/executor.hpp index 0d0497168f..c9994a49a1 100644 --- a/rclcpp/include/rclcpp/executor.hpp +++ b/rclcpp/include/rclcpp/executor.hpp @@ -385,7 +385,7 @@ class Executor * \throws std::runtime_error if there is an issue triggering the guard condition */ RCLCPP_PUBLIC - virtual void + void cancel(); /// Support dynamic switching of the memory strategy. diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp index 895bf13f50..8f4b4d27ca 100644 --- a/rclcpp/include/rclcpp/executors/event_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -26,8 +26,9 @@ namespace executors * @brief This class provides a wrapper around the waitable object, that is * meant to be used with the EventsExecutor. * The waitset related methods are stubbed out as they should not be called. + * This class is abstract as the execute method of rclcpp::Waitable is not implemented. * Nodes who want to implement a custom EventWaitable, can derive from this class and override - * the execute function. + * the execute method. */ class EventWaitable : public rclcpp::Waitable { @@ -42,12 +43,6 @@ class EventWaitable : public rclcpp::Waitable RCLCPP_PUBLIC virtual ~EventWaitable() = default; - // Executing an EventWaitable is a no-op. - // Derive from this class to implement execute function. - RCLCPP_PUBLIC - virtual void - execute() = 0; - // Stub API: not used by EventsExecutor RCLCPP_PUBLIC bool diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index ead774011b..fd6ec0d50d 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -45,6 +45,8 @@ namespace executors */ class EventsExecutor : public rclcpp::Executor { + friend class EventsExecutorEntitiesCollector; + public: RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutor) @@ -121,15 +123,6 @@ class EventsExecutor : public rclcpp::Executor void remove_node(std::shared_ptr node_ptr, bool notify = true) override; - /// Cancel any running spin* function, causing it to return. - /** - * This function can be called asynchonously from any thread. - * \throws std::runtime_error if there is an issue triggering the guard condition - */ - RCLCPP_PUBLIC - void - cancel() override; - // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. @@ -184,11 +177,15 @@ class EventsExecutor : public rclcpp::Executor } } + /// Add a callback group to an executor. + /** + * \sa rclcpp::Executor::add_callback_group + */ void add_callback_group( rclcpp::CallbackGroup::SharedPtr group_ptr, rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, - bool notify) override; + bool notify = true) override; /// Remove callback group from the executor /** @@ -198,7 +195,7 @@ class EventsExecutor : public rclcpp::Executor void remove_callback_group( rclcpp::CallbackGroup::SharedPtr group_ptr, - bool notify) override; + bool notify = true) override; RCLCPP_PUBLIC std::vector diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index a3ffe6237c..46960edeed 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -56,8 +56,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable // Constructor RCLCPP_PUBLIC EventsExecutorEntitiesCollector( - EventsExecutor * executor_context, - std::shared_ptr timers_manager); + EventsExecutor * executor); // Destructor RCLCPP_PUBLIC diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 4afab2395a..fbe5fdc65f 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -30,7 +30,7 @@ EventsExecutor::EventsExecutor( : rclcpp::Executor(options) { timers_manager_ = std::make_shared(context_); - entities_collector_ = std::make_shared(this, timers_manager_); + entities_collector_ = std::make_shared(this); // This API uses the wait_set only as a token to identify different executors. auto context_interrupt_gc = options.context->get_interrupt_guard_condition(&wait_set_); @@ -251,20 +251,6 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) this->remove_node(node_ptr->get_node_base_interface(), notify); } -void -EventsExecutor::cancel() -{ - spinning.store(false); - rcl_ret_t ret = rcl_trigger_guard_condition(&interrupt_guard_condition_); - if (ret != RCL_RET_OK) { - rclcpp::exceptions::throw_from_rcl_error(ret, "Failed to trigger guard condition in cancel"); - } - - // This makes sure that the timers manager is stopped when we return from this function - // otherwise applications may call rclcpp::shutdown() while that thread is still running. - timers_manager_->stop(); -} - void EventsExecutor::consume_all_events(EventQueue & event_queue) { diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 41d2befa0d..d7bef87097 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -22,11 +22,14 @@ using rclcpp::executors::EventsExecutorEntitiesCollector; EventsExecutorEntitiesCollector::EventsExecutorEntitiesCollector( - EventsExecutor * executor_context, - TimersManager::SharedPtr timers_manager) + EventsExecutor * executor) { - associated_executor_ = executor_context; - timers_manager_ = timers_manager; + if (executor == nullptr) { + throw std::runtime_error("Received NULL executor in EventsExecutorEntitiesCollector."); + } + + associated_executor_ = executor; + timers_manager_ = associated_executor_->timers_manager_; } EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() @@ -80,7 +83,6 @@ EventsExecutorEntitiesCollector::add_node( { // Check if the node already has an executor and if not, set this to true std::atomic_bool & has_executor = node_ptr->get_associated_with_executor_atomic(); - if (has_executor.exchange(true)) { throw std::runtime_error("Node has already been added to an executor."); } @@ -99,7 +101,6 @@ EventsExecutorEntitiesCollector::add_node( weak_nodes_.push_back(node_ptr); } - void EventsExecutorEntitiesCollector::add_callback_group( rclcpp::CallbackGroup::SharedPtr group_ptr, @@ -346,35 +347,36 @@ EventsExecutorEntitiesCollector::remove_callback_group_from_map( // Look for the group to remove in the map auto iter = weak_groups_to_nodes.find(weak_group_ptr); - if (iter != weak_groups_to_nodes.end()) { - // Group found, get its associated node. - node_ptr = iter->second.lock(); - if (node_ptr == nullptr) { - throw std::runtime_error("Node must not be deleted before its callback group(s)."); - } - // Remove group from map - weak_groups_to_nodes.erase(iter); - - // For all the entities in the group, unset their callbacks - unset_callback_group_entities_callbacks(group_ptr); - - // Check if this node still has other callback groups associated with the executor - bool node_has_associated_callback_groups = - !has_node(node_ptr, weak_groups_associated_with_executor_to_nodes_) && - !has_node(node_ptr, weak_groups_to_nodes_associated_with_executor_); - - if (!node_has_associated_callback_groups) { - // Node doesn't have more callback groups associated to the executor. - // Unset the event callback for the node's notify guard condition, to stop - // receiving events if entities are added or removed to this node. - unset_guard_condition_callback(node_ptr->get_notify_guard_condition()); - - // Remove guard condition from list - rclcpp::node_interfaces::NodeBaseInterface::WeakPtr weak_node_ptr(node_ptr); - weak_nodes_to_guard_conditions_.erase(weak_node_ptr); - } - } else { - throw std::runtime_error("Callback group needs to be associated with executor."); + if (iter == weak_groups_to_nodes.end()) { + // Group not found. + throw std::runtime_error("Callback group needs to be associated with this executor."); + } + + // Group found, get its associated node. + node_ptr = iter->second.lock(); + if (node_ptr == nullptr) { + throw std::runtime_error("Node must not be deleted before its callback group(s)."); + } + // Remove group from map + weak_groups_to_nodes.erase(iter); + + // For all the entities in the group, unset their callbacks + unset_callback_group_entities_callbacks(group_ptr); + + // Check if this node still has other callback groups associated with the executor + bool node_has_associated_callback_groups = + has_node(node_ptr, weak_groups_associated_with_executor_to_nodes_) || + has_node(node_ptr, weak_groups_to_nodes_associated_with_executor_); + + if (!node_has_associated_callback_groups) { + // Node doesn't have more callback groups associated to the executor. + // Unset the event callback for the node's notify guard condition, to stop + // receiving events if entities are added or removed to this node. + unset_guard_condition_callback(node_ptr->get_notify_guard_condition()); + + // Remove guard condition from list + rclcpp::node_interfaces::NodeBaseInterface::WeakPtr weak_node_ptr(node_ptr); + weak_nodes_to_guard_conditions_.erase(weak_node_ptr); } } @@ -383,26 +385,27 @@ EventsExecutorEntitiesCollector::remove_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) { if (!node_ptr->get_associated_with_executor_atomic().load()) { - throw std::runtime_error("Node needs to be associated with this executor."); + throw std::runtime_error("Node needs to be associated with an executor."); return; } // Check if this node is currently stored here auto node_it = weak_nodes_.begin(); while (node_it != weak_nodes_.end()) { - if (node_it->lock() == node_ptr) { + bool matched = (node_it->lock() == node_ptr); + if (matched) { weak_nodes_.erase(node_it); break; } + ++node_it; } if (node_it == weak_nodes_.end()) { - // The node is not stored here, so nothing to do - throw std::runtime_error("Tried to remove node not stored in executor."); + // The node is not stored here + throw std::runtime_error("Tried to remove node not stored in this executor."); return; } // Find callback groups belonging to the node to remove std::vector found_group_ptrs; - std::for_each( weak_groups_to_nodes_associated_with_executor_.begin(), weak_groups_to_nodes_associated_with_executor_.end(), @@ -415,7 +418,6 @@ EventsExecutorEntitiesCollector::remove_node( found_group_ptrs.push_back(group_ptr); } }); - // Remove those callback groups std::for_each( found_group_ptrs.begin(), found_group_ptrs.end(), [this] diff --git a/rclcpp/test/CMakeLists.txt b/rclcpp/test/CMakeLists.txt index 939077ace7..5bd1c9db10 100644 --- a/rclcpp/test/CMakeLists.txt +++ b/rclcpp/test/CMakeLists.txt @@ -557,6 +557,15 @@ if(TARGET test_events_executor) target_link_libraries(test_events_executor ${PROJECT_NAME} mimick) endif() +ament_add_gtest(test_events_executor_entities_collector rclcpp/executors/test_events_executor_entities_collector.cpp + APPEND_LIBRARY_DIRS "${append_library_dirs}") +if(TARGET test_events_executor_entities_collector) + ament_target_dependencies(test_events_executor_entities_collector + "rcl" + "test_msgs") + target_link_libraries(test_events_executor_entities_collector ${PROJECT_NAME} mimick) +endif() + ament_add_gtest(test_static_single_threaded_executor rclcpp/executors/test_static_single_threaded_executor.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}") if(TARGET test_static_single_threaded_executor) diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index 7250186e28..e9c3fbd5a1 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -22,9 +22,12 @@ #include "test_msgs/srv/empty.hpp" #include "test_msgs/msg/empty.hpp" +#include "../../mocking_utils/patch.hpp" + using namespace std::chrono_literals; using rclcpp::executors::EventsExecutor; +using rclcpp::executors::EventsExecutorNotifyWaitable; class TestEventsExecutor : public ::testing::Test { @@ -40,6 +43,29 @@ class TestEventsExecutor : public ::testing::Test } }; +TEST_F(TestEventsExecutor, notify_waitable) +{ + auto notifier = std::make_shared(); + + // Waitset methods can't be used on EventsWaitable + rcl_wait_set_t wait_set = rcl_get_zero_initialized_wait_set(); + EXPECT_THROW(notifier->add_to_wait_set(&wait_set), std::runtime_error); + EXPECT_THROW(notifier->is_ready(&wait_set), std::runtime_error); + + EventsExecutor executor; + rcl_guard_condition_t gc = rcl_get_zero_initialized_guard_condition(); + notifier->add_guard_condition(&gc); + { + auto mock = mocking_utils::patch_and_return( + "lib:rclcpp", rcl_guard_condition_set_events_executor_callback, RCL_RET_ERROR); + EXPECT_THROW( + notifier->set_events_executor_callback( + &executor, + &EventsExecutor::push_event), + std::runtime_error); + } +} + TEST_F(TestEventsExecutor, run_clients_servers) { auto node = std::make_shared("node"); diff --git a/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp b/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp new file mode 100644 index 0000000000..47f0cc017d --- /dev/null +++ b/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp @@ -0,0 +1,178 @@ +// Copyright 2020 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include + +#include + +#include "rclcpp/rclcpp.hpp" + +#include "../../mocking_utils/patch.hpp" + +class TestEventsExecutorEntitiesCollector : public ::testing::Test +{ +public: + void SetUp() + { + rclcpp::init(0, nullptr); + dummy_executor_ = std::make_shared(); + entities_collector_ = + std::make_shared(dummy_executor_.get()); + } + + void TearDown() + { + rclcpp::shutdown(); + } + + rclcpp::executors::EventsExecutorEntitiesCollector::SharedPtr entities_collector_; + +private: + rclcpp::executors::EventsExecutor::SharedPtr dummy_executor_; +}; + +TEST_F(TestEventsExecutorEntitiesCollector, bad_init) +{ + EXPECT_THROW( + std::make_shared(nullptr), + std::runtime_error); +} + +TEST_F(TestEventsExecutorEntitiesCollector, add_remove_node) +{ + auto node1 = std::make_shared("node1", "ns"); + EXPECT_NO_THROW(entities_collector_->add_node(node1->get_node_base_interface())); + + // Check adding second time + EXPECT_THROW( + entities_collector_->add_node(node1->get_node_base_interface()), + std::runtime_error); + + auto node2 = std::make_shared("node2", "ns"); + EXPECT_THROW( + entities_collector_->remove_node(node2->get_node_base_interface()), + std::runtime_error); + EXPECT_NO_THROW(entities_collector_->add_node(node2->get_node_base_interface())); + + EXPECT_NO_THROW(entities_collector_->remove_node(node1->get_node_base_interface())); + EXPECT_THROW( + entities_collector_->remove_node(node1->get_node_base_interface()), + std::runtime_error); + EXPECT_NO_THROW(entities_collector_->remove_node(node2->get_node_base_interface())); + + auto node3 = std::make_shared("node3", "ns"); + node3->get_node_base_interface()->get_associated_with_executor_atomic().exchange(true); + EXPECT_THROW( + entities_collector_->remove_node(node3->get_node_base_interface()), + std::runtime_error); +} + +TEST_F(TestEventsExecutorEntitiesCollector, add_callback_group) +{ + auto node = std::make_shared("node1", "ns"); + rclcpp::CallbackGroup::SharedPtr cb_group = node->create_callback_group( + rclcpp::CallbackGroupType::MutuallyExclusive); + + entities_collector_->add_callback_group(cb_group, node->get_node_base_interface()); + ASSERT_EQ(entities_collector_->get_all_callback_groups().size(), 1u); +} + +TEST_F(TestEventsExecutorEntitiesCollector, add_callback_group_after_add_node) +{ + auto node = std::make_shared("node1", "ns"); + rclcpp::CallbackGroup::SharedPtr cb_group = node->create_callback_group( + rclcpp::CallbackGroupType::MutuallyExclusive); + + entities_collector_->add_node(node->get_node_base_interface()); + EXPECT_THROW( + entities_collector_->add_callback_group(cb_group, node->get_node_base_interface()), + std::runtime_error); +} + +TEST_F(TestEventsExecutorEntitiesCollector, add_callback_group_twice) +{ + auto node = std::make_shared("node1", "ns"); + rclcpp::CallbackGroup::SharedPtr cb_group = node->create_callback_group( + rclcpp::CallbackGroupType::MutuallyExclusive); + + entities_collector_->add_callback_group(cb_group, node->get_node_base_interface()); + ASSERT_EQ(entities_collector_->get_all_callback_groups().size(), 1u); + cb_group->get_associated_with_executor_atomic().exchange(false); + EXPECT_THROW( + entities_collector_->add_callback_group(cb_group, node->get_node_base_interface()), + std::runtime_error); +} + +TEST_F(TestEventsExecutorEntitiesCollector, remove_callback_group_after_node) +{ + auto node = std::make_shared("node1", "ns"); + rclcpp::CallbackGroup::SharedPtr cb_group = node->create_callback_group( + rclcpp::CallbackGroupType::MutuallyExclusive); + + entities_collector_->add_callback_group(cb_group, node->get_node_base_interface()); + ASSERT_EQ(entities_collector_->get_all_callback_groups().size(), 1u); + + node.reset(); + + EXPECT_THROW( + entities_collector_->remove_callback_group(cb_group), + std::runtime_error); +} + +TEST_F(TestEventsExecutorEntitiesCollector, remove_callback_group_twice) +{ + auto node = std::make_shared("node1", "ns"); + rclcpp::CallbackGroup::SharedPtr cb_group = node->create_callback_group( + rclcpp::CallbackGroupType::MutuallyExclusive); + + entities_collector_->add_callback_group(cb_group, node->get_node_base_interface()); + ASSERT_EQ(entities_collector_->get_all_callback_groups().size(), 1u); + + entities_collector_->remove_callback_group(cb_group); + + EXPECT_THROW( + entities_collector_->remove_callback_group(cb_group), + std::runtime_error); +} + +TEST_F(TestEventsExecutorEntitiesCollector, remove_node_opposite_order) +{ + auto node1 = std::make_shared("node1", "ns"); + EXPECT_NO_THROW(entities_collector_->add_node(node1->get_node_base_interface())); + + auto node2 = std::make_shared("node2", "ns"); + EXPECT_NO_THROW(entities_collector_->add_node(node2->get_node_base_interface())); + + EXPECT_NO_THROW(entities_collector_->remove_node(node2->get_node_base_interface())); +} + +TEST_F(TestEventsExecutorEntitiesCollector, test_fancy_name) +{ + auto node1 = std::make_shared("node1", "ns"); + auto node2 = std::make_shared("node2", "ns"); + entities_collector_->add_node(node1->get_node_base_interface()); + + { + auto mock = mocking_utils::patch_and_return( + "lib:rclcpp", rcl_guard_condition_set_events_executor_callback, RCL_RET_ERROR); + + EXPECT_THROW( + entities_collector_->add_node(node2->get_node_base_interface()), + std::runtime_error); + + EXPECT_THROW( + entities_collector_->remove_node(node1->get_node_base_interface()), + std::runtime_error); + } +} diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 809f2ea2a7..6c4fa69c62 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -128,8 +128,9 @@ TYPED_TEST_CASE(TestExecutors, ExecutorTypes, ExecutorTypeNames); // https://github.com/ros2/rclcpp/issues/1219 using StandardExecutors = ::testing::Types< - rclcpp::executors::EventsExecutor, - rclcpp::executors::SingleThreadedExecutor>; + rclcpp::executors::SingleThreadedExecutor, + rclcpp::executors::MultiThreadedExecutor, + rclcpp::executors::EventsExecutor>; TYPED_TEST_CASE(TestExecutorsStable, StandardExecutors, ExecutorTypeNames); // Make sure that executors detach from nodes when destructing @@ -148,7 +149,6 @@ TYPED_TEST(TestExecutors, detachOnDestruction) { // Make sure that the executor can automatically remove expired nodes correctly // Currently fails for StaticSingleThreadedExecutor so it is being skipped, see: // https://github.com/ros2/rclcpp/issues/1231 -// This test is also flaky for the MultiThreadedExecutor TYPED_TEST(TestExecutorsStable, addTemporaryNode) { using ExecutorType = TypeParam; ExecutorType executor; @@ -162,7 +162,7 @@ TYPED_TEST(TestExecutorsStable, addTemporaryNode) { // Sleep for a short time to verify executor.spin() is going, and didn't throw. std::thread spinner([&]() {EXPECT_NO_THROW(executor.spin());}); - std::this_thread::sleep_for(50ms); + std::this_thread::sleep_for(200ms); executor.cancel(); spinner.join(); } diff --git a/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp b/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp index deea28d915..c153a91187 100644 --- a/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp +++ b/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp @@ -50,7 +50,8 @@ using ExecutorTypes = ::testing::Types< rclcpp::executors::SingleThreadedExecutor, rclcpp::executors::MultiThreadedExecutor, - rclcpp::executors::StaticSingleThreadedExecutor>; + rclcpp::executors::StaticSingleThreadedExecutor, + rclcpp::executors::EventsExecutor>; class ExecutorTypeNames { @@ -71,6 +72,10 @@ class ExecutorTypeNames return "StaticSingleThreadedExecutor"; } + if (std::is_same()) { + return "EventsExecutor"; + } + return ""; } }; @@ -158,7 +163,8 @@ TYPED_TEST(TestAddCallbackGroupsToExecutor, remove_callback_groups) { */ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_duplicate_callback_groups) { - rclcpp::executors::MultiThreadedExecutor executor; + using ExecutorType = TypeParam; + ExecutorType executor; auto node = std::make_shared("my_node", "/ns"); auto timer_callback = []() {}; rclcpp::CallbackGroup::SharedPtr cb_grp = node->create_callback_group( @@ -176,7 +182,7 @@ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_duplicate_callback_groups) */ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_callback_groups_after_add_node_to_executor) { - rclcpp::executors::MultiThreadedExecutor executor; + rclcpp::executors::EventsExecutor executor; auto node = std::make_shared("my_node", "/ns"); executor.add_node(node->get_node_base_interface()); ASSERT_EQ(executor.get_all_callback_groups().size(), 1u); @@ -210,13 +216,14 @@ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_callback_groups_after_add_node_t */ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_unallowable_callback_groups) { + using ExecutorType = TypeParam; + ExecutorType executor; auto node = std::make_shared("my_node", "/ns"); auto timer_callback = []() {}; rclcpp::CallbackGroup::SharedPtr cb_grp = node->create_callback_group( rclcpp::CallbackGroupType::MutuallyExclusive, false); rclcpp::TimerBase::SharedPtr timer_ = node->create_wall_timer( 2s, timer_callback, cb_grp); - rclcpp::executors::MultiThreadedExecutor executor; executor.add_callback_group(cb_grp, node->get_node_base_interface()); ASSERT_EQ(executor.get_all_callback_groups().size(), 1u); @@ -245,14 +252,15 @@ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_unallowable_callback_groups) */ TYPED_TEST(TestAddCallbackGroupsToExecutor, one_node_many_callback_groups_many_executors) { + using ExecutorType = TypeParam; auto node = std::make_shared("my_node", "/ns"); auto timer_callback = []() {}; rclcpp::CallbackGroup::SharedPtr cb_grp = node->create_callback_group( rclcpp::CallbackGroupType::MutuallyExclusive, false); rclcpp::TimerBase::SharedPtr timer_ = node->create_wall_timer( 2s, timer_callback, cb_grp); - rclcpp::executors::MultiThreadedExecutor timer_executor; - rclcpp::executors::MultiThreadedExecutor sub_executor; + ExecutorType timer_executor; + ExecutorType sub_executor; timer_executor.add_callback_group(cb_grp, node->get_node_base_interface()); const rclcpp::QoS qos(10); auto options = rclcpp::SubscriptionOptions(); @@ -281,7 +289,8 @@ TYPED_TEST(TestAddCallbackGroupsToExecutor, one_node_many_callback_groups_many_e */ TYPED_TEST(TestAddCallbackGroupsToExecutor, remove_callback_group) { - rclcpp::executors::MultiThreadedExecutor executor; + using ExecutorType = TypeParam; + ExecutorType executor; auto node = std::make_shared("my_node", "/ns"); auto timer_callback = []() {}; rclcpp::CallbackGroup::SharedPtr cb_grp = node->create_callback_group( From 41cc6c0e5b1cd06cef23a25f401885c856d478f0 Mon Sep 17 00:00:00 2001 From: Mauro Date: Tue, 20 Oct 2020 15:42:11 +0100 Subject: [PATCH 080/168] Rename ExecutorEventCallback -> EventsExecutorCallback --- rclcpp/include/rclcpp/client.hpp | 2 +- .../rclcpp/executors/events_executor_notify_waitable.hpp | 2 +- .../rclcpp/experimental/subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/qos_event.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- rclcpp/src/rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- 14 files changed, 14 insertions(+), 14 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 5036005219..fc6c846d2a 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -160,7 +160,7 @@ class ClientBase void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const; + EventsExecutorCallback executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index bf4b675065..7d73c941c9 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -64,7 +64,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const override + EventsExecutorCallback executor_callback) const override { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 9d5af14b8d..d8be34cc74 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -74,7 +74,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const override; + EventsExecutorCallback executor_callback) const override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 8ff2cfc2fa..b21d58ae2c 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -110,7 +110,7 @@ class QOSEventHandlerBase : public Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const override; + EventsExecutorCallback executor_callback) const override; protected: rcl_event_t event_handle_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index c588f0a98a..5a3590cc43 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -130,7 +130,7 @@ class ServiceBase void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const; + EventsExecutorCallback executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 90df0f1cb5..ccdb57f77c 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -273,7 +273,7 @@ class SubscriptionBase : public std::enable_shared_from_this void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const; + EventsExecutorCallback executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 3bc6f84616..d1e2eb622c 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -176,7 +176,7 @@ class Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const; + EventsExecutorCallback executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 0534ba6bdc..5484597689 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -205,7 +205,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const + EventsExecutorCallback executor_callback) const { rcl_ret_t ret = rcl_client_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index c4837c09a9..910c6cb7ff 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -75,7 +75,7 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) void QOSEventHandlerBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const + EventsExecutorCallback executor_callback) const { rcl_ret_t ret = rcl_event_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 639de9858d..ca1e2e4db3 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -92,7 +92,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const + EventsExecutorCallback executor_callback) const { rcl_ret_t ret = rcl_service_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 4765adbfc6..099f376618 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -296,7 +296,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const + EventsExecutorCallback executor_callback) const { rcl_ret_t ret = rcl_subscription_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 56d91f3283..a3053b8e6d 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -47,7 +47,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const + EventsExecutorCallback executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index c1f4d4c66c..6f730f13c6 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -63,7 +63,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const + EventsExecutorCallback executor_callback) const { (void)executor; (void)executor_callback; diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 6c4fa69c62..1bebb4795c 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -462,7 +462,7 @@ class TestWaitable : public rclcpp::Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - ExecutorEventCallback executor_callback) const override + EventsExecutorCallback executor_callback) const override { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor, From 6f13d4a94368624a75c42a4df720f20811a38db6 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Tue, 20 Oct 2020 15:59:58 +0100 Subject: [PATCH 081/168] make functions private Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/events_executor.hpp | 104 +++++++++--------- .../rclcpp/executors/test_events_executor.cpp | 8 ++ 2 files changed, 60 insertions(+), 52 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index fd6ec0d50d..da903a188a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -123,6 +123,58 @@ class EventsExecutor : public rclcpp::Executor void remove_node(std::shared_ptr node_ptr, bool notify = true) override; + /// Add a callback group to an executor. + /** + * \sa rclcpp::Executor::add_callback_group + */ + void + add_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, + bool notify = true) override; + + /// Remove callback group from the executor + /** + * \sa rclcpp::Executor::remove_callback_group + */ + RCLCPP_PUBLIC + void + remove_callback_group( + rclcpp::CallbackGroup::SharedPtr group_ptr, + bool notify = true) override; + + RCLCPP_PUBLIC + std::vector + get_all_callback_groups() override; + + /// Get callback groups that belong to executor. + /** + * \sa rclcpp::Executor::get_manually_added_callback_groups() + */ + RCLCPP_PUBLIC + std::vector + get_manually_added_callback_groups() override; + + /// Get callback groups that belong to executor. + /** + * \sa rclcpp::Executor::get_automatically_added_callback_groups_from_nodes() + */ + RCLCPP_PUBLIC + std::vector + get_automatically_added_callback_groups_from_nodes() override; + +protected: + RCLCPP_PUBLIC + void + spin_once_impl(std::chrono::nanoseconds timeout) override; + +private: + RCLCPP_DISABLE_COPY(EventsExecutor) + + // Event queue implementation is a deque only to + // facilitate the removal of events from expired entities. + using EventQueue = std::deque; + // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. @@ -177,58 +229,6 @@ class EventsExecutor : public rclcpp::Executor } } - /// Add a callback group to an executor. - /** - * \sa rclcpp::Executor::add_callback_group - */ - void - add_callback_group( - rclcpp::CallbackGroup::SharedPtr group_ptr, - rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr, - bool notify = true) override; - - /// Remove callback group from the executor - /** - * \sa rclcpp::Executor::remove_callback_group - */ - RCLCPP_PUBLIC - void - remove_callback_group( - rclcpp::CallbackGroup::SharedPtr group_ptr, - bool notify = true) override; - - RCLCPP_PUBLIC - std::vector - get_all_callback_groups() override; - - /// Get callback groups that belong to executor. - /** - * \sa rclcpp::Executor::get_manually_added_callback_groups() - */ - RCLCPP_PUBLIC - std::vector - get_manually_added_callback_groups() override; - - /// Get callback groups that belong to executor. - /** - * \sa rclcpp::Executor::get_automatically_added_callback_groups_from_nodes() - */ - RCLCPP_PUBLIC - std::vector - get_automatically_added_callback_groups_from_nodes() override; - -protected: - RCLCPP_PUBLIC - void - spin_once_impl(std::chrono::nanoseconds timeout) override; - -private: - RCLCPP_DISABLE_COPY(EventsExecutor) - - // Event queue implementation is a deque only to - // facilitate the removal of events from expired entities. - using EventQueue = std::deque; - /// Extract and execute events from the queue until it is empty RCLCPP_PUBLIC void diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index e9c3fbd5a1..3a2a892cc3 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -52,6 +52,13 @@ TEST_F(TestEventsExecutor, notify_waitable) EXPECT_THROW(notifier->add_to_wait_set(&wait_set), std::runtime_error); EXPECT_THROW(notifier->is_ready(&wait_set), std::runtime_error); + { + auto mock = mocking_utils::patch_and_return( + "lib:rclcpp", rcl_guard_condition_set_events_executor_callback, RCL_RET_ERROR); + EXPECT_THROW(std::make_shared(), std::runtime_error); + } + + /* EventsExecutor executor; rcl_guard_condition_t gc = rcl_get_zero_initialized_guard_condition(); notifier->add_guard_condition(&gc); @@ -64,6 +71,7 @@ TEST_F(TestEventsExecutor, notify_waitable) &EventsExecutor::push_event), std::runtime_error); } + */ } TEST_F(TestEventsExecutor, run_clients_servers) From 4df706dd80a3930a6cb20b2ee04f741d6ee538e7 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Tue, 20 Oct 2020 16:04:03 +0100 Subject: [PATCH 082/168] remove commented code Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/test_events_executor.cpp | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index 3a2a892cc3..93258e3d83 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -57,21 +57,6 @@ TEST_F(TestEventsExecutor, notify_waitable) "lib:rclcpp", rcl_guard_condition_set_events_executor_callback, RCL_RET_ERROR); EXPECT_THROW(std::make_shared(), std::runtime_error); } - - /* - EventsExecutor executor; - rcl_guard_condition_t gc = rcl_get_zero_initialized_guard_condition(); - notifier->add_guard_condition(&gc); - { - auto mock = mocking_utils::patch_and_return( - "lib:rclcpp", rcl_guard_condition_set_events_executor_callback, RCL_RET_ERROR); - EXPECT_THROW( - notifier->set_events_executor_callback( - &executor, - &EventsExecutor::push_event), - std::runtime_error); - } - */ } TEST_F(TestEventsExecutor, run_clients_servers) From 7d6a1e82e2da2176c370d8bd86336f8c7ef1bdcb Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Wed, 21 Oct 2020 12:18:13 +0100 Subject: [PATCH 083/168] fix comments Signed-off-by: Soragna, Alberto --- rclcpp/include/rclcpp/executors/events_executor.hpp | 5 ++++- .../rclcpp/executors/events_executor_notify_waitable.hpp | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index da903a188a..66e11c4cf4 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -35,7 +35,10 @@ namespace executors /// Events executor implementation /** - * Add description + * This executor is a variation of the standard one that does not uses a waitset. + * The executor uses an events queue and a timers manager to execute entities from its + * associated nodes and callback groups. + * This provides improved performance as it allows to skip all the waitset maintenance operations. * * To run this executor: * rclcpp::executors::EventsExecutor executor; diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 7d73c941c9..5b2398c13d 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -75,7 +75,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable false); if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set guard condition callback"); + throw std::runtime_error("Couldn't set guard condition events callback"); } } } From f6a97c53ca273f07835d938747cc8e45f8613961 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 13 Nov 2020 14:15:03 +0000 Subject: [PATCH 084/168] Add spin_some default arg --- rclcpp/include/rclcpp/executors/events_executor.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 66e11c4cf4..3525ce5583 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -84,7 +84,7 @@ class EventsExecutor : public rclcpp::Executor */ RCLCPP_PUBLIC void - spin_some(std::chrono::nanoseconds max_duration) override; + spin_some(std::chrono::nanoseconds max_duration = std::chrono::nanoseconds(0)) override; RCLCPP_PUBLIC void From d1d940e6d196877127da6e46c8aa7cb1d8719aa2 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 19 Nov 2020 18:00:20 +0000 Subject: [PATCH 085/168] rename local_event_queue->execution_event_queue --- .../include/rclcpp/executors/events_executor.hpp | 8 ++++---- rclcpp/src/rclcpp/executors/events_executor.cpp | 14 +++++++------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 3525ce5583..f5d4c5453f 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -223,12 +223,12 @@ class EventsExecutor : public rclcpp::Executor // Remove events associated with this entity from the local event queue { std::unique_lock lock(execution_mutex_); - local_event_queue_.erase( + execution_event_queue_.erase( std::remove_if( - local_event_queue_.begin(), local_event_queue_.end(), + execution_event_queue_.begin(), execution_event_queue_.end(), [&entity](ExecutorEvent event) { return event.entity == entity; - }), local_event_queue_.end()); + }), execution_event_queue_.end()); } } @@ -244,7 +244,7 @@ class EventsExecutor : public rclcpp::Executor // We use two instances of EventQueue to allow threads to push events while we execute them EventQueue event_queue_; - EventQueue local_event_queue_; + EventQueue execution_event_queue_; EventsExecutorEntitiesCollector::SharedPtr entities_collector_; EventsExecutorNotifyWaitable::SharedPtr executor_notifier_; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index fbe5fdc65f..9934a258bd 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -67,11 +67,11 @@ EventsExecutor::spin() event_queue_cv_.wait(push_lock, has_event_predicate); std::unique_lock execution_lock(execution_mutex_); // We got an event! Swap queues while we hold both mutexes - std::swap(local_event_queue_, event_queue_); + std::swap(execution_event_queue_, event_queue_); // After swapping the queues, we don't need the push lock anymore push_lock.unlock(); // Consume events while under the execution lock only - this->consume_all_events(local_event_queue_); + this->consume_all_events(execution_event_queue_); } timers_manager_->stop(); } @@ -108,9 +108,9 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) std::unique_lock push_lock(push_mutex_); event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); std::unique_lock execution_lock(execution_mutex_); - std::swap(local_event_queue_, event_queue_); + std::swap(execution_event_queue_, event_queue_); push_lock.unlock(); - this->consume_all_events(local_event_queue_); + this->consume_all_events(execution_event_queue_); execution_lock.unlock(); timers_manager_->execute_ready_timers(); @@ -154,18 +154,18 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { std::unique_lock push_lock(push_mutex_); std::unique_lock execution_lock(execution_mutex_); - std::swap(local_event_queue_, event_queue_); + std::swap(execution_event_queue_, event_queue_); push_lock.unlock(); bool ready_timer = timers_manager_->get_head_timeout() < 0ns; - bool has_events = !local_event_queue_.empty(); + bool has_events = !execution_event_queue_.empty(); if (!ready_timer && !has_events) { // Exit as there is no more work to do break; } // Execute all ready work - this->consume_all_events(local_event_queue_); + this->consume_all_events(execution_event_queue_); execution_lock.unlock(); timers_manager_->execute_ready_timers(); From 7ddd6d1e475d2c6f1613e1e3c7cdbf8e1b4004cf Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 20 Nov 2020 12:26:59 +0000 Subject: [PATCH 086/168] Move rcutils/executor_event_types.h to rmw/ --- rclcpp/include/rclcpp/executors/events_executor.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 3525ce5583..339aae4bc6 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -20,14 +20,14 @@ #include #include -#include "rcutils/executor_event_types.h" - #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" #include "rclcpp/executors/events_executor_notify_waitable.hpp" #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node.hpp" +#include "rmw/executor_event_types.h" + namespace rclcpp { namespace executors From fb7c97c936ec0342f255e2398155f2a48ce0ba3f Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 20 Nov 2020 15:35:52 +0000 Subject: [PATCH 087/168] rename event types --- rclcpp/include/rclcpp/client.hpp | 2 +- rclcpp/include/rclcpp/executors/events_executor.hpp | 12 ++++++------ .../executors/events_executor_notify_waitable.hpp | 2 +- .../experimental/subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/executors/events_executor.cpp | 6 +++--- rclcpp/src/rclcpp/qos_event.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- .../src/rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- 16 files changed, 23 insertions(+), 23 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index fc6c846d2a..080c8f9c08 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -160,7 +160,7 @@ class ClientBase void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const; + rmw_listener_cb_t executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index a0ec73d9b7..1c78f79728 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -26,7 +26,7 @@ #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node.hpp" -#include "rmw/executor_event_types.h" +#include "rmw/listener_event_types.h" namespace rclcpp { @@ -176,13 +176,13 @@ class EventsExecutor : public rclcpp::Executor // Event queue implementation is a deque only to // facilitate the removal of events from expired entities. - using EventQueue = std::deque; + using EventQueue = std::deque; // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. static void - push_event(const void * executor_ptr, ExecutorEvent event) + push_event(const void * executor_ptr, rmw_listener_event_t event) { // Cast executor_ptr to this (need to remove constness) auto this_executor = const_cast( @@ -217,7 +217,7 @@ class EventsExecutor : public rclcpp::Executor event_queue_.erase( std::remove_if( event_queue_.begin(), event_queue_.end(), - [&entity](ExecutorEvent event) {return event.entity == entity;}), event_queue_.end()); + [&entity](rmw_listener_event_t event) {return event.entity == entity;}), event_queue_.end()); } // Remove events associated with this entity from the local event queue @@ -226,7 +226,7 @@ class EventsExecutor : public rclcpp::Executor execution_event_queue_.erase( std::remove_if( execution_event_queue_.begin(), execution_event_queue_.end(), - [&entity](ExecutorEvent event) { + [&entity](rmw_listener_event_t event) { return event.entity == entity; }), execution_event_queue_.end()); } @@ -240,7 +240,7 @@ class EventsExecutor : public rclcpp::Executor // Execute a single event RCLCPP_PUBLIC void - execute_event(const ExecutorEvent & event); + execute_event(const rmw_listener_event_t & event); // We use two instances of EventQueue to allow threads to push events while we execute them EventQueue event_queue_; diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 5b2398c13d..d038d5bb0f 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -64,7 +64,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const override + rmw_listener_cb_t executor_callback) const override { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index d8be34cc74..c5d2e33455 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -74,7 +74,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const override; + rmw_listener_cb_t executor_callback) const override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index b21d58ae2c..f43e0712cb 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -110,7 +110,7 @@ class QOSEventHandlerBase : public Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const override; + rmw_listener_cb_t executor_callback) const override; protected: rcl_event_t event_handle_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 5a3590cc43..9e33c05374 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -130,7 +130,7 @@ class ServiceBase void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const; + rmw_listener_cb_t executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index ccdb57f77c..f9a591dbd1 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -273,7 +273,7 @@ class SubscriptionBase : public std::enable_shared_from_this void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const; + rmw_listener_cb_t executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index d1e2eb622c..8e45533754 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -176,7 +176,7 @@ class Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const; + rmw_listener_cb_t executor_callback) const; RCLCPP_PUBLIC void diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 5484597689..2ecf8e072d 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -205,7 +205,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const + rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_client_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 9934a258bd..0ac56f6e34 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -189,7 +189,7 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !event_queue_.empty();}; - ExecutorEvent event; + rmw_listener_event_t event; bool has_event = false; { @@ -255,7 +255,7 @@ void EventsExecutor::consume_all_events(EventQueue & event_queue) { while (!event_queue.empty()) { - ExecutorEvent event = event_queue.front(); + rmw_listener_event_t event = event_queue.front(); event_queue.pop_front(); this->execute_event(event); @@ -263,7 +263,7 @@ EventsExecutor::consume_all_events(EventQueue & event_queue) } void -EventsExecutor::execute_event(const ExecutorEvent & event) +EventsExecutor::execute_event(const rmw_listener_event_t & event) { switch (event.type) { case SUBSCRIPTION_EVENT: diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 910c6cb7ff..c8725220be 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -75,7 +75,7 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) void QOSEventHandlerBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const + rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_event_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index ca1e2e4db3..0a39943488 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -92,7 +92,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const + rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_service_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 099f376618..74e19dc43b 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -296,7 +296,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const + rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_subscription_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index a3053b8e6d..e89d332c45 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -47,7 +47,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const + rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 6f730f13c6..1df8e2090f 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -63,7 +63,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const + rmw_listener_cb_t executor_callback) const { (void)executor; (void)executor_callback; diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 355e8f60f7..e7af9f527e 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -467,7 +467,7 @@ class TestWaitable : public rclcpp::Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - EventsExecutorCallback executor_callback) const override + rmw_listener_cb_t executor_callback) const override { rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( executor, From 3318c2d666d19edaed249e856b21f05c90769c36 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 20 Nov 2020 15:46:47 +0000 Subject: [PATCH 088/168] rename set_events_executor_callback to set_listener_callback --- rclcpp/include/rclcpp/client.hpp | 2 +- .../rclcpp/executors/events_executor.hpp | 2 +- .../events_executor_notify_waitable.hpp | 4 ++-- .../subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 4 ++-- .../src/rclcpp/executors/events_executor.cpp | 2 +- .../events_executor_entities_collector.cpp | 20 +++++++++---------- rclcpp/src/rclcpp/qos_event.cpp | 4 ++-- rclcpp/src/rclcpp/service.cpp | 4 ++-- rclcpp/src/rclcpp/subscription_base.cpp | 4 ++-- .../subscription_intra_process_base.cpp | 4 ++-- rclcpp/src/rclcpp/waitable.cpp | 4 ++-- .../rclcpp/executors/test_events_executor.cpp | 2 +- ...est_events_executor_entities_collector.cpp | 2 +- .../test/rclcpp/executors/test_executors.cpp | 4 ++-- 19 files changed, 36 insertions(+), 36 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 080c8f9c08..e0d532869e 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -158,7 +158,7 @@ class ClientBase RCLCPP_PUBLIC void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 1c78f79728..3f9c1d5045 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -209,7 +209,7 @@ class EventsExecutor : public rclcpp::Executor // We need to unset the callbacks to make sure that after removing events from the // queues, this entity will not push anymore before being completely destroyed. // This assumes that all supported entities implement this function. - entity->set_events_executor_callback(nullptr, nullptr); + entity->set_listener_callback(nullptr, nullptr); // Remove events associated with this entity from the event queue { diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index d038d5bb0f..95f9d5853b 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -62,12 +62,12 @@ class EventsExecutorNotifyWaitable final : public EventWaitable RCLCPP_PUBLIC void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override { for (auto gc : notify_guard_conditions_) { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_listener_callback( executor, executor_callback, this, diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index c5d2e33455..1a980c635f 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -72,7 +72,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index f43e0712cb..da9d8cd846 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -108,7 +108,7 @@ class QOSEventHandlerBase : public Waitable /// Set EventsExecutor's callback RCLCPP_PUBLIC void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 9e33c05374..cd24a44c59 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -128,7 +128,7 @@ class ServiceBase RCLCPP_PUBLIC void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index f9a591dbd1..398134bb7b 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -271,7 +271,7 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 8e45533754..19c78d7513 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -174,7 +174,7 @@ class Waitable RCLCPP_PUBLIC virtual void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 2ecf8e072d..7f3726d118 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -203,11 +203,11 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ClientBase::set_events_executor_callback( +ClientBase::set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { - rcl_ret_t ret = rcl_client_set_events_executor_callback( + rcl_ret_t ret = rcl_client_set_listener_callback( executor, executor_callback, this, diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 0ac56f6e34..2bdce8a3dc 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -40,7 +40,7 @@ EventsExecutor::EventsExecutor( executor_notifier_ = std::make_shared(); executor_notifier_->add_guard_condition(context_interrupt_gc); executor_notifier_->add_guard_condition(&interrupt_guard_condition_); - executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); + executor_notifier_->set_listener_callback(this, &EventsExecutor::push_event); executor_notifier_->set_on_destruction_callback( std::bind( &EventsExecutor::remove_entity, diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index d7bef87097..24bf0dc180 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -224,7 +224,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_events_executor_callback( + subscription->set_listener_callback( associated_executor_, &EventsExecutor::push_event); subscription->set_on_destruction_callback( @@ -238,7 +238,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_events_executor_callback( + service->set_listener_callback( associated_executor_, &EventsExecutor::push_event); service->set_on_destruction_callback( @@ -252,7 +252,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_events_executor_callback( + client->set_listener_callback( associated_executor_, &EventsExecutor::push_event); client->set_on_destruction_callback( @@ -266,7 +266,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_events_executor_callback( + waitable->set_listener_callback( associated_executor_, &EventsExecutor::push_event); waitable->set_on_destruction_callback( @@ -297,7 +297,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_events_executor_callback(nullptr, nullptr); + subscription->set_listener_callback(nullptr, nullptr); subscription->set_on_destruction_callback(nullptr); } return false; @@ -305,7 +305,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_events_executor_callback(nullptr, nullptr); + service->set_listener_callback(nullptr, nullptr); service->set_on_destruction_callback(nullptr); } return false; @@ -313,7 +313,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_events_executor_callback(nullptr, nullptr); + client->set_listener_callback(nullptr, nullptr); client->set_on_destruction_callback(nullptr); } return false; @@ -321,7 +321,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_events_executor_callback(nullptr, nullptr); + waitable->set_listener_callback(nullptr, nullptr); waitable->set_on_destruction_callback(nullptr); } return false; @@ -484,7 +484,7 @@ void EventsExecutorEntitiesCollector::set_guard_condition_callback( const rcl_guard_condition_t * guard_condition) { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_listener_callback( associated_executor_, &EventsExecutor::push_event, this, @@ -500,7 +500,7 @@ void EventsExecutorEntitiesCollector::unset_guard_condition_callback( const rcl_guard_condition_t * guard_condition) { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_listener_callback( nullptr, nullptr, nullptr, diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index c8725220be..5d114c7394 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -73,11 +73,11 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) } void -QOSEventHandlerBase::set_events_executor_callback( +QOSEventHandlerBase::set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { - rcl_ret_t ret = rcl_event_set_events_executor_callback( + rcl_ret_t ret = rcl_event_set_listener_callback( executor, executor_callback, this, diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 0a39943488..64e24af5d4 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -90,11 +90,11 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ServiceBase::set_events_executor_callback( +ServiceBase::set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { - rcl_ret_t ret = rcl_service_set_events_executor_callback( + rcl_ret_t ret = rcl_service_set_listener_callback( executor, executor_callback, this, diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 74e19dc43b..bb37b901a7 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -294,11 +294,11 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( } void -SubscriptionBase::set_events_executor_callback( +SubscriptionBase::set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { - rcl_ret_t ret = rcl_subscription_set_events_executor_callback( + rcl_ret_t ret = rcl_subscription_set_listener_callback( executor, executor_callback, this, diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index e89d332c45..e3287a3cb0 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -45,11 +45,11 @@ SubscriptionIntraProcessBase::get_actual_qos() const } void -SubscriptionIntraProcessBase::set_events_executor_callback( +SubscriptionIntraProcessBase::set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_listener_callback( executor, executor_callback, this, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 1df8e2090f..2f7522ed32 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -61,7 +61,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) } void -Waitable::set_events_executor_callback( +Waitable::set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { @@ -69,7 +69,7 @@ Waitable::set_events_executor_callback( (void)executor_callback; throw std::runtime_error( - "Custom waitables should override set_events_executor_callback() to use events executor"); + "Custom waitables should override set_listener_callback() to use events executor"); } void diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index 93258e3d83..8fad5b40b0 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -54,7 +54,7 @@ TEST_F(TestEventsExecutor, notify_waitable) { auto mock = mocking_utils::patch_and_return( - "lib:rclcpp", rcl_guard_condition_set_events_executor_callback, RCL_RET_ERROR); + "lib:rclcpp", rcl_guard_condition_set_listener_callback, RCL_RET_ERROR); EXPECT_THROW(std::make_shared(), std::runtime_error); } } diff --git a/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp b/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp index 47f0cc017d..1860ffd2c1 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp @@ -165,7 +165,7 @@ TEST_F(TestEventsExecutorEntitiesCollector, test_fancy_name) { auto mock = mocking_utils::patch_and_return( - "lib:rclcpp", rcl_guard_condition_set_events_executor_callback, RCL_RET_ERROR); + "lib:rclcpp", rcl_guard_condition_set_listener_callback, RCL_RET_ERROR); EXPECT_THROW( entities_collector_->add_node(node2->get_node_base_interface()), diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index e7af9f527e..23c1b1700a 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -465,11 +465,11 @@ class TestWaitable : public rclcpp::Waitable } void - set_events_executor_callback( + set_listener_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override { - rcl_ret_t ret = rcl_guard_condition_set_events_executor_callback( + rcl_ret_t ret = rcl_guard_condition_set_listener_callback( executor, executor_callback, this, From 0a5f3f27bc35fb1a02ce0801d5cd3428f49af9b9 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 20 Nov 2020 17:25:17 +0000 Subject: [PATCH 089/168] Revert. set_listener_callback->set_events_executor_callback --- rclcpp/include/rclcpp/client.hpp | 2 +- .../include/rclcpp/executors/events_executor.hpp | 2 +- .../events_executor_notify_waitable.hpp | 2 +- .../subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 +- .../events_executor_entities_collector.cpp | 16 ++++++++-------- rclcpp/src/rclcpp/qos_event.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- .../rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 4 ++-- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- 17 files changed, 25 insertions(+), 25 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index e0d532869e..080c8f9c08 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -158,7 +158,7 @@ class ClientBase RCLCPP_PUBLIC void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 3f9c1d5045..1c78f79728 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -209,7 +209,7 @@ class EventsExecutor : public rclcpp::Executor // We need to unset the callbacks to make sure that after removing events from the // queues, this entity will not push anymore before being completely destroyed. // This assumes that all supported entities implement this function. - entity->set_listener_callback(nullptr, nullptr); + entity->set_events_executor_callback(nullptr, nullptr); // Remove events associated with this entity from the event queue { diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 95f9d5853b..ce67004ca5 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -62,7 +62,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable RCLCPP_PUBLIC void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override { diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 1a980c635f..c5d2e33455 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -72,7 +72,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index da9d8cd846..f43e0712cb 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -108,7 +108,7 @@ class QOSEventHandlerBase : public Waitable /// Set EventsExecutor's callback RCLCPP_PUBLIC void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index cd24a44c59..9e33c05374 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -128,7 +128,7 @@ class ServiceBase RCLCPP_PUBLIC void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 398134bb7b..f9a591dbd1 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -271,7 +271,7 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 19c78d7513..8e45533754 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -174,7 +174,7 @@ class Waitable RCLCPP_PUBLIC virtual void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 7f3726d118..55fdbca3b4 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -203,7 +203,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ClientBase::set_listener_callback( +ClientBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 2bdce8a3dc..0ac56f6e34 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -40,7 +40,7 @@ EventsExecutor::EventsExecutor( executor_notifier_ = std::make_shared(); executor_notifier_->add_guard_condition(context_interrupt_gc); executor_notifier_->add_guard_condition(&interrupt_guard_condition_); - executor_notifier_->set_listener_callback(this, &EventsExecutor::push_event); + executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); executor_notifier_->set_on_destruction_callback( std::bind( &EventsExecutor::remove_entity, diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 24bf0dc180..1b214b27d8 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -224,7 +224,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_listener_callback( + subscription->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); subscription->set_on_destruction_callback( @@ -238,7 +238,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_listener_callback( + service->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); service->set_on_destruction_callback( @@ -252,7 +252,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_listener_callback( + client->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); client->set_on_destruction_callback( @@ -266,7 +266,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_listener_callback( + waitable->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); waitable->set_on_destruction_callback( @@ -297,7 +297,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_listener_callback(nullptr, nullptr); + subscription->set_events_executor_callback(nullptr, nullptr); subscription->set_on_destruction_callback(nullptr); } return false; @@ -305,7 +305,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_listener_callback(nullptr, nullptr); + service->set_events_executor_callback(nullptr, nullptr); service->set_on_destruction_callback(nullptr); } return false; @@ -313,7 +313,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_listener_callback(nullptr, nullptr); + client->set_events_executor_callback(nullptr, nullptr); client->set_on_destruction_callback(nullptr); } return false; @@ -321,7 +321,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_listener_callback(nullptr, nullptr); + waitable->set_events_executor_callback(nullptr, nullptr); waitable->set_on_destruction_callback(nullptr); } return false; diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 5d114c7394..4e14f1bc13 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -73,7 +73,7 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) } void -QOSEventHandlerBase::set_listener_callback( +QOSEventHandlerBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 64e24af5d4..f4e4092962 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -90,7 +90,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ServiceBase::set_listener_callback( +ServiceBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index bb37b901a7..81b6c91646 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -294,7 +294,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( } void -SubscriptionBase::set_listener_callback( +SubscriptionBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index e3287a3cb0..4a2a4ccc3d 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -45,7 +45,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const } void -SubscriptionIntraProcessBase::set_listener_callback( +SubscriptionIntraProcessBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 2f7522ed32..1df8e2090f 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -61,7 +61,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) } void -Waitable::set_listener_callback( +Waitable::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const { @@ -69,7 +69,7 @@ Waitable::set_listener_callback( (void)executor_callback; throw std::runtime_error( - "Custom waitables should override set_listener_callback() to use events executor"); + "Custom waitables should override set_events_executor_callback() to use events executor"); } void diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 23c1b1700a..f6a66768c7 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -465,7 +465,7 @@ class TestWaitable : public rclcpp::Waitable } void - set_listener_callback( + set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const override { From fe96d7439870cdae4a9e66f61fd41a708e9b5932 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Tue, 15 Dec 2020 09:27:21 -0300 Subject: [PATCH 090/168] Add vectors of entities weak pointers --- rclcpp/include/rclcpp/client.hpp | 6 - rclcpp/include/rclcpp/executor.hpp | 12 -- .../rclcpp/executors/events_executor.hpp | 34 ---- .../events_executor_entities_collector.hpp | 53 +++++++ .../events_executor_notify_waitable.hpp | 7 +- rclcpp/include/rclcpp/service.hpp | 6 - rclcpp/include/rclcpp/subscription_base.hpp | 6 - rclcpp/include/rclcpp/waitable.hpp | 7 - rclcpp/src/rclcpp/client.cpp | 10 -- rclcpp/src/rclcpp/executor.cpp | 18 --- .../src/rclcpp/executors/events_executor.cpp | 51 ++++-- .../events_executor_entities_collector.cpp | 145 ++++++++++++++---- rclcpp/src/rclcpp/qos_event.cpp | 4 - rclcpp/src/rclcpp/service.cpp | 13 +- rclcpp/src/rclcpp/subscription_base.cpp | 11 -- .../subscription_intra_process_base.cpp | 7 - rclcpp/src/rclcpp/waitable.cpp | 7 - .../test/rclcpp/executors/test_executors.cpp | 4 - 18 files changed, 207 insertions(+), 194 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 080c8f9c08..44b2724b7b 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -162,10 +162,6 @@ class ClientBase const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; - RCLCPP_PUBLIC - void - set_on_destruction_callback(std::function on_destruction_callback); - protected: RCLCPP_DISABLE_COPY(ClientBase) @@ -181,8 +177,6 @@ class ClientBase const rcl_node_t * get_rcl_node_handle() const; - std::function on_destruction_callback_; - rclcpp::node_interfaces::NodeGraphInterface::WeakPtr node_graph_; std::shared_ptr node_handle_; std::shared_ptr context_; diff --git a/rclcpp/include/rclcpp/executor.hpp b/rclcpp/include/rclcpp/executor.hpp index c9994a49a1..fec492306e 100644 --- a/rclcpp/include/rclcpp/executor.hpp +++ b/rclcpp/include/rclcpp/executor.hpp @@ -437,18 +437,6 @@ class Executor static void execute_client(rclcpp::ClientBase::SharedPtr client); - RCLCPP_PUBLIC - static void - execute_subscription(rclcpp::SubscriptionBase * subscription); - - RCLCPP_PUBLIC - static void - execute_service(rclcpp::ServiceBase * service); - - RCLCPP_PUBLIC - static void - execute_client(rclcpp::ClientBase * client); - /** * \throws std::runtime_error if the wait set can be cleared */ diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 1c78f79728..2ba700ada2 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -198,40 +198,6 @@ class EventsExecutor : public rclcpp::Executor this_executor->event_queue_cv_.notify_one(); } - // This function allows to remove an entity from the EventsExecutor. - // Entities are any of SubscriptionBase, PublisherBase, ClientBase, ServerBase, Waitable. - // After an entity has been removed using this API, it can be safely destroyed without the risk - // that the executor would try to access it again. - template - void - remove_entity(T * entity) - { - // We need to unset the callbacks to make sure that after removing events from the - // queues, this entity will not push anymore before being completely destroyed. - // This assumes that all supported entities implement this function. - entity->set_events_executor_callback(nullptr, nullptr); - - // Remove events associated with this entity from the event queue - { - std::unique_lock lock(push_mutex_); - event_queue_.erase( - std::remove_if( - event_queue_.begin(), event_queue_.end(), - [&entity](rmw_listener_event_t event) {return event.entity == entity;}), event_queue_.end()); - } - - // Remove events associated with this entity from the local event queue - { - std::unique_lock lock(execution_mutex_); - execution_event_queue_.erase( - std::remove_if( - execution_event_queue_.begin(), execution_event_queue_.end(), - [&entity](rmw_listener_event_t event) { - return event.entity == entity; - }), execution_event_queue_.end()); - } - } - /// Extract and execute events from the queue until it is empty RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 46960edeed..b92335e07c 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -141,6 +141,50 @@ class EventsExecutorEntitiesCollector final : public EventWaitable std::vector get_automatically_added_callback_groups_from_nodes(); + /// + /** + * Get the subscription shared pointer corresponding + * to the type erased raw subscription pointer + */ + RCLCPP_PUBLIC + rclcpp::SubscriptionBase::SharedPtr + subscription_get_shared(const void * subscription); + + /// + /** + * Get the client shared pointer corresponding + * to the type erased raw client pointer + */ + RCLCPP_PUBLIC + rclcpp::ClientBase::SharedPtr + client_get_shared(const void * client); + + /// + /** + * Get the service shared pointer corresponding + * to the type erased raw service pointer + */ + RCLCPP_PUBLIC + rclcpp::ServiceBase::SharedPtr + service_get_shared(const void * service); + + /// + /** + * Get the waitable shared pointer corresponding + * to the type erased raw waitable pointer + */ + RCLCPP_PUBLIC + rclcpp::Waitable::SharedPtr + waitable_get_shared(const void * waitable); + + /// + /** + * Add a weak pointer to a waitable + */ + RCLCPP_PUBLIC + void + add_waitable(rclcpp::Waitable::SharedPtr waitable); + private: void set_callback_group_entities_callbacks(rclcpp::CallbackGroup::SharedPtr group); @@ -190,6 +234,15 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// List of weak nodes registered in the events executor std::list weak_nodes_; + // Vector of weak pointers to subscriptions registered in the executor + std::vector weak_subscriptions_; + // Vector of weak pointers to services registered in the executor + std::vector weak_services_; + // Vector of weak pointers to clients registered in the executor + std::vector weak_clients_; + // Vector of weak pointers to waitables registered in the executor + std::vector weak_waitables_; + /// Executor using this entities collector object EventsExecutor * associated_executor_ = nullptr; /// Instance of the timers manager used by the associated executor diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index ce67004ca5..1b585d5237 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -40,12 +40,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable // Destructor RCLCPP_PUBLIC - virtual ~EventsExecutorNotifyWaitable() - { - if (on_destruction_callback_) { - on_destruction_callback_(this); - } - } + virtual ~EventsExecutorNotifyWaitable() = default; // The function is a no-op, since we only care of waking up the executor RCLCPP_PUBLIC diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 9e33c05374..8ffbef820f 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -132,10 +132,6 @@ class ServiceBase const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; - RCLCPP_PUBLIC - void - set_on_destruction_callback(std::function on_destruction_callback); - protected: RCLCPP_DISABLE_COPY(ServiceBase) @@ -147,8 +143,6 @@ class ServiceBase const rcl_node_t * get_rcl_node_handle() const; - std::function on_destruction_callback_; - std::shared_ptr node_handle_; std::shared_ptr service_handle_; diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index f9a591dbd1..81d3be4bec 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -275,10 +275,6 @@ class SubscriptionBase : public std::enable_shared_from_this const rclcpp::executors::EventsExecutor * executor, rmw_listener_cb_t executor_callback) const; - RCLCPP_PUBLIC - void - set_on_destruction_callback(std::function on_destruction_callback); - protected: template void @@ -321,8 +317,6 @@ class SubscriptionBase : public std::enable_shared_from_this rosidl_message_type_support_t type_support_; bool is_serialized_; - std::function on_destruction_callback_; - std::atomic subscription_in_use_by_wait_set_{false}; std::atomic intra_process_subscription_waitable_in_use_by_wait_set_{false}; std::unordered_map on_destruction_callback); - -protected: - std::function on_destruction_callback_; - private: std::atomic in_use_by_wait_set_{false}; }; // class Waitable diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 55fdbca3b4..785057c369 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -66,9 +66,6 @@ ClientBase::ClientBase( ClientBase::~ClientBase() { - if (on_destruction_callback_) { - on_destruction_callback_(this); - } // Make sure the client handle is destructed as early as possible and before the node handle client_handle_.reset(); } @@ -217,10 +214,3 @@ ClientBase::set_events_executor_callback( throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); } } - -void -ClientBase::set_on_destruction_callback( - std::function on_destruction_callback) -{ - on_destruction_callback_ = on_destruction_callback; -} diff --git a/rclcpp/src/rclcpp/executor.cpp b/rclcpp/src/rclcpp/executor.cpp index be138fc487..6867698c5a 100644 --- a/rclcpp/src/rclcpp/executor.cpp +++ b/rclcpp/src/rclcpp/executor.cpp @@ -556,12 +556,6 @@ take_and_do_error_handling( void Executor::execute_subscription(rclcpp::SubscriptionBase::SharedPtr subscription) -{ - execute_subscription(subscription.get()); -} - -void -Executor::execute_subscription(rclcpp::SubscriptionBase * subscription) { rclcpp::MessageInfo message_info; message_info.get_rmw_message_info().from_intra_process = false; @@ -636,12 +630,6 @@ Executor::execute_timer(rclcpp::TimerBase::SharedPtr timer) void Executor::execute_service(rclcpp::ServiceBase::SharedPtr service) -{ - execute_service(service.get()); -} - -void -Executor::execute_service(rclcpp::ServiceBase * service) { auto request_header = service->create_request_header(); std::shared_ptr request = service->create_request(); @@ -654,12 +642,6 @@ Executor::execute_service(rclcpp::ServiceBase * service) void Executor::execute_client(rclcpp::ClientBase::SharedPtr client) -{ - execute_client(client.get()); -} - -void -Executor::execute_client(rclcpp::ClientBase * client) { auto request_header = client->create_request_header(); std::shared_ptr response = client->create_response(); diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 0ac56f6e34..931b4f75b9 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -41,11 +41,8 @@ EventsExecutor::EventsExecutor( executor_notifier_->add_guard_condition(context_interrupt_gc); executor_notifier_->add_guard_condition(&interrupt_guard_condition_); executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); - executor_notifier_->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - this, - std::placeholders::_1)); + entities_collector_->add_waitable(executor_notifier_); + entities_collector_->add_waitable(entities_collector_); } void @@ -268,33 +265,53 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) switch (event.type) { case SUBSCRIPTION_EVENT: { - auto subscription = const_cast( - static_cast(event.entity)); - execute_subscription(subscription); + auto subscription = entities_collector_->subscription_get_shared(event.entity); + + if(subscription) { + execute_subscription(subscription); + } else { + RCUTILS_LOG_WARN_NAMED( + "events_executor.cpp", + "Failed to find subscription. Either not found or expired."); + } break; } case SERVICE_EVENT: { - auto service = const_cast( - static_cast(event.entity)); - execute_service(service); + auto service = entities_collector_->service_get_shared(event.entity); + + if(service) { + execute_service(service); + } else { + throw std::runtime_error("Failed to find service. Either not found or expired."); + } break; } case CLIENT_EVENT: { - auto client = const_cast( - static_cast(event.entity)); - execute_client(client); + auto client = entities_collector_->client_get_shared(event.entity); + + if(client) { + execute_client(client); + } else { + throw std::runtime_error("Failed to find client. Either not found or expired."); + } break; } case WAITABLE_EVENT: { - auto waitable = const_cast( - static_cast(event.entity)); - waitable->execute(); + auto waitable = entities_collector_->waitable_get_shared(event.entity); + + if(waitable) { + waitable->execute(); + } else { + RCUTILS_LOG_WARN_NAMED( + "events_executor.cpp", + "Failed to find waitable. Either not found or expired."); + } break; } } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 1b214b27d8..6c1e81a4ec 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -70,11 +70,15 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() } } - // Clear all lists + // Clear all containers + weak_nodes_.clear(); + weak_clients_.clear(); + weak_services_.clear(); + weak_waitables_.clear(); + weak_subscriptions_.clear(); + weak_nodes_to_guard_conditions_.clear(); weak_groups_associated_with_executor_to_nodes_.clear(); weak_groups_to_nodes_associated_with_executor_.clear(); - weak_nodes_to_guard_conditions_.clear(); - weak_nodes_.clear(); } void @@ -227,11 +231,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( subscription->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - subscription->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); + weak_subscriptions_.push_back(subscription); } return false; }); @@ -241,11 +241,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( service->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - service->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); + weak_services_.push_back(service); } return false; }); @@ -255,11 +251,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( client->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - client->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); + weak_clients_.push_back(client); } return false; }); @@ -269,11 +261,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( waitable->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - waitable->set_on_destruction_callback( - std::bind( - &EventsExecutor::remove_entity, - associated_executor_, - std::placeholders::_1)); + weak_waitables_.push_back(waitable); } return false; }); @@ -298,7 +286,16 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { subscription->set_events_executor_callback(nullptr, nullptr); - subscription->set_on_destruction_callback(nullptr); + // Remove subscription weak pointer + auto subscription_it = weak_subscriptions_.begin(); + while (subscription_it != weak_subscriptions_.end()) { + bool matched = (subscription_it->lock() == subscription); + if (matched) { + weak_subscriptions_.erase(subscription_it); + break; + } + ++subscription_it; + } } return false; }); @@ -306,7 +303,17 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { service->set_events_executor_callback(nullptr, nullptr); - service->set_on_destruction_callback(nullptr); + // Remove service weak pointer + auto service_it = weak_services_.begin(); + while (service_it != weak_services_.end()) { + bool matched = (service_it->lock() == service); + if (matched) { + weak_services_.erase(service_it); + break; + } + ++service_it; + } + } return false; }); @@ -314,7 +321,16 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { client->set_events_executor_callback(nullptr, nullptr); - client->set_on_destruction_callback(nullptr); + // Remove client weak pointer + auto client_it = weak_clients_.begin(); + while (client_it != weak_clients_.end()) { + bool matched = (client_it->lock() == client); + if (matched) { + weak_clients_.erase(client_it); + break; + } + ++client_it; + } } return false; }); @@ -322,7 +338,16 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { waitable->set_events_executor_callback(nullptr, nullptr); - waitable->set_on_destruction_callback(nullptr); + // Remove waitable weak pointer + auto waitable_it = weak_waitables_.begin(); + while (waitable_it != weak_waitables_.end()) { + bool matched = (waitable_it->lock() == waitable); + if (matched) { + weak_waitables_.erase(waitable_it); + break; + } + ++waitable_it; + } } return false; }); @@ -511,3 +536,69 @@ EventsExecutorEntitiesCollector::unset_guard_condition_callback( throw std::runtime_error("Couldn't unset guard condition event callback"); } } + +rclcpp::SubscriptionBase::SharedPtr +EventsExecutorEntitiesCollector::subscription_get_shared(const void * subscription) +{ + // Check if this subscription is currently stored here + for (auto & weak_sub_it : weak_subscriptions_) { + if (auto sub_shared_ptr = weak_sub_it.lock()) { + if (sub_shared_ptr.get() == subscription) { + return sub_shared_ptr; + } + } + } + // The subscription was either not found or expired + return nullptr; +} + +rclcpp::ClientBase::SharedPtr +EventsExecutorEntitiesCollector::client_get_shared(const void * client) +{ + // Check if this client is currently stored here + for (auto & weak_client_it : weak_clients_) { + if (auto client_shared_ptr = weak_client_it.lock()) { + if (client_shared_ptr.get() == client) { + return client_shared_ptr; + } + } + } + // The client was either not found or expired + return nullptr; +} + +rclcpp::ServiceBase::SharedPtr +EventsExecutorEntitiesCollector::service_get_shared(const void * service) +{ + // Check if this service is currently stored here + for (auto & weak_service_it : weak_services_) { + if (auto service_shared_ptr = weak_service_it.lock()) { + if (service_shared_ptr.get() == service) { + return service_shared_ptr; + } + } + } + // The service was either not found or expired + return nullptr; +} + +rclcpp::Waitable::SharedPtr +EventsExecutorEntitiesCollector::waitable_get_shared(const void * waitable) +{ + // Check if this waitable is currently stored here + for (auto & weak_waitable_it : weak_waitables_) { + if (auto waitable_shared_ptr = weak_waitable_it.lock()) { + if (waitable_shared_ptr.get() == waitable) { + return waitable_shared_ptr; + } + } + } + // The waitable was either not found or expired + return nullptr; +} + +void +EventsExecutorEntitiesCollector::add_waitable(rclcpp::Waitable::SharedPtr waitable) +{ + weak_waitables_.push_back(waitable); +} diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 4e14f1bc13..5ae3dd1a93 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -35,10 +35,6 @@ UnsupportedEventTypeException::UnsupportedEventTypeException( QOSEventHandlerBase::~QOSEventHandlerBase() { - if (on_destruction_callback_) { - on_destruction_callback_(this); - } - if (rcl_event_fini(&event_handle_) != RCL_RET_OK) { RCUTILS_LOG_ERROR_NAMED( "rclcpp", diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index f4e4092962..0396e728c5 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -32,11 +32,7 @@ ServiceBase::ServiceBase(std::shared_ptr node_handle) {} ServiceBase::~ServiceBase() -{ - if (on_destruction_callback_) { - on_destruction_callback_(this); - } -} +{} bool ServiceBase::take_type_erased_request(void * request_out, rmw_request_id_t & request_id_out) @@ -104,10 +100,3 @@ ServiceBase::set_events_executor_callback( throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); } } - -void -ServiceBase::set_on_destruction_callback( - std::function on_destruction_callback) -{ - on_destruction_callback_ = on_destruction_callback; -} diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 81b6c91646..f272a976a9 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -83,10 +83,6 @@ SubscriptionBase::SubscriptionBase( SubscriptionBase::~SubscriptionBase() { - if (on_destruction_callback_) { - on_destruction_callback_(this); - } - if (!use_intra_process_) { return; } @@ -308,10 +304,3 @@ SubscriptionBase::set_events_executor_callback( throw std::runtime_error("Couldn't set the EventsExecutor's callback to subscription"); } } - -void -SubscriptionBase::set_on_destruction_callback( - std::function on_destruction_callback) -{ - on_destruction_callback_ = on_destruction_callback; -} diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 4a2a4ccc3d..797ddb2453 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -16,13 +16,6 @@ using rclcpp::experimental::SubscriptionIntraProcessBase; -SubscriptionIntraProcessBase::~SubscriptionIntraProcessBase() -{ - if (on_destruction_callback_) { - on_destruction_callback_(this); - } -} - bool SubscriptionIntraProcessBase::add_to_wait_set(rcl_wait_set_t * wait_set) { diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 1df8e2090f..e30ff05f4e 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -71,10 +71,3 @@ Waitable::set_events_executor_callback( throw std::runtime_error( "Custom waitables should override set_events_executor_callback() to use events executor"); } - -void -Waitable::set_on_destruction_callback( - std::function on_destruction_callback) -{ - on_destruction_callback_ = on_destruction_callback; -} diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index f6a66768c7..5dcb907328 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -418,10 +418,6 @@ class TestWaitable : public rclcpp::Waitable ~TestWaitable() { - if (on_destruction_callback_) { - on_destruction_callback_(this); - } - rcl_ret_t ret = rcl_guard_condition_fini(&gc_); if (RCL_RET_OK != ret) { fprintf(stderr, "failed to call rcl_guard_condition_fini\n"); From 4958e05cd4ecbb0d9cf164a4bc7b313ec8f128d9 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Tue, 15 Dec 2020 11:21:18 -0300 Subject: [PATCH 091/168] Use map for weak pointers --- .../events_executor_entities_collector.hpp | 16 +-- .../subscription_intra_process_base.hpp | 2 +- .../events_executor_entities_collector.cpp | 131 ++++++++---------- 3 files changed, 66 insertions(+), 83 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index b92335e07c..7a275df594 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -234,14 +234,14 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// List of weak nodes registered in the events executor std::list weak_nodes_; - // Vector of weak pointers to subscriptions registered in the executor - std::vector weak_subscriptions_; - // Vector of weak pointers to services registered in the executor - std::vector weak_services_; - // Vector of weak pointers to clients registered in the executor - std::vector weak_clients_; - // Vector of weak pointers to waitables registered in the executor - std::vector weak_waitables_; + // Map of weak pointers to subscriptions registered in the executor + std::map weak_subscriptions_map_; + // Map of weak pointers to services registered in the executor + std::map weak_services_map_; + // Map of weak pointers to clients registered in the executor + std::map weak_clients_map_; + // Map of weak pointers to waitables registered in the executor + std::map weak_waitables_map_; /// Executor using this entities collector object EventsExecutor * associated_executor_ = nullptr; diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index c5d2e33455..8684c9f632 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -43,7 +43,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable : topic_name_(topic_name), qos_profile_(qos_profile) {} - virtual ~SubscriptionIntraProcessBase(); + virtual ~SubscriptionIntraProcessBase() = default; RCLCPP_PUBLIC size_t diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 6c1e81a4ec..bbbf92eb13 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -72,10 +72,10 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() // Clear all containers weak_nodes_.clear(); - weak_clients_.clear(); - weak_services_.clear(); - weak_waitables_.clear(); - weak_subscriptions_.clear(); + weak_clients_map_.clear(); + weak_services_map_.clear(); + weak_waitables_map_.clear(); + weak_subscriptions_map_.clear(); weak_nodes_to_guard_conditions_.clear(); weak_groups_associated_with_executor_to_nodes_.clear(); weak_groups_to_nodes_associated_with_executor_.clear(); @@ -231,7 +231,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( subscription->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - weak_subscriptions_.push_back(subscription); + weak_subscriptions_map_.emplace(subscription.get(), subscription); } return false; }); @@ -241,7 +241,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( service->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - weak_services_.push_back(service); + weak_services_map_.emplace(service.get(), service); } return false; }); @@ -251,7 +251,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( client->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - weak_clients_.push_back(client); + weak_clients_map_.emplace(client.get(), client); } return false; }); @@ -261,7 +261,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( waitable->set_events_executor_callback( associated_executor_, &EventsExecutor::push_event); - weak_waitables_.push_back(waitable); + weak_waitables_map_.emplace(waitable.get(), waitable); } return false; }); @@ -286,15 +286,13 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { subscription->set_events_executor_callback(nullptr, nullptr); - // Remove subscription weak pointer - auto subscription_it = weak_subscriptions_.begin(); - while (subscription_it != weak_subscriptions_.end()) { - bool matched = (subscription_it->lock() == subscription); - if (matched) { - weak_subscriptions_.erase(subscription_it); - break; + // Remove subscription from map + for (const auto & pair : weak_subscriptions_map_) { + if (auto subscription_shared_ptr = pair.second.lock()) { + if(subscription == subscription_shared_ptr) { + weak_subscriptions_map_.erase(pair.first); + } } - ++subscription_it; } } return false; @@ -303,17 +301,14 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { service->set_events_executor_callback(nullptr, nullptr); - // Remove service weak pointer - auto service_it = weak_services_.begin(); - while (service_it != weak_services_.end()) { - bool matched = (service_it->lock() == service); - if (matched) { - weak_services_.erase(service_it); - break; + // Remove service from map + for (const auto & pair : weak_services_map_) { + if (auto service_shared_ptr = pair.second.lock()) { + if(service == service_shared_ptr) { + weak_services_map_.erase(pair.first); + } } - ++service_it; } - } return false; }); @@ -321,15 +316,13 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { client->set_events_executor_callback(nullptr, nullptr); - // Remove client weak pointer - auto client_it = weak_clients_.begin(); - while (client_it != weak_clients_.end()) { - bool matched = (client_it->lock() == client); - if (matched) { - weak_clients_.erase(client_it); - break; + // Remove client from map + for (const auto & pair : weak_clients_map_) { + if (auto client_shared_ptr = pair.second.lock()) { + if(client == client_shared_ptr) { + weak_clients_map_.erase(pair.first); + } } - ++client_it; } } return false; @@ -338,15 +331,13 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { waitable->set_events_executor_callback(nullptr, nullptr); - // Remove waitable weak pointer - auto waitable_it = weak_waitables_.begin(); - while (waitable_it != weak_waitables_.end()) { - bool matched = (waitable_it->lock() == waitable); - if (matched) { - weak_waitables_.erase(waitable_it); - break; + // Remove waitable from map + for (const auto & pair : weak_waitables_map_) { + if (auto waitable_shared_ptr = pair.second.lock()) { + if(waitable == waitable_shared_ptr) { + weak_waitables_map_.erase(pair.first); + } } - ++waitable_it; } } return false; @@ -540,65 +531,57 @@ EventsExecutorEntitiesCollector::unset_guard_condition_callback( rclcpp::SubscriptionBase::SharedPtr EventsExecutorEntitiesCollector::subscription_get_shared(const void * subscription) { - // Check if this subscription is currently stored here - for (auto & weak_sub_it : weak_subscriptions_) { - if (auto sub_shared_ptr = weak_sub_it.lock()) { - if (sub_shared_ptr.get() == subscription) { - return sub_shared_ptr; - } - } + auto subscription_weak_ptr = weak_subscriptions_map_.at(subscription); + + if(auto subscription_shared_ptr = subscription_weak_ptr.lock()) { + return subscription_shared_ptr; } - // The subscription was either not found or expired + + // The waitable expired, return a null pointer return nullptr; } rclcpp::ClientBase::SharedPtr EventsExecutorEntitiesCollector::client_get_shared(const void * client) { - // Check if this client is currently stored here - for (auto & weak_client_it : weak_clients_) { - if (auto client_shared_ptr = weak_client_it.lock()) { - if (client_shared_ptr.get() == client) { - return client_shared_ptr; - } - } + auto client_weak_ptr = weak_clients_map_.at(client); + + if(auto client_shared_ptr = client_weak_ptr.lock()) { + return client_shared_ptr; } - // The client was either not found or expired + + // The client expired, return a null pointer return nullptr; } rclcpp::ServiceBase::SharedPtr EventsExecutorEntitiesCollector::service_get_shared(const void * service) { - // Check if this service is currently stored here - for (auto & weak_service_it : weak_services_) { - if (auto service_shared_ptr = weak_service_it.lock()) { - if (service_shared_ptr.get() == service) { - return service_shared_ptr; - } - } + auto service_weak_ptr = weak_services_map_.at(service); + + if(auto service_shared_ptr = service_weak_ptr.lock()) { + return service_shared_ptr; } - // The service was either not found or expired + + // The service expired, return a null pointer return nullptr; } rclcpp::Waitable::SharedPtr EventsExecutorEntitiesCollector::waitable_get_shared(const void * waitable) { - // Check if this waitable is currently stored here - for (auto & weak_waitable_it : weak_waitables_) { - if (auto waitable_shared_ptr = weak_waitable_it.lock()) { - if (waitable_shared_ptr.get() == waitable) { - return waitable_shared_ptr; - } - } + auto waitable_weak_ptr = weak_waitables_map_.at(waitable); + + if(auto waitable_shared_ptr = waitable_weak_ptr.lock()) { + return waitable_shared_ptr; } - // The waitable was either not found or expired + + // The waitable expired, return a null pointer return nullptr; } void EventsExecutorEntitiesCollector::add_waitable(rclcpp::Waitable::SharedPtr waitable) { - weak_waitables_.push_back(waitable); + weak_waitables_map_.emplace(waitable.get(), waitable); } From e5a3174615223713a8083d49875bb0e6914ad837 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Wed, 16 Dec 2020 09:26:48 -0300 Subject: [PATCH 092/168] Use ownership model on timers --- .../rclcpp/executors/timers_manager.hpp | 60 +++++++++++++------ rclcpp/include/rclcpp/timer.hpp | 5 -- .../events_executor_entities_collector.cpp | 5 +- .../src/rclcpp/executors/timers_manager.cpp | 15 +++-- rclcpp/src/rclcpp/timer.cpp | 10 ---- 5 files changed, 49 insertions(+), 46 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 7b75fc2e94..d3e1819ec7 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -121,12 +121,6 @@ class TimersManager */ void remove_timer(rclcpp::TimerBase::SharedPtr timer); - /** - * @brief Remove a single timer stored in the object, passed as a raw ptr. - * @param timer the timer to remove. - */ - void remove_timer_raw(rclcpp::TimerBase * timer); - // This is what the TimersManager uses to denote a duration forever. // We don't use std::chrono::nanoseconds::max because it will overflow. // See https://en.cppreference.com/w/cpp/thread/condition_variable/wait_for @@ -135,7 +129,7 @@ class TimersManager private: RCLCPP_DISABLE_COPY(TimersManager) - using TimerPtr = rclcpp::TimerBase *; + using TimerPtr = rclcpp::TimerBase::SharedPtr; /** * @brief This struct provides convenient access to a MinHeap of timers. @@ -147,11 +141,31 @@ class TimersManager void add_timer(TimerPtr timer) { // Nothing to do if the timer is already stored here - auto it = std::find(timers_.begin(), timers_.end(), timer); - if (it != timers_.end()) { - return; + for(const auto & weak_timer : weak_timers_) { + bool matched = (weak_timer.lock() == timer); + if (matched) { + return; + } + } + + weak_timers_.push_back(timer); + } + + void own_timers() + { + auto weak_timer_it = weak_timers_.begin(); + + while (weak_timer_it != weak_timers_.end()) { + if (auto timer_shared_ptr = weak_timer_it->lock()) { + // The timer is valid, add it to owned timers vector + timers_.push_back(timer_shared_ptr); + } else { + // The timer went out of scope, remove it + weak_timers_.erase(weak_timer_it); + } + weak_timer_it++; } - timers_.push_back(timer); + std::make_heap(timers_.begin(), timers_.end(), timer_greater); } @@ -170,13 +184,19 @@ class TimersManager void remove_timer(TimerPtr timer) { - // Nothing to do if the timer is not stored here - auto it = std::find(timers_.begin(), timers_.end(), timer); - if (it == timers_.end()) { - return; + auto weak_timer_it = weak_timers_.begin(); + + while (weak_timer_it != weak_timers_.end()) { + bool matched = (weak_timer_it->lock() == timer); + if (matched) { + weak_timers_.erase(weak_timer_it); + break; + } + weak_timer_it++; + } + if ((weak_timer_it == weak_timers_.end()) && (!weak_timers_.empty())) { + throw std::runtime_error("Tried to remove timer not stored in the timers manager."); } - timers_.erase(it); - std::make_heap(timers_.begin(), timers_.end(), timer_greater); } TimerPtr & front() @@ -194,7 +214,7 @@ class TimersManager return timers_.empty(); } - void clear() + void release_timers() { timers_.clear(); } @@ -204,8 +224,10 @@ class TimersManager return a->time_until_trigger() > b->time_until_trigger(); } - // Vector of pointers to timers used to implement the priority queue + // Vector of temporary owned timers used to implement the priority queue std::vector timers_; + // Vector of weak pointers to timers registered in the executor + std::vector weak_timers_; }; /** diff --git a/rclcpp/include/rclcpp/timer.hpp b/rclcpp/include/rclcpp/timer.hpp index 9ec679b2ff..a2662f4abe 100644 --- a/rclcpp/include/rclcpp/timer.hpp +++ b/rclcpp/include/rclcpp/timer.hpp @@ -137,14 +137,9 @@ class TimerBase bool exchange_in_use_by_wait_set_state(bool in_use_state); - RCLCPP_PUBLIC - void - set_on_destruction_callback(std::function on_destruction_callback); - protected: Clock::SharedPtr clock_; std::shared_ptr timer_handle_; - std::function on_destruction_callback_; std::atomic in_use_by_wait_set_{false}; }; diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index bbbf92eb13..78570b2d55 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -218,8 +218,6 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { timers_manager_->add_timer(timer); - timer->set_on_destruction_callback( - std::bind(&TimersManager::remove_timer_raw, timers_manager_, std::placeholders::_1)); } return false; }); @@ -276,7 +274,6 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::TimerBase::SharedPtr & timer) { if (timer) { timers_manager_->remove_timer(timer); - timer->set_on_destruction_callback(nullptr); } return false; }); @@ -537,7 +534,7 @@ EventsExecutorEntitiesCollector::subscription_get_shared(const void * subscripti return subscription_shared_ptr; } - // The waitable expired, return a null pointer + // The subscription expired, return a null pointer return nullptr; } diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index b8c97a96f3..49dc687e78 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -43,7 +43,7 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) { std::unique_lock timers_lock(timers_mutex_); - heap_.add_timer(timer.get()); + heap_.add_timer(timer); timers_updated_ = true; } @@ -164,10 +164,14 @@ void TimersManager::run_timers() // Get timeout before next timer expires auto time_to_sleep = this->get_head_timeout_unsafe(); + // Release timers ownership + heap_.release_timers(); // Wait until timeout or notification that timers have been updated timers_cv_.wait_for(timers_lock, time_to_sleep, [this]() {return timers_updated_;}); // Reset timers updated flag timers_updated_ = false; + // Own timer during execution + heap_.own_timers(); // Execute timers this->execute_ready_timers_unsafe(); } @@ -182,7 +186,7 @@ void TimersManager::clear() { // Lock mutex and then clear all data structures std::unique_lock timers_lock(timers_mutex_); - heap_.clear(); + heap_.release_timers(); timers_updated_ = true; } @@ -191,12 +195,7 @@ void TimersManager::clear() timers_cv_.notify_one(); } -void TimersManager::remove_timer(rclcpp::TimerBase::SharedPtr timer) -{ - this->remove_timer_raw(timer.get()); -} - -void TimersManager::remove_timer_raw(rclcpp::TimerBase * timer) +void TimersManager::remove_timer(TimerPtr timer) { { std::unique_lock timers_lock(timers_mutex_); diff --git a/rclcpp/src/rclcpp/timer.cpp b/rclcpp/src/rclcpp/timer.cpp index 4659898055..e312fe84eb 100644 --- a/rclcpp/src/rclcpp/timer.cpp +++ b/rclcpp/src/rclcpp/timer.cpp @@ -72,9 +72,6 @@ TimerBase::TimerBase( TimerBase::~TimerBase() { - if (on_destruction_callback_) { - on_destruction_callback_(this); - } } void @@ -140,10 +137,3 @@ TimerBase::exchange_in_use_by_wait_set_state(bool in_use_state) { return in_use_by_wait_set_.exchange(in_use_state); } - -void -TimerBase::set_on_destruction_callback( - std::function on_destruction_callback) -{ - on_destruction_callback_ = on_destruction_callback; -} From 0c36f1ec9628cb94e7af982cc75b86be87c66081 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Wed, 16 Dec 2020 11:59:19 -0300 Subject: [PATCH 093/168] Use unordered map --- .../events_executor_entities_collector.hpp | 8 ++--- .../events_executor_entities_collector.cpp | 36 +++---------------- 2 files changed, 8 insertions(+), 36 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 7a275df594..23165452df 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -235,13 +235,13 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// List of weak nodes registered in the events executor std::list weak_nodes_; // Map of weak pointers to subscriptions registered in the executor - std::map weak_subscriptions_map_; + std::unordered_map weak_subscriptions_map_; // Map of weak pointers to services registered in the executor - std::map weak_services_map_; + std::unordered_map weak_services_map_; // Map of weak pointers to clients registered in the executor - std::map weak_clients_map_; + std::unordered_map weak_clients_map_; // Map of weak pointers to waitables registered in the executor - std::map weak_waitables_map_; + std::unordered_map weak_waitables_map_; /// Executor using this entities collector object EventsExecutor * associated_executor_ = nullptr; diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 78570b2d55..1f71757c72 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -283,14 +283,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { subscription->set_events_executor_callback(nullptr, nullptr); - // Remove subscription from map - for (const auto & pair : weak_subscriptions_map_) { - if (auto subscription_shared_ptr = pair.second.lock()) { - if(subscription == subscription_shared_ptr) { - weak_subscriptions_map_.erase(pair.first); - } - } - } + weak_subscriptions_map_.erase(subscription.get()); } return false; }); @@ -298,14 +291,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { service->set_events_executor_callback(nullptr, nullptr); - // Remove service from map - for (const auto & pair : weak_services_map_) { - if (auto service_shared_ptr = pair.second.lock()) { - if(service == service_shared_ptr) { - weak_services_map_.erase(pair.first); - } - } - } + weak_services_map_.erase(service.get()); } return false; }); @@ -313,14 +299,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { client->set_events_executor_callback(nullptr, nullptr); - // Remove client from map - for (const auto & pair : weak_clients_map_) { - if (auto client_shared_ptr = pair.second.lock()) { - if(client == client_shared_ptr) { - weak_clients_map_.erase(pair.first); - } - } - } + weak_clients_map_.erase(client.get()); } return false; }); @@ -328,14 +307,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { waitable->set_events_executor_callback(nullptr, nullptr); - // Remove waitable from map - for (const auto & pair : weak_waitables_map_) { - if (auto waitable_shared_ptr = pair.second.lock()) { - if(waitable == waitable_shared_ptr) { - weak_waitables_map_.erase(pair.first); - } - } - } + weak_waitables_map_.erase(waitable.get()); } return false; }); From 2c8aa07453b9acd814485f58f7a417280fee7306 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Wed, 16 Dec 2020 12:30:13 -0300 Subject: [PATCH 094/168] release timers ownership when stopping --- rclcpp/src/rclcpp/executors/timers_manager.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 49dc687e78..a0d4875385 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -73,6 +73,7 @@ void TimersManager::stop() { std::unique_lock timers_lock(timers_mutex_); timers_updated_ = true; + heap_.release_timers(); } timers_cv_.notify_one(); From 97ca9fb637332771f56b93363825c8871cf79e84 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 17 Dec 2020 15:56:31 -0300 Subject: [PATCH 095/168] Rename functions --- .../events_executor_entities_collector.hpp | 8 ++-- .../src/rclcpp/executors/events_executor.cpp | 20 ++-------- .../events_executor_entities_collector.cpp | 40 ++++--------------- 3 files changed, 16 insertions(+), 52 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 23165452df..c26f6f0e80 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -148,7 +148,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable */ RCLCPP_PUBLIC rclcpp::SubscriptionBase::SharedPtr - subscription_get_shared(const void * subscription); + get_subscription(const void * subscription); /// /** @@ -157,7 +157,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable */ RCLCPP_PUBLIC rclcpp::ClientBase::SharedPtr - client_get_shared(const void * client); + get_client(const void * client); /// /** @@ -166,7 +166,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable */ RCLCPP_PUBLIC rclcpp::ServiceBase::SharedPtr - service_get_shared(const void * service); + get_service(const void * service); /// /** @@ -175,7 +175,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable */ RCLCPP_PUBLIC rclcpp::Waitable::SharedPtr - waitable_get_shared(const void * waitable); + get_waitable(const void * waitable); /// /** diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 931b4f75b9..f5617a1227 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -265,52 +265,40 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) switch (event.type) { case SUBSCRIPTION_EVENT: { - auto subscription = entities_collector_->subscription_get_shared(event.entity); + auto subscription = entities_collector_->get_subscription(event.entity); if(subscription) { execute_subscription(subscription); - } else { - RCUTILS_LOG_WARN_NAMED( - "events_executor.cpp", - "Failed to find subscription. Either not found or expired."); } break; } case SERVICE_EVENT: { - auto service = entities_collector_->service_get_shared(event.entity); + auto service = entities_collector_->get_service(event.entity); if(service) { execute_service(service); - } else { - throw std::runtime_error("Failed to find service. Either not found or expired."); } break; } case CLIENT_EVENT: { - auto client = entities_collector_->client_get_shared(event.entity); + auto client = entities_collector_->get_client(event.entity); if(client) { execute_client(client); - } else { - throw std::runtime_error("Failed to find client. Either not found or expired."); } break; } case WAITABLE_EVENT: { - auto waitable = entities_collector_->waitable_get_shared(event.entity); + auto waitable = entities_collector_->get_waitable(event.entity); if(waitable) { waitable->execute(); - } else { - RCUTILS_LOG_WARN_NAMED( - "events_executor.cpp", - "Failed to find waitable. Either not found or expired."); } break; } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 1f71757c72..8c8f015b50 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -498,55 +498,31 @@ EventsExecutorEntitiesCollector::unset_guard_condition_callback( } rclcpp::SubscriptionBase::SharedPtr -EventsExecutorEntitiesCollector::subscription_get_shared(const void * subscription) +EventsExecutorEntitiesCollector::get_subscription(const void * subscription) { auto subscription_weak_ptr = weak_subscriptions_map_.at(subscription); - - if(auto subscription_shared_ptr = subscription_weak_ptr.lock()) { - return subscription_shared_ptr; - } - - // The subscription expired, return a null pointer - return nullptr; + return subscription_weak_ptr.lock(); } rclcpp::ClientBase::SharedPtr -EventsExecutorEntitiesCollector::client_get_shared(const void * client) +EventsExecutorEntitiesCollector::get_client(const void * client) { auto client_weak_ptr = weak_clients_map_.at(client); - - if(auto client_shared_ptr = client_weak_ptr.lock()) { - return client_shared_ptr; - } - - // The client expired, return a null pointer - return nullptr; + return client_weak_ptr.lock(); } rclcpp::ServiceBase::SharedPtr -EventsExecutorEntitiesCollector::service_get_shared(const void * service) +EventsExecutorEntitiesCollector::get_service(const void * service) { auto service_weak_ptr = weak_services_map_.at(service); - - if(auto service_shared_ptr = service_weak_ptr.lock()) { - return service_shared_ptr; - } - - // The service expired, return a null pointer - return nullptr; + return service_weak_ptr.lock(); } rclcpp::Waitable::SharedPtr -EventsExecutorEntitiesCollector::waitable_get_shared(const void * waitable) +EventsExecutorEntitiesCollector::get_waitable(const void * waitable) { auto waitable_weak_ptr = weak_waitables_map_.at(waitable); - - if(auto waitable_shared_ptr = waitable_weak_ptr.lock()) { - return waitable_shared_ptr; - } - - // The waitable expired, return a null pointer - return nullptr; + return waitable_weak_ptr.lock(); } void From b0ad69f1c2b509e51b3d8a913d9f996c5da02cac Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 17 Dec 2020 18:10:26 -0300 Subject: [PATCH 096/168] Remove entity if expired --- .../events_executor_entities_collector.cpp | 36 ++++++++++++++++--- 1 file changed, 32 insertions(+), 4 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 8c8f015b50..49f9151538 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -501,28 +501,56 @@ rclcpp::SubscriptionBase::SharedPtr EventsExecutorEntitiesCollector::get_subscription(const void * subscription) { auto subscription_weak_ptr = weak_subscriptions_map_.at(subscription); - return subscription_weak_ptr.lock(); + + if(auto subscription_shared_ptr = subscription_weak_ptr.lock()) { + return subscription_shared_ptr; + } + + // The subscription expired, remove from map + weak_subscriptions_map_.erase(subscription); + return nullptr; } rclcpp::ClientBase::SharedPtr EventsExecutorEntitiesCollector::get_client(const void * client) { auto client_weak_ptr = weak_clients_map_.at(client); - return client_weak_ptr.lock(); + + if(auto client_shared_ptr = client_weak_ptr.lock()) { + return client_shared_ptr; + } + + // The client expired, remove from map + weak_clients_map_.erase(client); + return nullptr; } rclcpp::ServiceBase::SharedPtr EventsExecutorEntitiesCollector::get_service(const void * service) { auto service_weak_ptr = weak_services_map_.at(service); - return service_weak_ptr.lock(); + + if(auto service_shared_ptr = service_weak_ptr.lock()) { + return service_shared_ptr; + } + + // The service expired, remove from map + weak_services_map_.erase(service); + return nullptr; } rclcpp::Waitable::SharedPtr EventsExecutorEntitiesCollector::get_waitable(const void * waitable) { auto waitable_weak_ptr = weak_waitables_map_.at(waitable); - return waitable_weak_ptr.lock(); + + if(auto waitable_shared_ptr = waitable_weak_ptr.lock()) { + return waitable_shared_ptr; + } + + // The waitable expired, remove from map + weak_waitables_map_.erase(waitable); + return nullptr; } void From 105d39e9c88345cc092facc418be20e676482e50 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 09:36:22 -0300 Subject: [PATCH 097/168] Remove entities if not previously removed --- .../events_executor_entities_collector.cpp | 25 +++++++++++++------ 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 49f9151538..cac2b5342e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -506,8 +506,11 @@ EventsExecutorEntitiesCollector::get_subscription(const void * subscription) return subscription_shared_ptr; } - // The subscription expired, remove from map - weak_subscriptions_map_.erase(subscription); + // The subscription expired, remove from map if not previously removed + if (weak_subscriptions_map_.find(subscription) != weak_subscriptions_map_.end()) { + weak_subscriptions_map_.erase(subscription); + } + return nullptr; } @@ -520,8 +523,10 @@ EventsExecutorEntitiesCollector::get_client(const void * client) return client_shared_ptr; } - // The client expired, remove from map - weak_clients_map_.erase(client); + // The client expired, remove from map if not previously removed + if (weak_clients_map_.find(client) != weak_clients_map_.end()) { + weak_clients_map_.erase(client); + } return nullptr; } @@ -534,8 +539,10 @@ EventsExecutorEntitiesCollector::get_service(const void * service) return service_shared_ptr; } - // The service expired, remove from map - weak_services_map_.erase(service); + // The service expired, remove from map if not previously removed + if (weak_services_map_.find(service) != weak_services_map_.end()) { + weak_services_map_.erase(service); + } return nullptr; } @@ -548,8 +555,10 @@ EventsExecutorEntitiesCollector::get_waitable(const void * waitable) return waitable_shared_ptr; } - // The waitable expired, remove from map - weak_waitables_map_.erase(waitable); + // The waitable expired, remove from map if not previously removed + if (weak_waitables_map_.find(waitable) != weak_waitables_map_.end()) { + weak_waitables_map_.erase(waitable); + } return nullptr; } From 1cf77848ff46a75f99aec66b3a38677bae12a55c Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 09:49:28 -0300 Subject: [PATCH 098/168] Check if not removed before anything --- .../events_executor_entities_collector.cpp | 49 +++++++++---------- 1 file changed, 24 insertions(+), 25 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index cac2b5342e..b174f16a5a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -500,31 +500,30 @@ EventsExecutorEntitiesCollector::unset_guard_condition_callback( rclcpp::SubscriptionBase::SharedPtr EventsExecutorEntitiesCollector::get_subscription(const void * subscription) { - auto subscription_weak_ptr = weak_subscriptions_map_.at(subscription); + if (weak_subscriptions_map_.find(subscription) != weak_subscriptions_map_.end()) { + auto subscription_weak_ptr = weak_subscriptions_map_.at(subscription); - if(auto subscription_shared_ptr = subscription_weak_ptr.lock()) { - return subscription_shared_ptr; - } + if(auto subscription_shared_ptr = subscription_weak_ptr.lock()) { + return subscription_shared_ptr; + } - // The subscription expired, remove from map if not previously removed - if (weak_subscriptions_map_.find(subscription) != weak_subscriptions_map_.end()) { + // The subscription expired, remove from map weak_subscriptions_map_.erase(subscription); } - return nullptr; } rclcpp::ClientBase::SharedPtr EventsExecutorEntitiesCollector::get_client(const void * client) { - auto client_weak_ptr = weak_clients_map_.at(client); + if (weak_clients_map_.find(client) != weak_clients_map_.end()) { + auto client_weak_ptr = weak_clients_map_.at(client); - if(auto client_shared_ptr = client_weak_ptr.lock()) { - return client_shared_ptr; - } + if(auto client_shared_ptr = client_weak_ptr.lock()) { + return client_shared_ptr; + } - // The client expired, remove from map if not previously removed - if (weak_clients_map_.find(client) != weak_clients_map_.end()) { + // The client expired, remove from map weak_clients_map_.erase(client); } return nullptr; @@ -533,14 +532,14 @@ EventsExecutorEntitiesCollector::get_client(const void * client) rclcpp::ServiceBase::SharedPtr EventsExecutorEntitiesCollector::get_service(const void * service) { - auto service_weak_ptr = weak_services_map_.at(service); + if (weak_services_map_.find(service) != weak_services_map_.end()) { + auto service_weak_ptr = weak_services_map_.at(service); - if(auto service_shared_ptr = service_weak_ptr.lock()) { - return service_shared_ptr; - } + if(auto service_shared_ptr = service_weak_ptr.lock()) { + return service_shared_ptr; + } - // The service expired, remove from map if not previously removed - if (weak_services_map_.find(service) != weak_services_map_.end()) { + // The service expired, remove from map weak_services_map_.erase(service); } return nullptr; @@ -549,14 +548,14 @@ EventsExecutorEntitiesCollector::get_service(const void * service) rclcpp::Waitable::SharedPtr EventsExecutorEntitiesCollector::get_waitable(const void * waitable) { - auto waitable_weak_ptr = weak_waitables_map_.at(waitable); + if (weak_waitables_map_.find(waitable) != weak_waitables_map_.end()) { + auto waitable_weak_ptr = weak_waitables_map_.at(waitable); - if(auto waitable_shared_ptr = waitable_weak_ptr.lock()) { - return waitable_shared_ptr; - } + if(auto waitable_shared_ptr = waitable_weak_ptr.lock()) { + return waitable_shared_ptr; + } - // The waitable expired, remove from map if not previously removed - if (weak_waitables_map_.find(waitable) != weak_waitables_map_.end()) { + // The waitable expired, remove from map weak_waitables_map_.erase(waitable); } return nullptr; From 78a01586b98ec148dfc47a38af3791932544f1d5 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 10:16:25 -0300 Subject: [PATCH 099/168] Optimize get_ --- .../events_executor_entities_collector.cpp | 52 ++++++++++++------- 1 file changed, 32 insertions(+), 20 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index b174f16a5a..07fadd38a7 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -498,65 +498,77 @@ EventsExecutorEntitiesCollector::unset_guard_condition_callback( } rclcpp::SubscriptionBase::SharedPtr -EventsExecutorEntitiesCollector::get_subscription(const void * subscription) +EventsExecutorEntitiesCollector::get_subscription(const void * subscription_id) { - if (weak_subscriptions_map_.find(subscription) != weak_subscriptions_map_.end()) { - auto subscription_weak_ptr = weak_subscriptions_map_.at(subscription); + auto it = weak_subscriptions_map_.find(subscription_id); - if(auto subscription_shared_ptr = subscription_weak_ptr.lock()) { + if (it != weak_subscriptions_map_.end()) { + auto subscription_weak_ptr = it->second; + auto subscription_shared_ptr = subscription_weak_ptr.lock(); + + if(subscription_shared_ptr) { return subscription_shared_ptr; } // The subscription expired, remove from map - weak_subscriptions_map_.erase(subscription); + weak_subscriptions_map_.erase(it); } return nullptr; } rclcpp::ClientBase::SharedPtr -EventsExecutorEntitiesCollector::get_client(const void * client) +EventsExecutorEntitiesCollector::get_client(const void * client_id) { - if (weak_clients_map_.find(client) != weak_clients_map_.end()) { - auto client_weak_ptr = weak_clients_map_.at(client); + auto it = weak_clients_map_.find(client_id); + + if (it != weak_clients_map_.end()) { + auto client_weak_ptr = it->second; + auto client_shared_ptr = client_weak_ptr.lock(); - if(auto client_shared_ptr = client_weak_ptr.lock()) { + if(client_shared_ptr) { return client_shared_ptr; } // The client expired, remove from map - weak_clients_map_.erase(client); + weak_clients_map_.erase(it); } return nullptr; } rclcpp::ServiceBase::SharedPtr -EventsExecutorEntitiesCollector::get_service(const void * service) +EventsExecutorEntitiesCollector::get_service(const void * service_id) { - if (weak_services_map_.find(service) != weak_services_map_.end()) { - auto service_weak_ptr = weak_services_map_.at(service); + auto it = weak_services_map_.find(service_id); - if(auto service_shared_ptr = service_weak_ptr.lock()) { + if (it != weak_services_map_.end()) { + auto service_weak_ptr = it->second; + auto service_shared_ptr = service_weak_ptr.lock(); + + if(service_shared_ptr) { return service_shared_ptr; } // The service expired, remove from map - weak_services_map_.erase(service); + weak_services_map_.erase(it); } return nullptr; } rclcpp::Waitable::SharedPtr -EventsExecutorEntitiesCollector::get_waitable(const void * waitable) +EventsExecutorEntitiesCollector::get_waitable(const void * waitable_id) { - if (weak_waitables_map_.find(waitable) != weak_waitables_map_.end()) { - auto waitable_weak_ptr = weak_waitables_map_.at(waitable); + auto it = weak_waitables_map_.find(waitable_id); + + if (it != weak_waitables_map_.end()) { + auto waitable_weak_ptr = it->second; + auto waitable_shared_ptr = waitable_weak_ptr.lock(); - if(auto waitable_shared_ptr = waitable_weak_ptr.lock()) { + if(waitable_shared_ptr) { return waitable_shared_ptr; } // The waitable expired, remove from map - weak_waitables_map_.erase(waitable); + weak_waitables_map_.erase(it); } return nullptr; } From 1d8ed45e690473b631aab50bb081dcc244b83b39 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 10:20:05 -0300 Subject: [PATCH 100/168] rename entity identifiers --- .../events_executor_entities_collector.hpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index c26f6f0e80..4f6130f180 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -144,38 +144,38 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// /** * Get the subscription shared pointer corresponding - * to the type erased raw subscription pointer + * to the subscription identifier */ RCLCPP_PUBLIC rclcpp::SubscriptionBase::SharedPtr - get_subscription(const void * subscription); + get_subscription(const void * subscription_id); /// /** * Get the client shared pointer corresponding - * to the type erased raw client pointer + * to the client identifier */ RCLCPP_PUBLIC rclcpp::ClientBase::SharedPtr - get_client(const void * client); + get_client(const void * client_id); /// /** * Get the service shared pointer corresponding - * to the type erased raw service pointer + * to the service identifier */ RCLCPP_PUBLIC rclcpp::ServiceBase::SharedPtr - get_service(const void * service); + get_service(const void * service_id); /// /** * Get the waitable shared pointer corresponding - * to the type erased raw waitable pointer + * to the waitable identifier */ RCLCPP_PUBLIC rclcpp::Waitable::SharedPtr - get_waitable(const void * waitable); + get_waitable(const void * waitable_id); /// /** From 4f6d66cb5d2a2e8519aa220fe29190b6ad9230e6 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 10:31:00 -0300 Subject: [PATCH 101/168] clarify comment --- .../rclcpp/executors/events_executor_entities_collector.hpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 4f6130f180..a5b5c78bba 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -234,13 +234,11 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// List of weak nodes registered in the events executor std::list weak_nodes_; - // Map of weak pointers to subscriptions registered in the executor + + // Maps: entity identifiers to weak pointers from the entities registered in the executor std::unordered_map weak_subscriptions_map_; - // Map of weak pointers to services registered in the executor std::unordered_map weak_services_map_; - // Map of weak pointers to clients registered in the executor std::unordered_map weak_clients_map_; - // Map of weak pointers to waitables registered in the executor std::unordered_map weak_waitables_map_; /// Executor using this entities collector object From f2c2356e494cce4bc3fa1bf6858040de0fbf6da6 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 10:32:47 -0300 Subject: [PATCH 102/168] clarify comment --- .../executors/events_executor_entities_collector.hpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index a5b5c78bba..2a37c43b8a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -144,7 +144,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// /** * Get the subscription shared pointer corresponding - * to the subscription identifier + * to a subscription identifier */ RCLCPP_PUBLIC rclcpp::SubscriptionBase::SharedPtr @@ -153,7 +153,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// /** * Get the client shared pointer corresponding - * to the client identifier + * to a client identifier */ RCLCPP_PUBLIC rclcpp::ClientBase::SharedPtr @@ -162,7 +162,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// /** * Get the service shared pointer corresponding - * to the service identifier + * to a service identifier */ RCLCPP_PUBLIC rclcpp::ServiceBase::SharedPtr @@ -171,7 +171,7 @@ class EventsExecutorEntitiesCollector final : public EventWaitable /// /** * Get the waitable shared pointer corresponding - * to the waitable identifier + * to a waitable identifier */ RCLCPP_PUBLIC rclcpp::Waitable::SharedPtr From b13618111b40626e8507a79549f98d00f71217cf Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 12:10:29 -0300 Subject: [PATCH 103/168] refactor timers_manager ownership --- .../rclcpp/executors/timers_manager.hpp | 80 +++++-------- .../src/rclcpp/executors/timers_manager.cpp | 106 ++++++++++++++---- 2 files changed, 110 insertions(+), 76 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index d3e1819ec7..30217e4d09 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -88,6 +88,16 @@ class TimersManager */ void stop(); + /** + * @brief Acquire ownership of timers and make heap + */ + void own_timers(); + + /** + * @brief Release ownership of timers + */ + void release_timers(); + /** * @brief Executes all the timers currently ready when the function is invoked * while keeping the heap correctly sorted. @@ -140,32 +150,11 @@ class TimersManager { void add_timer(TimerPtr timer) { - // Nothing to do if the timer is already stored here - for(const auto & weak_timer : weak_timers_) { - bool matched = (weak_timer.lock() == timer); - if (matched) { - return; - } - } - - weak_timers_.push_back(timer); + timers_.push_back(timer); } - void own_timers() + void make_heap() { - auto weak_timer_it = weak_timers_.begin(); - - while (weak_timer_it != weak_timers_.end()) { - if (auto timer_shared_ptr = weak_timer_it->lock()) { - // The timer is valid, add it to owned timers vector - timers_.push_back(timer_shared_ptr); - } else { - // The timer went out of scope, remove it - weak_timers_.erase(weak_timer_it); - } - weak_timer_it++; - } - std::make_heap(timers_.begin(), timers_.end(), timer_greater); } @@ -184,19 +173,13 @@ class TimersManager void remove_timer(TimerPtr timer) { - auto weak_timer_it = weak_timers_.begin(); - - while (weak_timer_it != weak_timers_.end()) { - bool matched = (weak_timer_it->lock() == timer); - if (matched) { - weak_timers_.erase(weak_timer_it); - break; - } - weak_timer_it++; - } - if ((weak_timer_it == weak_timers_.end()) && (!weak_timers_.empty())) { - throw std::runtime_error("Tried to remove timer not stored in the timers manager."); + // Nothing to do if the timer is not stored here + auto it = std::find(timers_.begin(), timers_.end(), timer); + if (it == timers_.end()) { + return; } + timers_.erase(it); + std::make_heap(timers_.begin(), timers_.end(), timer_greater); } TimerPtr & front() @@ -214,7 +197,7 @@ class TimersManager return timers_.empty(); } - void release_timers() + void clear() { timers_.clear(); } @@ -224,10 +207,8 @@ class TimersManager return a->time_until_trigger() > b->time_until_trigger(); } - // Vector of temporary owned timers used to implement the priority queue + // Vector of pointers to timers used to implement the priority queue std::vector timers_; - // Vector of weak pointers to timers registered in the executor - std::vector weak_timers_; }; /** @@ -237,27 +218,14 @@ class TimersManager void run_timers(); /** - * @brief Get the amount of time before the next timer expires. + * @brief Executes all the timers currently ready when the function is invoked + * while keeping the heap correctly sorted. * This function is not thread safe, acquire a mutex before calling it. - * * @return std::chrono::nanoseconds to wait, * the returned value could be negative if the timer is already expired * or MAX_TIME if the heap is empty. */ - std::chrono::nanoseconds get_head_timeout_unsafe() - { - if (heap_.empty()) { - return MAX_TIME; - } - return (heap_.front())->time_until_trigger(); - } - - /** - * @brief Executes all the timers currently ready when the function is invoked - * while keeping the heap correctly sorted. - * This function is not thread safe, acquire a mutex before calling it. - */ - void execute_ready_timers_unsafe(); + std::chrono::nanoseconds execute_ready_timers_unsafe(); /** * @brief Helper function that checks whether a timer was already ready @@ -289,6 +257,8 @@ class TimersManager std::shared_ptr context_; // MinHeap of timers TimersHeap heap_; + // Vector of weak pointers to registered timers + std::vector weak_timers_; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index a0d4875385..e571b93573 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -43,11 +43,21 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) { std::unique_lock timers_lock(timers_mutex_); - heap_.add_timer(timer); + + // Nothing to do if the timer is already stored here + for(const auto & weak_timer : weak_timers_) { + bool matched = (weak_timer.lock() == timer); + + if (matched) { + return; + } + } + + weak_timers_.push_back(timer); timers_updated_ = true; } - // Notify that a timer has been added to the heap + // Notify that a timer has been added to the vector of weak timers timers_cv_.notify_one(); } @@ -73,7 +83,7 @@ void TimersManager::stop() { std::unique_lock timers_lock(timers_mutex_); timers_updated_ = true; - heap_.release_timers(); + this->release_timers(); } timers_cv_.notify_one(); @@ -83,16 +93,45 @@ void TimersManager::stop() } } -std::chrono::nanoseconds TimersManager::get_head_timeout() +void TimersManager::own_timers() { - // Do not allow to interfere with the thread running - if (running_) { - throw std::runtime_error( - "TimersManager::get_head_timeout() can't be used while timers thread is running"); + auto it = weak_timers_.begin(); + + while (it != weak_timers_.end()) { + if (auto timer_shared_ptr = it->lock()) { + // The timer is valid, own it and add to heap + heap_.add_timer(timer_shared_ptr); + } else { + // The timer went out of scope, remove it + weak_timers_.erase(it); + } + it++; } + heap_.make_heap(); +} + +void TimersManager::release_timers() +{ + heap_.clear(); +} + + +std::chrono::nanoseconds TimersManager::get_head_timeout() +{ + this->own_timers(); + std::unique_lock lock(timers_mutex_); - return this->get_head_timeout_unsafe(); + + if (heap_.empty()) { + return MAX_TIME; + } + + auto time_until_trigger = (heap_.front())->time_until_trigger(); + + this->release_timers(); + + return time_until_trigger; } void TimersManager::execute_ready_timers() @@ -117,6 +156,8 @@ bool TimersManager::execute_head_timer() std::unique_lock lock(timers_mutex_); + this->own_timers(); + // Nothing to do if we don't have any timer if (heap_.empty()) { return false; @@ -126,19 +167,22 @@ bool TimersManager::execute_head_timer() if (head->is_ready()) { // Head timer is ready, execute and re-heapify head->execute_callback(); - heap_.heapify_root(); + this->release_timers(); return true; } else { // Head timer was not ready yet + this->release_timers(); return false; } } -void TimersManager::execute_ready_timers_unsafe() +std::chrono::nanoseconds TimersManager::execute_ready_timers_unsafe() { + this->own_timers(); + // Nothing to do if we don't have any timer if (heap_.empty()) { - return; + return MAX_TIME; } // Keep executing timers until they are read and they were already ready when we started. @@ -155,26 +199,28 @@ void TimersManager::execute_ready_timers_unsafe() // Get new head timer head = heap_.front(); } + + auto time_until_trigger = (heap_.front())->time_until_trigger(); + + this->release_timers(); + + return time_until_trigger; } void TimersManager::run_timers() { + auto time_to_sleep = this->get_head_timeout(); // Aquires and releases ownership + while (rclcpp::ok(context_) && running_) { // Lock mutex std::unique_lock timers_lock(timers_mutex_); - // Get timeout before next timer expires - auto time_to_sleep = this->get_head_timeout_unsafe(); - // Release timers ownership - heap_.release_timers(); // Wait until timeout or notification that timers have been updated timers_cv_.wait_for(timers_lock, time_to_sleep, [this]() {return timers_updated_;}); // Reset timers updated flag timers_updated_ = false; - // Own timer during execution - heap_.own_timers(); // Execute timers - this->execute_ready_timers_unsafe(); + time_to_sleep = this->execute_ready_timers_unsafe(); } // Make sure the running flag is set to false when we exit from this function @@ -187,7 +233,7 @@ void TimersManager::clear() { // Lock mutex and then clear all data structures std::unique_lock timers_lock(timers_mutex_); - heap_.release_timers(); + heap_.clear(); timers_updated_ = true; } @@ -200,7 +246,25 @@ void TimersManager::remove_timer(TimerPtr timer) { { std::unique_lock timers_lock(timers_mutex_); - heap_.remove_timer(timer); + + bool matched = false; + auto it = weak_timers_.begin(); + + while (it != weak_timers_.end()) { + matched = (it->lock() == timer); + + if (matched) { + weak_timers_.erase(it); + break; + } + + it++; + } + + if ((it == weak_timers_.end()) && !matched ) { + throw std::runtime_error("Tried to remove timer not stored in the timers manager."); + } + timers_updated_ = true; } From 323d9b65e0dfc307a63d9f43fc9e37df68f1327e Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 18 Dec 2020 13:23:26 -0300 Subject: [PATCH 104/168] Reimplement onwership timers --- .../rclcpp/executors/timers_manager.hpp | 61 +++--------------- .../src/rclcpp/executors/timers_manager.cpp | 63 ++++++++----------- 2 files changed, 36 insertions(+), 88 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 30217e4d09..ae2416b58d 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -56,6 +56,8 @@ namespace executors * so timers may be executed out of order. * */ +using TimerPtr = rclcpp::TimerBase::SharedPtr; + class TimersManager { public: @@ -91,12 +93,7 @@ class TimersManager /** * @brief Acquire ownership of timers and make heap */ - void own_timers(); - - /** - * @brief Release ownership of timers - */ - void release_timers(); + void own_timers(std::vector & timers); /** * @brief Executes all the timers currently ready when the function is invoked @@ -139,7 +136,6 @@ class TimersManager private: RCLCPP_DISABLE_COPY(TimersManager) - using TimerPtr = rclcpp::TimerBase::SharedPtr; /** * @brief This struct provides convenient access to a MinHeap of timers. @@ -148,67 +144,28 @@ class TimersManager */ struct TimersHeap { - void add_timer(TimerPtr timer) - { - timers_.push_back(timer); - } - - void make_heap() + void make_heap(std::vector & timers) { - std::make_heap(timers_.begin(), timers_.end(), timer_greater); + std::make_heap(timers.begin(), timers.end(), timer_greater); } /** * @brief Restore a valid heap after the root value is replaced. */ - void heapify_root() + void heapify_root(std::vector & timers) { // Push the modified element (i.e. the current root) at the bottom of the heap - timers_.push_back(timers_[0]); + timers.push_back(timers[0]); // Exchange first and last elements and reheapify - std::pop_heap(timers_.begin(), timers_.end(), timer_greater); + std::pop_heap(timers.begin(), timers.end(), timer_greater); // Remove last element - timers_.pop_back(); - } - - void remove_timer(TimerPtr timer) - { - // Nothing to do if the timer is not stored here - auto it = std::find(timers_.begin(), timers_.end(), timer); - if (it == timers_.end()) { - return; - } - timers_.erase(it); - std::make_heap(timers_.begin(), timers_.end(), timer_greater); - } - - TimerPtr & front() - { - return timers_.front(); - } - - const TimerPtr & front() const - { - return timers_.front(); - } - - bool empty() - { - return timers_.empty(); - } - - void clear() - { - timers_.clear(); + timers.pop_back(); } static bool timer_greater(TimerPtr a, TimerPtr b) { return a->time_until_trigger() > b->time_until_trigger(); } - - // Vector of pointers to timers used to implement the priority queue - std::vector timers_; }; /** diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index e571b93573..19252f51cf 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -83,7 +83,6 @@ void TimersManager::stop() { std::unique_lock timers_lock(timers_mutex_); timers_updated_ = true; - this->release_timers(); } timers_cv_.notify_one(); @@ -93,14 +92,14 @@ void TimersManager::stop() } } -void TimersManager::own_timers() +void TimersManager::own_timers(std::vector & timers) { auto it = weak_timers_.begin(); while (it != weak_timers_.end()) { if (auto timer_shared_ptr = it->lock()) { // The timer is valid, own it and add to heap - heap_.add_timer(timer_shared_ptr); + timers.push_back(timer_shared_ptr); } else { // The timer went out of scope, remove it weak_timers_.erase(it); @@ -108,30 +107,20 @@ void TimersManager::own_timers() it++; } - heap_.make_heap(); + heap_.make_heap(timers); } -void TimersManager::release_timers() -{ - heap_.clear(); -} - - std::chrono::nanoseconds TimersManager::get_head_timeout() { - this->own_timers(); + // Vector of pointers to timers used to implement the priority queue + std::vector timers; + this->own_timers(timers); - std::unique_lock lock(timers_mutex_); - - if (heap_.empty()) { + if (timers.empty()) { return MAX_TIME; } - auto time_until_trigger = (heap_.front())->time_until_trigger(); - - this->release_timers(); - - return time_until_trigger; + return (timers.front())->time_until_trigger(); } void TimersManager::execute_ready_timers() @@ -154,34 +143,38 @@ bool TimersManager::execute_head_timer() "TimersManager::execute_head_timer() can't be used while timers thread is running"); } - std::unique_lock lock(timers_mutex_); + std::unique_lock lock(timers_mutex_); //needed? + + // Vector of pointers to timers used to implement the priority queue + std::vector timers; - this->own_timers(); + this->own_timers(timers); // Nothing to do if we don't have any timer - if (heap_.empty()) { + if (timers.empty()) { return false; } - TimerPtr head = heap_.front(); + TimerPtr head = timers.front(); if (head->is_ready()) { // Head timer is ready, execute and re-heapify head->execute_callback(); - this->release_timers(); return true; } else { // Head timer was not ready yet - this->release_timers(); return false; } } std::chrono::nanoseconds TimersManager::execute_ready_timers_unsafe() { - this->own_timers(); + // Vector of pointers to timers used to implement the priority queue + std::vector timers; + + this->own_timers(timers); // Nothing to do if we don't have any timer - if (heap_.empty()) { + if (timers.empty()) { return MAX_TIME; } @@ -190,21 +183,19 @@ std::chrono::nanoseconds TimersManager::execute_ready_timers_unsafe() // time required for executing the timers is longer than their period. auto start = std::chrono::steady_clock::now(); - TimerPtr head = heap_.front(); + + TimerPtr head = timers.front(); + while (head->is_ready() && this->timer_was_ready_at_tp(head, start)) { // Execute head timer head->execute_callback(); // Executing a timer will result in updating its time_until_trigger, so re-heapify - heap_.heapify_root(); + heap_.heapify_root(timers); // Get new head timer - head = heap_.front(); + head = timers.front(); } - auto time_until_trigger = (heap_.front())->time_until_trigger(); - - this->release_timers(); - - return time_until_trigger; + return (timers.front())->time_until_trigger(); } void TimersManager::run_timers() @@ -233,7 +224,7 @@ void TimersManager::clear() { // Lock mutex and then clear all data structures std::unique_lock timers_lock(timers_mutex_); - heap_.clear(); + weak_timers_.clear(); timers_updated_ = true; } From 43487e27f9b2f795ed1bd41c76e38f78a7b82689 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Wed, 13 Jan 2021 12:10:18 -0300 Subject: [PATCH 105/168] Correct ownership timers --- .../rclcpp/executors/timers_manager.hpp | 74 ++++++++++-- .../src/rclcpp/executors/events_executor.cpp | 8 +- .../src/rclcpp/executors/timers_manager.cpp | 111 ++++++++++-------- 3 files changed, 128 insertions(+), 65 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index ae2416b58d..c312329955 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -90,16 +90,14 @@ class TimersManager */ void stop(); - /** - * @brief Acquire ownership of timers and make heap - */ - void own_timers(std::vector & timers); - /** * @brief Executes all the timers currently ready when the function is invoked * while keeping the heap correctly sorted. + * @return std::chrono::nanoseconds for next timer to expire, + * the returned value could be negative if the timer is already expired + * or MAX_TIME if the heap is empty. */ - void execute_ready_timers(); + std::chrono::nanoseconds execute_ready_timers(); /** * @brief Executes one ready timer if available @@ -144,9 +142,52 @@ class TimersManager */ struct TimersHeap { - void make_heap(std::vector & timers) + + void make_heap(std::vector & weak_timers) { + std::vector timers; + + auto it = weak_timers.begin(); + + while (it != weak_timers.end()) { + if (auto timer_shared_ptr = it->lock()) { + // The timer is valid, own it and add to heap + timers.push_back(timer_shared_ptr); + } else { + // The timer went out of scope, remove it + weak_timers.erase(it); + } + it++; + } + std::make_heap(timers.begin(), timers.end(), timer_greater); + + heapified_weak_timers_.clear(); + + for(auto & timer : timers) { + heapified_weak_timers_.push_back(timer); + } + } + + void own_timers_heap(std::vector & timers) + { + auto it = heapified_weak_timers_.begin(); + + while (it != heapified_weak_timers_.end()) { + if (auto timer_shared_ptr = it->lock()) { + // The timer is valid, own it and add to heap + timers.push_back(timer_shared_ptr); + } else { + // The timer went out of scope, remove it + heapified_weak_timers_.erase(it); + } + it++; + } + } + + bool empty() + { + return heapified_weak_timers_.empty(); } /** @@ -156,16 +197,27 @@ class TimersManager { // Push the modified element (i.e. the current root) at the bottom of the heap timers.push_back(timers[0]); - // Exchange first and last elements and reheapify + + // Exchange first and last-1 elements and reheapify std::pop_heap(timers.begin(), timers.end(), timer_greater); + // Remove last element timers.pop_back(); + + heapified_weak_timers_.clear(); + + for(auto & timer : timers) { + heapified_weak_timers_.push_back(timer); + } } + static bool timer_greater(TimerPtr a, TimerPtr b) { return a->time_until_trigger() > b->time_until_trigger(); } + + std::vector heapified_weak_timers_; }; /** @@ -178,11 +230,11 @@ class TimersManager * @brief Executes all the timers currently ready when the function is invoked * while keeping the heap correctly sorted. * This function is not thread safe, acquire a mutex before calling it. - * @return std::chrono::nanoseconds to wait, + * @return std::chrono::nanoseconds for next timer to expire, * the returned value could be negative if the timer is already expired * or MAX_TIME if the heap is empty. */ - std::chrono::nanoseconds execute_ready_timers_unsafe(); + std::chrono::nanoseconds execute_ready_timers_unsafe(TimersHeap & heap); /** * @brief Helper function that checks whether a timer was already ready @@ -212,8 +264,6 @@ class TimersManager std::atomic running_ {false}; // Context of the parent executor std::shared_ptr context_; - // MinHeap of timers - TimersHeap heap_; // Vector of weak pointers to registered timers std::vector weak_timers_; }; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index f5617a1227..d7f46ff35e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -147,14 +147,16 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); } - // Keep executing until work available or timeout expired + auto timeout = timers_manager_->get_head_timeout(); + + // Keep executing until no more work to do or timeout expired while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { std::unique_lock push_lock(push_mutex_); std::unique_lock execution_lock(execution_mutex_); std::swap(execution_event_queue_, event_queue_); push_lock.unlock(); - bool ready_timer = timers_manager_->get_head_timeout() < 0ns; + bool ready_timer = timeout < 0ns; bool has_events = !execution_event_queue_.empty(); if (!ready_timer && !has_events) { // Exit as there is no more work to do @@ -165,7 +167,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) this->consume_all_events(execution_event_queue_); execution_lock.unlock(); - timers_manager_->execute_ready_timers(); + timeout = timers_manager_->execute_ready_timers(); } } diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 19252f51cf..824f19f6e1 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -92,38 +92,7 @@ void TimersManager::stop() } } -void TimersManager::own_timers(std::vector & timers) -{ - auto it = weak_timers_.begin(); - - while (it != weak_timers_.end()) { - if (auto timer_shared_ptr = it->lock()) { - // The timer is valid, own it and add to heap - timers.push_back(timer_shared_ptr); - } else { - // The timer went out of scope, remove it - weak_timers_.erase(it); - } - it++; - } - - heap_.make_heap(timers); -} - -std::chrono::nanoseconds TimersManager::get_head_timeout() -{ - // Vector of pointers to timers used to implement the priority queue - std::vector timers; - this->own_timers(timers); - - if (timers.empty()) { - return MAX_TIME; - } - - return (timers.front())->time_until_trigger(); -} - -void TimersManager::execute_ready_timers() +std::chrono::nanoseconds TimersManager::execute_ready_timers() { // Do not allow to interfere with the thread running if (running_) { @@ -132,7 +101,12 @@ void TimersManager::execute_ready_timers() } std::unique_lock lock(timers_mutex_); - this->execute_ready_timers_unsafe(); + + // Generate heapified weak timers + TimersHeap heap; + heap.make_heap(weak_timers_); + + return this->execute_ready_timers_unsafe(heap); } bool TimersManager::execute_head_timer() @@ -143,21 +117,25 @@ bool TimersManager::execute_head_timer() "TimersManager::execute_head_timer() can't be used while timers thread is running"); } - std::unique_lock lock(timers_mutex_); //needed? - - // Vector of pointers to timers used to implement the priority queue - std::vector timers; + std::unique_lock lock(timers_mutex_); - this->own_timers(timers); + // Generate heapified weak timers + TimersHeap heap; + heap.make_heap(weak_timers_); // Nothing to do if we don't have any timer - if (timers.empty()) { + if (heap.empty()) { return false; } + // Own timers during the following operations + std::vector timers; + heap.own_timers_heap(timers); + TimerPtr head = timers.front(); + if (head->is_ready()) { - // Head timer is ready, execute and re-heapify + // Head timer is ready, execute head->execute_callback(); return true; } else { @@ -166,31 +144,50 @@ bool TimersManager::execute_head_timer() } } -std::chrono::nanoseconds TimersManager::execute_ready_timers_unsafe() +std::chrono::nanoseconds TimersManager::get_head_timeout() { - // Vector of pointers to timers used to implement the priority queue + std::unique_lock lock(timers_mutex_); + + TimersHeap heap; + // Generate heapified weak timers + heap.make_heap(weak_timers_); + + if (heap.empty()) { + return MAX_TIME; + } + + // Own timers during the following operations std::vector timers; + heap.own_timers_heap(timers); - this->own_timers(timers); + return (timers.front())->time_until_trigger(); +} +std::chrono::nanoseconds TimersManager::execute_ready_timers_unsafe(TimersHeap & heap) +{ // Nothing to do if we don't have any timer - if (timers.empty()) { + if (heap.empty()) { return MAX_TIME; } - // Keep executing timers until they are read and they were already ready when we started. - // The second check prevents this function from blocking indefinitely if the - // time required for executing the timers is longer than their period. + // Own timers during the following operations + std::vector timers; + heap.own_timers_heap(timers); + auto start = std::chrono::steady_clock::now(); TimerPtr head = timers.front(); + // Keep executing timers until they are ready and they were already ready when we started. + // The second check prevents this function from blocking indefinitely if the + // time required for executing the timers is longer than their period. + while (head->is_ready() && this->timer_was_ready_at_tp(head, start)) { // Execute head timer head->execute_callback(); // Executing a timer will result in updating its time_until_trigger, so re-heapify - heap_.heapify_root(timers); + heap.heapify_root(timers); // Get new head timer head = timers.front(); } @@ -200,7 +197,16 @@ std::chrono::nanoseconds TimersManager::execute_ready_timers_unsafe() void TimersManager::run_timers() { - auto time_to_sleep = this->get_head_timeout(); // Aquires and releases ownership + auto time_to_sleep = this->get_head_timeout(); + + TimersHeap heap; + + { + // Protect weak_timers_ + std::unique_lock lock(timers_mutex_); + // Generate heapified weak timers + heap.make_heap(weak_timers_); + } while (rclcpp::ok(context_) && running_) { // Lock mutex @@ -208,10 +214,15 @@ void TimersManager::run_timers() // Wait until timeout or notification that timers have been updated timers_cv_.wait_for(timers_lock, time_to_sleep, [this]() {return timers_updated_;}); + + // If timers_updated_, update the heap: + if(timers_updated_) { + heap.make_heap(weak_timers_); + } // Reset timers updated flag timers_updated_ = false; // Execute timers - time_to_sleep = this->execute_ready_timers_unsafe(); + time_to_sleep = this->execute_ready_timers_unsafe(heap); } // Make sure the running flag is set to false when we exit from this function From cc7910910c84a5807c510312f347a53f3f02cf48 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 14 Jan 2021 10:45:29 -0300 Subject: [PATCH 106/168] remove mutex not needed since new ownership model --- rclcpp/include/rclcpp/executors/events_executor.hpp | 2 -- rclcpp/src/rclcpp/executors/events_executor.cpp | 3 --- 2 files changed, 5 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 2ba700ada2..4eea365fe1 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -216,8 +216,6 @@ class EventsExecutor : public rclcpp::Executor EventsExecutorNotifyWaitable::SharedPtr executor_notifier_; // Mutex to protect the insertion of events in the queue std::mutex push_mutex_; - // Mutex to protect the execution of events - std::mutex execution_mutex_; // Variable used to notify when an event is added to the queue std::condition_variable event_queue_cv_; // Timers manager diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index d7f46ff35e..094b7cac7b 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -62,7 +62,6 @@ EventsExecutor::spin() std::unique_lock push_lock(push_mutex_); // We wait here until something has been pushed to the event queue event_queue_cv_.wait(push_lock, has_event_predicate); - std::unique_lock execution_lock(execution_mutex_); // We got an event! Swap queues while we hold both mutexes std::swap(execution_event_queue_, event_queue_); // After swapping the queues, we don't need the push lock anymore @@ -104,7 +103,6 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) // Wait until timeout or event std::unique_lock push_lock(push_mutex_); event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); - std::unique_lock execution_lock(execution_mutex_); std::swap(execution_event_queue_, event_queue_); push_lock.unlock(); this->consume_all_events(execution_event_queue_); @@ -152,7 +150,6 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Keep executing until no more work to do or timeout expired while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { std::unique_lock push_lock(push_mutex_); - std::unique_lock execution_lock(execution_mutex_); std::swap(execution_event_queue_, event_queue_); push_lock.unlock(); From fc903e502d026f2d5cca7d45d15d9bf4574a1781 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 14 Jan 2021 11:20:03 -0300 Subject: [PATCH 107/168] Move using TimerPtr to private section --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 2 +- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index c312329955..8fc7754691 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -56,7 +56,6 @@ namespace executors * so timers may be executed out of order. * */ -using TimerPtr = rclcpp::TimerBase::SharedPtr; class TimersManager { @@ -134,6 +133,7 @@ class TimersManager private: RCLCPP_DISABLE_COPY(TimersManager) + using TimerPtr = rclcpp::TimerBase::SharedPtr; /** * @brief This struct provides convenient access to a MinHeap of timers. diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 094b7cac7b..2d915cce3a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -106,7 +106,6 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) std::swap(execution_event_queue_, event_queue_); push_lock.unlock(); this->consume_all_events(execution_event_queue_); - execution_lock.unlock(); timers_manager_->execute_ready_timers(); } @@ -162,7 +161,6 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Execute all ready work this->consume_all_events(execution_event_queue_); - execution_lock.unlock(); timeout = timers_manager_->execute_ready_timers(); } From 979ca878e27b83aafa4405daa978972a161800c3 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 14 Jan 2021 12:09:55 -0300 Subject: [PATCH 108/168] If a timer was removed, re-heapify vector --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 8fc7754691..5789b018f9 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -172,6 +172,7 @@ class TimersManager void own_timers_heap(std::vector & timers) { auto it = heapified_weak_timers_.begin(); + bool invalid_heap = false; while (it != heapified_weak_timers_.end()) { if (auto timer_shared_ptr = it->lock()) { @@ -180,9 +181,20 @@ class TimersManager } else { // The timer went out of scope, remove it heapified_weak_timers_.erase(it); + invalid_heap = true; } it++; } + + // If the vector of weak timers is not a heap anymore, recreate heap. + if (invalid_heap) { + std::make_heap(timers.begin(), timers.end(), timer_greater); + heapified_weak_timers_.clear(); + + for(auto & timer : timers) { + heapified_weak_timers_.push_back(timer); + } + } } bool empty() From e3b258ac4538f0f3bde9cc40d083d3cbf78a3470 Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Thu, 14 Jan 2021 17:56:30 +0000 Subject: [PATCH 109/168] rework timers manager weak ownership Signed-off-by: Alberto Soragna --- .../rclcpp/executors/events_executor.hpp | 12 +- .../rclcpp/executors/timers_manager.hpp | 268 ++++++++++++++---- .../src/rclcpp/executors/events_executor.cpp | 54 ++-- .../src/rclcpp/executors/timers_manager.cpp | 155 ++++------ .../rclcpp/executors/test_timers_manager.cpp | 1 + .../src/lifecycle_node_interface_impl.hpp | 4 +- 6 files changed, 313 insertions(+), 181 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 4eea365fe1..df63a96535 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -16,8 +16,8 @@ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_HPP_ #include -#include #include +#include #include #include "rclcpp/executor.hpp" @@ -174,9 +174,7 @@ class EventsExecutor : public rclcpp::Executor private: RCLCPP_DISABLE_COPY(EventsExecutor) - // Event queue implementation is a deque only to - // facilitate the removal of events from expired entities. - using EventQueue = std::deque; + using EventQueue = std::queue; // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, @@ -192,7 +190,7 @@ class EventsExecutor : public rclcpp::Executor { std::unique_lock lock(this_executor->push_mutex_); - this_executor->event_queue_.push_back(event); + this_executor->event_queue_.push(event); } // Notify that the event queue has some events in it. this_executor->event_queue_cv_.notify_one(); @@ -208,12 +206,12 @@ class EventsExecutor : public rclcpp::Executor void execute_event(const rmw_listener_event_t & event); - // We use two instances of EventQueue to allow threads to push events while we execute them + // Queue where entities can push events EventQueue event_queue_; - EventQueue execution_event_queue_; EventsExecutorEntitiesCollector::SharedPtr entities_collector_; EventsExecutorNotifyWaitable::SharedPtr executor_notifier_; + // Mutex to protect the insertion of events in the queue std::mutex push_mutex_; // Variable used to notify when an event is added to the queue diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index c312329955..7822b2be7c 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -56,8 +56,6 @@ namespace executors * so timers may be executed out of order. * */ -using TimerPtr = rclcpp::TimerBase::SharedPtr; - class TimersManager { public: @@ -134,90 +132,247 @@ class TimersManager private: RCLCPP_DISABLE_COPY(TimersManager) + using TimerPtr = rclcpp::TimerBase::SharedPtr; + using WeakTimerPtr = rclcpp::TimerBase::WeakPtr; + + // Forward declaration + class TimersHeap; /** - * @brief This struct provides convenient access to a MinHeap of timers. + * @brief This class allows to store weak pointers to timers in a heap data structure. + * The "validate_and_lock" API allows to get ownership of the timers and also makes sure that + * the heap property is respected. * The root of the heap is the timer that expires first. - * This struct is not thread safe and requires external mutexes to protect its usage. + * This class is not thread safe and requires external mutexes to protect its usage. */ - struct TimersHeap + class WeakTimersHeap { + public: + /** + * @brief Try to add a new timer to the heap. + * After the addition, the heap property is preserved. + * @param timer new timer to add + * @return true if timer has been added, false if it was already there + */ + bool add_timer(TimerPtr timer) + { + TimersHeap locked_heap = this->validate_and_lock(); + bool added = locked_heap.add_timer(std::move(timer)); + + if (added) { + // Re-create the weak heap with the new timer added + this->store(locked_heap); + } + + return added; + } - void make_heap(std::vector & weak_timers) + /** + * @brief Try to remove a timer from the heap. + * After the removal, the heap property is preserved. + * @param timer timer to remove + * @return true if timer has been removed, false if it was not there + */ + bool remove_timer(TimerPtr timer) { - std::vector timers; + TimersHeap locked_heap = this->validate_and_lock(); + bool removed = locked_heap.remove_timer(std::move(timer)); + + if (removed) { + // Re-create the weak heap with the timer removed + this->store(locked_heap); + } + + return removed; + } - auto it = weak_timers.begin(); + /** + * @brief This function restores the current object as a valid heap + * and it also returns a locked version of it + * @return TimersHeap owned timers corresponding to the current object + */ + TimersHeap validate_and_lock() + { + TimersHeap locked_heap; + bool any_timer_destroyed = false; - while (it != weak_timers.end()) { + auto it = weak_heap_.begin(); + auto end = weak_heap_.end(); + while (it != end) { if (auto timer_shared_ptr = it->lock()) { - // The timer is valid, own it and add to heap - timers.push_back(timer_shared_ptr); + // This timer is valid, add it to the vector + locked_heap.push_back(std::move(timer_shared_ptr)); + it++; } else { - // The timer went out of scope, remove it - weak_timers.erase(it); + // This timer went out of scope, remove it + it = weak_heap_.erase(it); + any_timer_destroyed = true; } - it++; + } + + // If a timer has gone out of scope, then the remaining elements may not represent + // a valid heap anymore. We need to re heapify the timers heap. + if (any_timer_destroyed) { + locked_heap.heapify(); + // Re-create the weak heap now that elements have been heapified again + this->store(locked_heap); } - std::make_heap(timers.begin(), timers.end(), timer_greater); - - heapified_weak_timers_.clear(); + return locked_heap; + } - for(auto & timer : timers) { - heapified_weak_timers_.push_back(timer); + /** + * @brief This function allows to recreate the heap of weak pointers + * from an heap of owned pointers. + * @param heap + */ + void store(const TimersHeap & heap) + { + weak_heap_.clear(); + for (auto t : heap.owned_heap_) { + weak_heap_.push_back(t); } } - void own_timers_heap(std::vector & timers) + /** + * @brief Remove all timers from the heap. + */ + void clear() { - auto it = heapified_weak_timers_.begin(); + weak_heap_.clear(); + } - while (it != heapified_weak_timers_.end()) { - if (auto timer_shared_ptr = it->lock()) { - // The timer is valid, own it and add to heap - timers.push_back(timer_shared_ptr); - } else { - // The timer went out of scope, remove it - heapified_weak_timers_.erase(it); - } - it++; + private: + std::vector weak_heap_; + }; + + /** + * @brief This class is the equivalent of WeakTimersHeap but with ownership of the timers. + * It can be generated by locking the weak version. + * It provides operations to manipulate the heap + */ + class TimersHeap + { + public: + /** + * @brief Try to add a new timer to the heap. + * After the addition, the heap property is preserved. + * @param timer new timer to add + * @return true if timer has been added, false if it was already there + */ + bool add_timer(TimerPtr timer) + { + // Nothing to do if the timer is already stored here + auto it = std::find(owned_heap_.begin(), owned_heap_.end(), timer); + if (it != owned_heap_.end()) { + return false; } + + owned_heap_.push_back(std::move(timer)); + std::push_heap(owned_heap_.begin(), owned_heap_.end(), timer_greater); + + return true; } - bool empty() + /** + * @brief Try to remove a timer from the heap. + * After the removal, the heap property is preserved. + * @param timer timer to remove + * @return true if timer has been removed, false if it was not there + */ + bool remove_timer(TimerPtr timer) { - return heapified_weak_timers_.empty(); + // Nothing to do if the timer is not stored here + auto it = std::find(owned_heap_.begin(), owned_heap_.end(), timer); + if (it == owned_heap_.end()) { + return false; + } + + owned_heap_.erase(it); + std::make_heap(owned_heap_.begin(), owned_heap_.end(), timer_greater); + + return true; } /** - * @brief Restore a valid heap after the root value is replaced. + * @brief Returns a reference to the front element + */ + TimerPtr & front() + { + return owned_heap_.front(); + } + + /** + * @brief Returns a const reference to the front element + */ + const TimerPtr & front() const + { + return owned_heap_.front(); + } + + /** + * @brief Returns whether the heap is empty or not + */ + bool empty() const + { + return owned_heap_.empty(); + } + + /** + * @brief Restore a valid heap after the root value has been replaced. */ - void heapify_root(std::vector & timers) + void heapify_root() { - // Push the modified element (i.e. the current root) at the bottom of the heap - timers.push_back(timers[0]); + // The following code is a more efficient version of doing + // - pop_heap; pop_back; + // - push_back; push_heap; + // as it removes the need for the last push_heap + // Push the modified element (i.e. the current root) at the bottom of the heap + owned_heap_.push_back(owned_heap_[0]); // Exchange first and last-1 elements and reheapify - std::pop_heap(timers.begin(), timers.end(), timer_greater); - + std::pop_heap(owned_heap_.begin(), owned_heap_.end(), timer_greater); // Remove last element - timers.pop_back(); + owned_heap_.pop_back(); + } - heapified_weak_timers_.clear(); + /** + * @brief Completely restores the structure to a valid heap + */ + void heapify() + { + std::make_heap(owned_heap_.begin(), owned_heap_.end(), timer_greater); + } - for(auto & timer : timers) { - heapified_weak_timers_.push_back(timer); - } + /** + * @brief Convenience function that allows to push an element at the bottom of the heap. + * It will not perform any check on whether the heap remains valid or not. + * Those checks are responsibility of the calling code. + * + * @param timer timer to push at the back of the data structure representing the heap + */ + void push_back(TimerPtr timer) + { + owned_heap_.push_back(timer); } + /** + * @brief Friend declaration to allow the store function to access the underlying + * heap container + */ + friend void WeakTimersHeap::store(const TimersHeap& heap); + + private: + /** + * @brief Comparison function between timers. + */ static bool timer_greater(TimerPtr a, TimerPtr b) { return a->time_until_trigger() > b->time_until_trigger(); } - std::vector heapified_weak_timers_; + std::vector owned_heap_; }; /** @@ -227,14 +382,27 @@ class TimersManager void run_timers(); /** - * @brief Executes all the timers currently ready when the function is invoked - * while keeping the heap correctly sorted. + * @brief Get the amount of time before the next timer expires. * This function is not thread safe, acquire a mutex before calling it. - * @return std::chrono::nanoseconds for next timer to expire, + * + * @return std::chrono::nanoseconds to wait, * the returned value could be negative if the timer is already expired * or MAX_TIME if the heap is empty. */ - std::chrono::nanoseconds execute_ready_timers_unsafe(TimersHeap & heap); + std::chrono::nanoseconds get_head_timeout_unsafe(const TimersHeap & heap) + { + if (heap.empty()) { + return MAX_TIME; + } + return (heap.front())->time_until_trigger(); + } + + /** + * @brief Executes all the timers currently ready when the function is invoked + * while keeping the heap correctly sorted. + * This function is not thread safe, acquire a mutex before calling it. + */ + void execute_ready_timers_unsafe(TimersHeap & heap); /** * @brief Helper function that checks whether a timer was already ready @@ -264,8 +432,8 @@ class TimersManager std::atomic running_ {false}; // Context of the parent executor std::shared_ptr context_; - // Vector of weak pointers to registered timers - std::vector weak_timers_; + // Timers heap with weak ownership + WeakTimersHeap weak_timers_heap_; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 094b7cac7b..14762e359e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -56,18 +56,21 @@ EventsExecutor::spin() // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !event_queue_.empty();}; + // Local event queue to allow entities to push events while we execute them + EventQueue execution_event_queue; + timers_manager_->start(); while (rclcpp::ok(context_) && spinning.load()) { std::unique_lock push_lock(push_mutex_); // We wait here until something has been pushed to the event queue event_queue_cv_.wait(push_lock, has_event_predicate); - // We got an event! Swap queues while we hold both mutexes - std::swap(execution_event_queue_, event_queue_); - // After swapping the queues, we don't need the push lock anymore + // We got an event! Swap queues + std::swap(execution_event_queue, event_queue_); + // After swapping the queues, we don't need the lock anymore push_lock.unlock(); - // Consume events while under the execution lock only - this->consume_all_events(execution_event_queue_); + // Consume all available events, this queue will be empty at the end of the function + this->consume_all_events(execution_event_queue); } timers_manager_->stop(); } @@ -93,22 +96,27 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !event_queue_.empty();}; + // Local event queue to allow entities to push events while we execute them + EventQueue execution_event_queue; + // Select the smallest between input max_duration and timer timeout auto next_timer_timeout = timers_manager_->get_head_timeout(); if (next_timer_timeout < max_duration) { max_duration = next_timer_timeout; } - - // Wait until timeout or event std::unique_lock push_lock(push_mutex_); + // Wait until timeout or event event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); - std::swap(execution_event_queue_, event_queue_); + // Time to swap queues as the wait is over + std::swap(execution_event_queue, event_queue_); + // After swapping the queues, we don't need the lock anymore push_lock.unlock(); - this->consume_all_events(execution_event_queue_); - execution_lock.unlock(); + // Execute all ready timers timers_manager_->execute_ready_timers(); + // Consume all available events, this queue will be empty at the end of the function + this->consume_all_events(execution_event_queue); } void @@ -126,21 +134,23 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !event_queue_.empty();}; + // Local event queue to allow entities to push events while we execute them + EventQueue execution_event_queue; + auto start = std::chrono::steady_clock::now(); auto max_duration_not_elapsed = [max_duration, start]() { auto elapsed_time = std::chrono::steady_clock::now() - start; return elapsed_time < max_duration; }; - // Wait once - // Select the smallest between input timeout and timer timeout + // Select the smallest between input max duration and timer timeout auto next_timer_timeout = timers_manager_->get_head_timeout(); if (next_timer_timeout < max_duration) { max_duration = next_timer_timeout; } { - // Wait until timeout or event + // Wait once until timeout or event std::unique_lock push_lock(push_mutex_); event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); } @@ -150,21 +160,19 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Keep executing until no more work to do or timeout expired while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { std::unique_lock push_lock(push_mutex_); - std::swap(execution_event_queue_, event_queue_); + std::swap(execution_event_queue, event_queue_); push_lock.unlock(); - bool ready_timer = timeout < 0ns; - bool has_events = !execution_event_queue_.empty(); + // Exit if there is no more work to do + const bool ready_timer = timeout < 0ns; + const bool has_events = !execution_event_queue.empty(); if (!ready_timer && !has_events) { - // Exit as there is no more work to do break; } - // Execute all ready work - - this->consume_all_events(execution_event_queue_); - execution_lock.unlock(); + // Execute all ready work timeout = timers_manager_->execute_ready_timers(); + this->consume_all_events(execution_event_queue); } } @@ -197,7 +205,7 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) has_event = !event_queue_.empty(); if (has_event) { event = event_queue_.front(); - event_queue_.pop_front(); + event_queue_.pop(); } } @@ -252,7 +260,7 @@ EventsExecutor::consume_all_events(EventQueue & event_queue) { while (!event_queue.empty()) { rmw_listener_event_t event = event_queue.front(); - event_queue.pop_front(); + event_queue.pop(); this->execute_event(event); } diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 824f19f6e1..7e29ec0f23 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -41,24 +41,17 @@ void TimersManager::add_timer(rclcpp::TimerBase::SharedPtr timer) throw std::invalid_argument("TimersManager::add_timer() trying to add nullptr timer"); } + bool added = false; { std::unique_lock timers_lock(timers_mutex_); - - // Nothing to do if the timer is already stored here - for(const auto & weak_timer : weak_timers_) { - bool matched = (weak_timer.lock() == timer); - - if (matched) { - return; - } - } - - weak_timers_.push_back(timer); - timers_updated_ = true; + added = weak_timers_heap_.add_timer(timer); + timers_updated_ = timers_updated_ || added; } - // Notify that a timer has been added to the vector of weak timers - timers_cv_.notify_one(); + if (added) { + // Notify that a timer has been added + timers_cv_.notify_one(); + } } void TimersManager::start() @@ -80,6 +73,7 @@ void TimersManager::stop() return; } + // Notify the timers manager thread to wake up { std::unique_lock timers_lock(timers_mutex_); timers_updated_ = true; @@ -92,6 +86,19 @@ void TimersManager::stop() } } +std::chrono::nanoseconds TimersManager::get_head_timeout() +{ + // Do not allow to interfere with the thread running + if (running_) { + throw std::runtime_error( + "TimersManager::get_head_timeout() can't be used while timers thread is running"); + } + + std::unique_lock lock(timers_mutex_); + TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); + return this->get_head_timeout_unsafe(timers_heap); +} + std::chrono::nanoseconds TimersManager::execute_ready_timers() { // Do not allow to interfere with the thread running @@ -102,11 +109,11 @@ std::chrono::nanoseconds TimersManager::execute_ready_timers() std::unique_lock lock(timers_mutex_); - // Generate heapified weak timers - TimersHeap heap; - heap.make_heap(weak_timers_); + TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); + this->execute_ready_timers_unsafe(timers_heap); + weak_timers_heap_.store(timers_heap); - return this->execute_ready_timers_unsafe(heap); + return this->get_head_timeout_unsafe(timers_heap); } bool TimersManager::execute_head_timer() @@ -119,24 +126,18 @@ bool TimersManager::execute_head_timer() std::unique_lock lock(timers_mutex_); - // Generate heapified weak timers - TimersHeap heap; - heap.make_heap(weak_timers_); - + TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); // Nothing to do if we don't have any timer - if (heap.empty()) { + if (timers_heap.empty()) { return false; } - // Own timers during the following operations - std::vector timers; - heap.own_timers_heap(timers); - - TimerPtr head = timers.front(); - + TimerPtr head = timers_heap.front(); if (head->is_ready()) { // Head timer is ready, execute head->execute_callback(); + timers_heap.heapify_root(); + weak_timers_heap_.store(timers_heap); return true; } else { // Head timer was not ready yet @@ -144,68 +145,36 @@ bool TimersManager::execute_head_timer() } } -std::chrono::nanoseconds TimersManager::get_head_timeout() -{ - std::unique_lock lock(timers_mutex_); - - TimersHeap heap; - // Generate heapified weak timers - heap.make_heap(weak_timers_); - - if (heap.empty()) { - return MAX_TIME; - } - - // Own timers during the following operations - std::vector timers; - heap.own_timers_heap(timers); - - return (timers.front())->time_until_trigger(); -} - -std::chrono::nanoseconds TimersManager::execute_ready_timers_unsafe(TimersHeap & heap) +void TimersManager::execute_ready_timers_unsafe(TimersHeap & heap) { // Nothing to do if we don't have any timer if (heap.empty()) { - return MAX_TIME; + return; } - // Own timers during the following operations - std::vector timers; - heap.own_timers_heap(timers); - - - auto start = std::chrono::steady_clock::now(); - - TimerPtr head = timers.front(); - // Keep executing timers until they are ready and they were already ready when we started. // The second check prevents this function from blocking indefinitely if the // time required for executing the timers is longer than their period. - while (head->is_ready() && this->timer_was_ready_at_tp(head, start)) { + TimerPtr head = heap.front(); + auto start_time = std::chrono::steady_clock::now(); + while (head->is_ready() && this->timer_was_ready_at_tp(head, start_time)) { // Execute head timer head->execute_callback(); // Executing a timer will result in updating its time_until_trigger, so re-heapify - heap.heapify_root(timers); + heap.heapify_root(); // Get new head timer - head = timers.front(); + head = heap.front(); } - - return (timers.front())->time_until_trigger(); } void TimersManager::run_timers() { - auto time_to_sleep = this->get_head_timeout(); - - TimersHeap heap; - + std::chrono::nanoseconds time_to_sleep; { - // Protect weak_timers_ std::unique_lock lock(timers_mutex_); - // Generate heapified weak timers - heap.make_heap(weak_timers_); + TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); + time_to_sleep = this->get_head_timeout_unsafe(timers_heap); } while (rclcpp::ok(context_) && running_) { @@ -214,15 +183,17 @@ void TimersManager::run_timers() // Wait until timeout or notification that timers have been updated timers_cv_.wait_for(timers_lock, time_to_sleep, [this]() {return timers_updated_;}); - - // If timers_updated_, update the heap: - if(timers_updated_) { - heap.make_heap(weak_timers_); - } // Reset timers updated flag timers_updated_ = false; + + // Get ownership of timers + TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); // Execute timers - time_to_sleep = this->execute_ready_timers_unsafe(heap); + this->execute_ready_timers_unsafe(timers_heap); + // Store updated order of elements to efficiently re-use it next iteration + weak_timers_heap_.store(timers_heap); + // Get next timeout + time_to_sleep = this->get_head_timeout_unsafe(timers_heap); } // Make sure the running flag is set to false when we exit from this function @@ -235,7 +206,7 @@ void TimersManager::clear() { // Lock mutex and then clear all data structures std::unique_lock timers_lock(timers_mutex_); - weak_timers_.clear(); + weak_timers_heap_.clear(); timers_updated_ = true; } @@ -246,30 +217,16 @@ void TimersManager::clear() void TimersManager::remove_timer(TimerPtr timer) { + bool removed = false; { std::unique_lock timers_lock(timers_mutex_); + removed = weak_timers_heap_.remove_timer(timer); - bool matched = false; - auto it = weak_timers_.begin(); - - while (it != weak_timers_.end()) { - matched = (it->lock() == timer); - - if (matched) { - weak_timers_.erase(it); - break; - } - - it++; - } - - if ((it == weak_timers_.end()) && !matched ) { - throw std::runtime_error("Tried to remove timer not stored in the timers manager."); - } - - timers_updated_ = true; + timers_updated_ = timers_updated_ || removed; } - // Notify timers thread such that it can re-compute its timeout - timers_cv_.notify_one(); + if (removed) { + // Notify timers thread such that it can re-compute its timeout + timers_cv_.notify_one(); + } } diff --git a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp index 0a3b8e44d7..ccafcd9c47 100644 --- a/rclcpp/test/rclcpp/executors/test_timers_manager.cpp +++ b/rclcpp/test/rclcpp/executors/test_timers_manager.cpp @@ -143,6 +143,7 @@ TEST_F(TestTimersManager, timers_thread_exclusive_usage) timers_manager->start(); + EXPECT_THROW(timers_manager->start(), std::exception); EXPECT_THROW(timers_manager->get_head_timeout(), std::exception); EXPECT_THROW(timers_manager->execute_ready_timers(), std::exception); EXPECT_THROW(timers_manager->execute_head_timer(), std::exception); diff --git a/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp b/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp index 9fc21e5171..df851ad94c 100644 --- a/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp +++ b/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp @@ -490,7 +490,7 @@ class LifecycleNode::LifecycleNodeInterfaceImpl void add_timer_handle(std::shared_ptr timer) { - weak_timers_.push_back(timer); + weak_timers_heap_.push_back(timer); } rcl_lifecycle_state_machine_t state_machine_; @@ -520,7 +520,7 @@ class LifecycleNode::LifecycleNodeInterfaceImpl // to controllable things std::vector> weak_pubs_; - std::vector> weak_timers_; + std::vector> weak_timers_heap_; }; } // namespace rclcpp_lifecycle From 08914c49cfb65093582c8eb6df1e226d4cc978d3 Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Thu, 14 Jan 2021 17:58:42 +0000 Subject: [PATCH 110/168] undo wrong replace Signed-off-by: Alberto Soragna --- rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp b/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp index df851ad94c..9fc21e5171 100644 --- a/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp +++ b/rclcpp_lifecycle/src/lifecycle_node_interface_impl.hpp @@ -490,7 +490,7 @@ class LifecycleNode::LifecycleNodeInterfaceImpl void add_timer_handle(std::shared_ptr timer) { - weak_timers_heap_.push_back(timer); + weak_timers_.push_back(timer); } rcl_lifecycle_state_machine_t state_machine_; @@ -520,7 +520,7 @@ class LifecycleNode::LifecycleNodeInterfaceImpl // to controllable things std::vector> weak_pubs_; - std::vector> weak_timers_heap_; + std::vector> weak_timers_; }; } // namespace rclcpp_lifecycle From ba7e72f2a35d90f67cbf5f17a363170498488531 Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Thu, 14 Jan 2021 19:40:05 +0000 Subject: [PATCH 111/168] cleanup linter errors Signed-off-by: Alberto Soragna --- .../events_executor_entities_collector.hpp | 1 + .../rclcpp/executors/timers_manager.hpp | 28 +++++++++---------- .../src/rclcpp/executors/events_executor.cpp | 8 +++--- .../events_executor_entities_collector.cpp | 8 +++--- 4 files changed, 23 insertions(+), 22 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 2a37c43b8a..9f26251b3a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include "rclcpp/executors/event_waitable.hpp" diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 7822b2be7c..aa5988486d 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include "rclcpp/context.hpp" @@ -147,7 +148,7 @@ class TimersManager */ class WeakTimersHeap { - public: +public: /** * @brief Try to add a new timer to the heap. * After the addition, the heap property is preserved. @@ -177,12 +178,12 @@ class TimersManager { TimersHeap locked_heap = this->validate_and_lock(); bool removed = locked_heap.remove_timer(std::move(timer)); - + if (removed) { // Re-create the weak heap with the timer removed this->store(locked_heap); } - + return removed; } @@ -208,7 +209,7 @@ class TimersManager it = weak_heap_.erase(it); any_timer_destroyed = true; } - } + } // If a timer has gone out of scope, then the remaining elements may not represent // a valid heap anymore. We need to re heapify the timers heap. @@ -218,13 +219,13 @@ class TimersManager this->store(locked_heap); } - return locked_heap; + return locked_heap; } /** * @brief This function allows to recreate the heap of weak pointers * from an heap of owned pointers. - * @param heap + * @param heap timers heap to store as weak pointers */ void store(const TimersHeap & heap) { @@ -242,7 +243,7 @@ class TimersManager weak_heap_.clear(); } - private: +private: std::vector weak_heap_; }; @@ -253,7 +254,7 @@ class TimersManager */ class TimersHeap { - public: +public: /** * @brief Try to add a new timer to the heap. * After the addition, the heap property is preserved. @@ -291,7 +292,7 @@ class TimersManager owned_heap_.erase(it); std::make_heap(owned_heap_.begin(), owned_heap_.end(), timer_greater); - return true; + return true; } /** @@ -341,14 +342,14 @@ class TimersManager */ void heapify() { - std::make_heap(owned_heap_.begin(), owned_heap_.end(), timer_greater); + std::make_heap(owned_heap_.begin(), owned_heap_.end(), timer_greater); } /** * @brief Convenience function that allows to push an element at the bottom of the heap. * It will not perform any check on whether the heap remains valid or not. * Those checks are responsibility of the calling code. - * + * * @param timer timer to push at the back of the data structure representing the heap */ void push_back(TimerPtr timer) @@ -360,10 +361,9 @@ class TimersManager * @brief Friend declaration to allow the store function to access the underlying * heap container */ - friend void WeakTimersHeap::store(const TimersHeap& heap); - - private: + friend void WeakTimersHeap::store(const TimersHeap & heap); +private: /** * @brief Comparison function between timers. */ diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 14762e359e..37415c0c9a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -274,7 +274,7 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) { auto subscription = entities_collector_->get_subscription(event.entity); - if(subscription) { + if (subscription) { execute_subscription(subscription); } break; @@ -284,7 +284,7 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) { auto service = entities_collector_->get_service(event.entity); - if(service) { + if (service) { execute_service(service); } break; @@ -294,7 +294,7 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) { auto client = entities_collector_->get_client(event.entity); - if(client) { + if (client) { execute_client(client); } break; @@ -304,7 +304,7 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) { auto waitable = entities_collector_->get_waitable(event.entity); - if(waitable) { + if (waitable) { waitable->execute(); } break; diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 07fadd38a7..5919d0504a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -506,7 +506,7 @@ EventsExecutorEntitiesCollector::get_subscription(const void * subscription_id) auto subscription_weak_ptr = it->second; auto subscription_shared_ptr = subscription_weak_ptr.lock(); - if(subscription_shared_ptr) { + if (subscription_shared_ptr) { return subscription_shared_ptr; } @@ -525,7 +525,7 @@ EventsExecutorEntitiesCollector::get_client(const void * client_id) auto client_weak_ptr = it->second; auto client_shared_ptr = client_weak_ptr.lock(); - if(client_shared_ptr) { + if (client_shared_ptr) { return client_shared_ptr; } @@ -544,7 +544,7 @@ EventsExecutorEntitiesCollector::get_service(const void * service_id) auto service_weak_ptr = it->second; auto service_shared_ptr = service_weak_ptr.lock(); - if(service_shared_ptr) { + if (service_shared_ptr) { return service_shared_ptr; } @@ -563,7 +563,7 @@ EventsExecutorEntitiesCollector::get_waitable(const void * waitable_id) auto waitable_weak_ptr = it->second; auto waitable_shared_ptr = waitable_weak_ptr.lock(); - if(waitable_shared_ptr) { + if (waitable_shared_ptr) { return waitable_shared_ptr; } From 613e68e1b2666b451fefae65150e10e9f8ef07cd Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 14 Jan 2021 18:30:57 -0300 Subject: [PATCH 112/168] Revert "If a timer was removed, re-heapify vector" This reverts commit 979ca878e27b83aafa4405daa978972a161800c3. --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 5789b018f9..8fc7754691 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -172,7 +172,6 @@ class TimersManager void own_timers_heap(std::vector & timers) { auto it = heapified_weak_timers_.begin(); - bool invalid_heap = false; while (it != heapified_weak_timers_.end()) { if (auto timer_shared_ptr = it->lock()) { @@ -181,20 +180,9 @@ class TimersManager } else { // The timer went out of scope, remove it heapified_weak_timers_.erase(it); - invalid_heap = true; } it++; } - - // If the vector of weak timers is not a heap anymore, recreate heap. - if (invalid_heap) { - std::make_heap(timers.begin(), timers.end(), timer_greater); - heapified_weak_timers_.clear(); - - for(auto & timer : timers) { - heapified_weak_timers_.push_back(timer); - } - } } bool empty() From 5fdf5a29cf10def1086be6ddabf17ceb2703ef78 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 14 Jan 2021 18:31:10 -0300 Subject: [PATCH 113/168] Revert "Move using TimerPtr to private section" This reverts commit fc903e502d026f2d5cca7d45d15d9bf4574a1781. --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 2 +- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 8fc7754691..c312329955 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -56,6 +56,7 @@ namespace executors * so timers may be executed out of order. * */ +using TimerPtr = rclcpp::TimerBase::SharedPtr; class TimersManager { @@ -133,7 +134,6 @@ class TimersManager private: RCLCPP_DISABLE_COPY(TimersManager) - using TimerPtr = rclcpp::TimerBase::SharedPtr; /** * @brief This struct provides convenient access to a MinHeap of timers. diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 2d915cce3a..094b7cac7b 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -106,6 +106,7 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) std::swap(execution_event_queue_, event_queue_); push_lock.unlock(); this->consume_all_events(execution_event_queue_); + execution_lock.unlock(); timers_manager_->execute_ready_timers(); } @@ -161,6 +162,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Execute all ready work this->consume_all_events(execution_event_queue_); + execution_lock.unlock(); timeout = timers_manager_->execute_ready_timers(); } From 51b54507c378c1720a48470b3b876b882e424e56 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 15 Jan 2021 11:44:21 -0300 Subject: [PATCH 114/168] EventsExecutorEntitiesCollector: Use shared_from_this() --- .../executors/events_executor_entities_collector.hpp | 10 +++++++++- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 +- .../executors/events_executor_entities_collector.cpp | 7 +++++++ 3 files changed, 17 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 9f26251b3a..6b7b8de531 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -49,7 +49,9 @@ class EventsExecutor; * When this occurs, the execute API takes care of handling changes * in the entities currently used by the executor. */ -class EventsExecutorEntitiesCollector final : public EventWaitable +class EventsExecutorEntitiesCollector final +: public EventWaitable, + public std::enable_shared_from_this { public: RCLCPP_SMART_PTR_DEFINITIONS(EventsExecutorEntitiesCollector) @@ -63,6 +65,10 @@ class EventsExecutorEntitiesCollector final : public EventWaitable RCLCPP_PUBLIC ~EventsExecutorEntitiesCollector(); + // Initialize entities collector + RCLCPP_PUBLIC + void init(); + // The purpose of "execute" is handling the situation of a new entity added to // a node, while the executor is already spinning. // This consists in setting that entitiy's callback. @@ -237,6 +243,8 @@ class EventsExecutorEntitiesCollector final : public EventWaitable std::list weak_nodes_; // Maps: entity identifiers to weak pointers from the entities registered in the executor + // so in the case of an event providing and ID, we can retrieve and own the corresponding + // entity while it performs work std::unordered_map weak_subscriptions_map_; std::unordered_map weak_services_map_; std::unordered_map weak_clients_map_; diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 37415c0c9a..31b99abbaa 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -31,6 +31,7 @@ EventsExecutor::EventsExecutor( { timers_manager_ = std::make_shared(context_); entities_collector_ = std::make_shared(this); + entities_collector_->init(); // This API uses the wait_set only as a token to identify different executors. auto context_interrupt_gc = options.context->get_interrupt_guard_condition(&wait_set_); @@ -42,7 +43,6 @@ EventsExecutor::EventsExecutor( executor_notifier_->add_guard_condition(&interrupt_guard_condition_); executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); entities_collector_->add_waitable(executor_notifier_); - entities_collector_->add_waitable(entities_collector_); } void diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 5919d0504a..7d6ce46131 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -81,6 +81,13 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() weak_groups_to_nodes_associated_with_executor_.clear(); } +void +EventsExecutorEntitiesCollector::init() +{ + // Add the EventsExecutorEntitiesCollector shared_ptr to waitables map + add_waitable(this->shared_from_this()); +} + void EventsExecutorEntitiesCollector::add_node( rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_ptr) From e7a4377a1bf908753c91edcd9d525f2eb0944b2a Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Fri, 15 Jan 2021 15:25:49 +0000 Subject: [PATCH 115/168] remove some unnecessary diffs Signed-off-by: Alberto Soragna --- rclcpp/src/rclcpp/executor.cpp | 3 ++- rclcpp/src/rclcpp/timer.cpp | 3 +-- .../executors/test_events_executor_entities_collector.cpp | 2 +- rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp | 3 ++- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/rclcpp/src/rclcpp/executor.cpp b/rclcpp/src/rclcpp/executor.cpp index bc046be04c..d33df58ee2 100644 --- a/rclcpp/src/rclcpp/executor.cpp +++ b/rclcpp/src/rclcpp/executor.cpp @@ -652,7 +652,8 @@ Executor::execute_service(rclcpp::ServiceBase::SharedPtr service) } void -Executor::execute_client(rclcpp::ClientBase::SharedPtr client) +Executor::execute_client( + rclcpp::ClientBase::SharedPtr client) { auto request_header = client->create_request_header(); std::shared_ptr response = client->create_response(); diff --git a/rclcpp/src/rclcpp/timer.cpp b/rclcpp/src/rclcpp/timer.cpp index e312fe84eb..70414f38c6 100644 --- a/rclcpp/src/rclcpp/timer.cpp +++ b/rclcpp/src/rclcpp/timer.cpp @@ -71,8 +71,7 @@ TimerBase::TimerBase( } TimerBase::~TimerBase() -{ -} +{} void TimerBase::cancel() diff --git a/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp b/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp index 1860ffd2c1..3f11f983fe 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor_entities_collector.cpp @@ -157,7 +157,7 @@ TEST_F(TestEventsExecutorEntitiesCollector, remove_node_opposite_order) EXPECT_NO_THROW(entities_collector_->remove_node(node2->get_node_base_interface())); } -TEST_F(TestEventsExecutorEntitiesCollector, test_fancy_name) +TEST_F(TestEventsExecutorEntitiesCollector, test_rcl_exception) { auto node1 = std::make_shared("node1", "ns"); auto node2 = std::make_shared("node2", "ns"); diff --git a/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp b/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp index 7da8577963..0d9da31613 100644 --- a/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp +++ b/rclcpp/test/rclcpp/test_add_callback_groups_to_executor.cpp @@ -186,7 +186,8 @@ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_duplicate_callback_groups) */ TYPED_TEST(TestAddCallbackGroupsToExecutor, add_callback_groups_after_add_node_to_executor) { - rclcpp::executors::EventsExecutor executor; + using ExecutorType = TypeParam; + ExecutorType executor; auto node = std::make_shared("my_node", "/ns"); executor.add_node(node->get_node_base_interface()); ASSERT_EQ(executor.get_all_callback_groups().size(), 1u); From 176d479b6383ecb44719ba256bbaa0d5054157de Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 15 Jan 2021 15:06:20 -0300 Subject: [PATCH 116/168] Update timers_manager.hpp documentation --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index aa5988486d..e781a073a5 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -40,7 +40,8 @@ namespace executors * * Timers management * This class provides APIs to add and remove timers. - * This class keeps ownership of the added timers through a shared pointer. + * It keeps a list of weak pointers from added timers, and owns them only when + * have expired and need to be executed. * Timers are kept ordered in a binary-heap priority queue. * Calls to add/remove APIs will temporarily block the execution of the timers and * will require to reorder the internal priority queue of timers. From f04d13c143733d57a8f8edf97f288fa6aa32b381 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 15 Jan 2021 15:15:45 -0300 Subject: [PATCH 117/168] Update doc --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index e781a073a5..7fc47ec095 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -74,8 +74,9 @@ class TimersManager ~TimersManager(); /** - * @brief Adds a new TimerBase to the storage. - * This object will keep ownership of the timer. + * @brief Adds a new TimerBase::WeakPtr to the storage. + * This object will store a weak pointer of the added timer + * in a heap data structure. * @param timer the timer to be added */ void add_timer(rclcpp::TimerBase::SharedPtr timer); From 4603352a9ce83aba6fb680fcf6086a036f0b0d97 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 08:54:28 -0300 Subject: [PATCH 118/168] update APIs --- rclcpp/include/rclcpp/executors/event_waitable.hpp | 7 +++++++ .../executors/events_executor_entities_collector.hpp | 2 +- .../executors/events_executor_notify_waitable.hpp | 10 ++++++---- rclcpp/src/rclcpp/executors/events_executor.cpp | 8 +++----- .../executors/events_executor_entities_collector.cpp | 3 ++- 5 files changed, 19 insertions(+), 11 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp index 8f4b4d27ca..e760881a0d 100644 --- a/rclcpp/include/rclcpp/executors/event_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -53,6 +53,13 @@ class EventWaitable : public rclcpp::Waitable return false; } + // Stub API: not used by EventsExecutor + std::shared_ptr + take_data() final + { + return nullptr; + } + // Stub API: not used by EventsExecutor RCLCPP_PUBLIC bool diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 6b7b8de531..e572aade98 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -75,7 +75,7 @@ class EventsExecutorEntitiesCollector final // If a entity is removed from a node, we should unset its callback RCLCPP_PUBLIC void - execute() override; + execute(std::shared_ptr & data) override; RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 1b585d5237..66629c7273 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -45,12 +45,14 @@ class EventsExecutorNotifyWaitable final : public EventWaitable // The function is a no-op, since we only care of waking up the executor RCLCPP_PUBLIC void - execute() override - {} + execute(std::shared_ptr & data) override + { + (void)data; + } RCLCPP_PUBLIC void - add_guard_condition(rcl_guard_condition_t * guard_condition) + add_guard_condition(const rcl_guard_condition_t * guard_condition) { notify_guard_conditions_.push_back(guard_condition); } @@ -76,7 +78,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable } private: - std::list notify_guard_conditions_; + std::list notify_guard_conditions_; }; } // namespace executors diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 31b99abbaa..9d8c38d720 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -33,13 +33,10 @@ EventsExecutor::EventsExecutor( entities_collector_ = std::make_shared(this); entities_collector_->init(); - // This API uses the wait_set only as a token to identify different executors. - auto context_interrupt_gc = options.context->get_interrupt_guard_condition(&wait_set_); - // Setup the executor notifier to wake up the executor when some guard conditions are tiggered. // The added guard conditions are guaranteed to not go out of scope before the executor itself. executor_notifier_ = std::make_shared(); - executor_notifier_->add_guard_condition(context_interrupt_gc); + executor_notifier_->add_guard_condition(&shutdown_guard_condition_->get_rcl_guard_condition()); executor_notifier_->add_guard_condition(&interrupt_guard_condition_); executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); entities_collector_->add_waitable(executor_notifier_); @@ -305,7 +302,8 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) auto waitable = entities_collector_->get_waitable(event.entity); if (waitable) { - waitable->execute(); + std::shared_ptr shared_ptr; + waitable->execute(shared_ptr); } break; } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 7d6ce46131..f0715de0ca 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -161,8 +161,9 @@ EventsExecutorEntitiesCollector::add_callback_group( } void -EventsExecutorEntitiesCollector::execute() +EventsExecutorEntitiesCollector::execute(std::shared_ptr & data) { + (void)data; // This function is called when the associated executor is notified that something changed. // We do not know if an entity has been added or removed so we have to rebuild everything. From baf136e1ef808d79009421d1f6fc2267bd5684e0 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 11:40:41 -0300 Subject: [PATCH 119/168] Fix IPC on EventsExecutor and StaticSingle... --- rclcpp/src/rclcpp/executors/events_executor.cpp | 4 ++-- .../src/rclcpp/executors/static_single_threaded_executor.cpp | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 9d8c38d720..97a673c5cf 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -302,8 +302,8 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) auto waitable = entities_collector_->get_waitable(event.entity); if (waitable) { - std::shared_ptr shared_ptr; - waitable->execute(shared_ptr); + auto data = waitable->take_data(); + waitable->execute(data); } break; } diff --git a/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp b/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp index da1600f04e..24d873e371 100644 --- a/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp +++ b/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp @@ -176,8 +176,9 @@ StaticSingleThreadedExecutor::execute_ready_executables() // Execute all the ready waitables for (size_t i = 0; i < entities_collector_->get_number_of_waitables(); ++i) { if (entities_collector_->get_waitable(i)->is_ready(&wait_set_)) { - std::shared_ptr shared_ptr; - entities_collector_->get_waitable(i)->execute(shared_ptr); + auto waitable = entities_collector_->get_waitable(i); + auto data = waitable->take_data(); + waitable->execute(data); } } } From a767750f0ce4aad59bb748c6262c7a6fcf9fb6da Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 11:48:23 -0300 Subject: [PATCH 120/168] Add comment --- rclcpp/include/rclcpp/executors/event_waitable.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp index e760881a0d..07b42f7df4 100644 --- a/rclcpp/include/rclcpp/executors/event_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -53,10 +53,10 @@ class EventWaitable : public rclcpp::Waitable return false; } - // Stub API: not used by EventsExecutor std::shared_ptr take_data() final { + // This waitable doesn't handle any data, return empty ptr return nullptr; } From c8f19db7cdac53356e9a90504d36b865cae9b3c5 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 12:19:23 -0300 Subject: [PATCH 121/168] Implement take_data on events_executor_notify_waitable --- rclcpp/include/rclcpp/executors/event_waitable.hpp | 7 ++----- .../rclcpp/executors/events_executor_notify_waitable.hpp | 9 +++++++++ .../rclcpp/executors/static_single_threaded_executor.cpp | 5 ++--- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp index 07b42f7df4..349396bb61 100644 --- a/rclcpp/include/rclcpp/executors/event_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -53,12 +53,9 @@ class EventWaitable : public rclcpp::Waitable return false; } + virtual std::shared_ptr - take_data() final - { - // This waitable doesn't handle any data, return empty ptr - return nullptr; - } + take_data(); // Stub API: not used by EventsExecutor RCLCPP_PUBLIC diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 66629c7273..c381bafa6c 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -77,6 +77,15 @@ class EventsExecutorNotifyWaitable final : public EventWaitable } } + + RCLCPP_PUBLIC + std::shared_ptr + take_data() + { + // This waitable doesn't handle any data, return void ptr + return nullptr; + } + private: std::list notify_guard_conditions_; }; diff --git a/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp b/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp index 24d873e371..da1600f04e 100644 --- a/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp +++ b/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp @@ -176,9 +176,8 @@ StaticSingleThreadedExecutor::execute_ready_executables() // Execute all the ready waitables for (size_t i = 0; i < entities_collector_->get_number_of_waitables(); ++i) { if (entities_collector_->get_waitable(i)->is_ready(&wait_set_)) { - auto waitable = entities_collector_->get_waitable(i); - auto data = waitable->take_data(); - waitable->execute(data); + std::shared_ptr shared_ptr; + entities_collector_->get_waitable(i)->execute(shared_ptr); } } } From 17180b1a4d9b7b6ae82acfb6dfc49f2df770ab56 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 12:26:38 -0300 Subject: [PATCH 122/168] Compiler wants take_data defined on event_waitable.hpp --- rclcpp/include/rclcpp/executors/event_waitable.hpp | 6 +++++- .../rclcpp/executors/events_executor_notify_waitable.hpp | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp index 349396bb61..57f61517e8 100644 --- a/rclcpp/include/rclcpp/executors/event_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -55,7 +55,11 @@ class EventWaitable : public rclcpp::Waitable virtual std::shared_ptr - take_data(); + take_data() + { + throw std::runtime_error("Custom EventWaitable's should define their own take_data"); + return nullptr; + } // Stub API: not used by EventsExecutor RCLCPP_PUBLIC diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index c381bafa6c..8ab233cbb4 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -80,7 +80,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable RCLCPP_PUBLIC std::shared_ptr - take_data() + take_data() override { // This waitable doesn't handle any data, return void ptr return nullptr; From d9d2885bcd05679603be8b1d3f79ad6da753bc96 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 14:35:42 -0300 Subject: [PATCH 123/168] Implement take_data only on derived classes --- rclcpp/include/rclcpp/executors/event_waitable.hpp | 8 -------- .../executors/events_executor_entities_collector.hpp | 8 ++++++++ .../rclcpp/executors/events_executor_notify_waitable.hpp | 5 ++--- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp index 57f61517e8..8f4b4d27ca 100644 --- a/rclcpp/include/rclcpp/executors/event_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -53,14 +53,6 @@ class EventWaitable : public rclcpp::Waitable return false; } - virtual - std::shared_ptr - take_data() - { - throw std::runtime_error("Custom EventWaitable's should define their own take_data"); - return nullptr; - } - // Stub API: not used by EventsExecutor RCLCPP_PUBLIC bool diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index e572aade98..5f0b549a30 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -192,6 +192,14 @@ class EventsExecutorEntitiesCollector final void add_waitable(rclcpp::Waitable::SharedPtr waitable); + RCLCPP_PUBLIC + std::shared_ptr + take_data() + { + // This waitable doesn't handle any data + return nullptr; + } + private: void set_callback_group_entities_callbacks(rclcpp::CallbackGroup::SharedPtr group); diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 8ab233cbb4..da499934dc 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -77,12 +77,11 @@ class EventsExecutorNotifyWaitable final : public EventWaitable } } - RCLCPP_PUBLIC std::shared_ptr - take_data() override + take_data() { - // This waitable doesn't handle any data, return void ptr + // This waitable doesn't handle any data return nullptr; } From dc341a3534b48e6e439ac845c8d03723cd23bc81 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 14:52:59 -0300 Subject: [PATCH 124/168] Fix tests compilation --- rclcpp/test/rclcpp/CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/rclcpp/test/rclcpp/CMakeLists.txt b/rclcpp/test/rclcpp/CMakeLists.txt index b32abf78c1..b950c9b354 100644 --- a/rclcpp/test/rclcpp/CMakeLists.txt +++ b/rclcpp/test/rclcpp/CMakeLists.txt @@ -559,7 +559,7 @@ if(TARGET test_executors) target_link_libraries(test_executors ${PROJECT_NAME}) endif() -ament_add_gtest(test_events_executor rclcpp/executors/test_events_executor.cpp +ament_add_gtest(test_events_executor executors/test_events_executor.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}") if(TARGET test_events_executor) ament_target_dependencies(test_events_executor @@ -568,7 +568,7 @@ if(TARGET test_events_executor) target_link_libraries(test_events_executor ${PROJECT_NAME} mimick) endif() -ament_add_gtest(test_events_executor_entities_collector rclcpp/executors/test_events_executor_entities_collector.cpp +ament_add_gtest(test_events_executor_entities_collector executors/test_events_executor_entities_collector.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}") if(TARGET test_events_executor_entities_collector) ament_target_dependencies(test_events_executor_entities_collector @@ -602,7 +602,7 @@ if(TARGET test_static_executor_entities_collector) target_link_libraries(test_static_executor_entities_collector ${PROJECT_NAME} mimick) endif() -ament_add_gtest(test_timers_manager rclcpp/executors/test_timers_manager.cpp +ament_add_gtest(test_timers_manager executors/test_timers_manager.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}") if(TARGET test_timers_manager) ament_target_dependencies(test_timers_manager From 5c81264ef97ddb2d0de8be1a734d3a5c0ff7571a Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 18 Jan 2021 15:03:17 -0300 Subject: [PATCH 125/168] fix linter errors --- .../include/rclcpp/executors/events_executor_notify_waitable.hpp | 1 + .../src/rclcpp/executors/events_executor_entities_collector.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index da499934dc..c3c8afd42e 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -16,6 +16,7 @@ #define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_NOTIFY_WAITABLE_HPP_ #include +#include #include "rcl/guard_condition.h" #include "rclcpp/executors/event_waitable.hpp" diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index f0715de0ca..635570ab37 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include From ccee397837203a7321de5826305f6b485938b17f Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Tue, 19 Jan 2021 11:32:09 -0300 Subject: [PATCH 126/168] Fix TimersHeap::validate_and_lock() --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index aa5988486d..36a7999403 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -198,8 +198,8 @@ class TimersManager bool any_timer_destroyed = false; auto it = weak_heap_.begin(); - auto end = weak_heap_.end(); - while (it != end) { + + while (it != weak_heap_.end()) { if (auto timer_shared_ptr = it->lock()) { // This timer is valid, add it to the vector locked_heap.push_back(std::move(timer_shared_ptr)); From ba9fee0872aae961d75b2152deb88f6d5ded5895 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Tue, 19 Jan 2021 13:26:44 -0300 Subject: [PATCH 127/168] Fix uncrustify errors --- .../rclcpp/executors/events_executor_entities_collector.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 5f0b549a30..7a07d61375 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -50,7 +50,7 @@ class EventsExecutor; * in the entities currently used by the executor. */ class EventsExecutorEntitiesCollector final -: public EventWaitable, + : public EventWaitable, public std::enable_shared_from_this { public: From f3fa8cbd4ae6c45d4a62d92f9b14df4d076b5f85 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Tue, 19 Jan 2021 14:22:10 -0300 Subject: [PATCH 128/168] Fix cppcheck error: deallocuse - Dereferencing 'node_ptr' after it is deallocated / released Seems that: NodeBaseInterface::WeakPtr node_weak_ptr(node_ptr); causes the cppcheck to fail if node_ptr is used after creating the weak pointer from it. --- .../rclcpp/executors/events_executor_entities_collector.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 635570ab37..f4f7099dba 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -142,8 +142,7 @@ EventsExecutorEntitiesCollector::add_callback_group( set_guard_condition_callback(node_ptr->get_notify_guard_condition()); // Store node's notify guard condition - rclcpp::node_interfaces::NodeBaseInterface::WeakPtr node_weak_ptr(node_ptr); - weak_nodes_to_guard_conditions_[node_weak_ptr] = node_ptr->get_notify_guard_condition(); + weak_nodes_to_guard_conditions_[node_ptr] = node_ptr->get_notify_guard_condition(); } // Add callback group to weak_groups_to_node From 0efab07d1884a454d842cb88133e053b3728da3d Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 21 Jan 2021 15:54:29 -0300 Subject: [PATCH 129/168] Reorder APIs arguments --- .../rclcpp/executors/events_executor_notify_waitable.hpp | 4 ++-- rclcpp/src/rclcpp/client.cpp | 6 +++--- .../rclcpp/executors/events_executor_entities_collector.cpp | 6 +++--- rclcpp/src/rclcpp/qos_event.cpp | 4 ++-- rclcpp/src/rclcpp/service.cpp | 6 +++--- rclcpp/src/rclcpp/subscription_base.cpp | 6 +++--- rclcpp/src/rclcpp/subscription_intra_process_base.cpp | 4 ++-- rclcpp/test/rclcpp/executors/test_executors.cpp | 4 ++-- 8 files changed, 20 insertions(+), 20 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index c3c8afd42e..ee7ed4a6dc 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -66,10 +66,10 @@ class EventsExecutorNotifyWaitable final : public EventWaitable { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( - executor, + gc, executor_callback, + executor, this, - gc, false); if (RCL_RET_OK != ret) { diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 785057c369..14f8e02b42 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -205,10 +205,10 @@ ClientBase::set_events_executor_callback( rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_client_set_listener_callback( - executor, + client_handle_.get(), executor_callback, - this, - client_handle_.get()); + executor, + this); if (RCL_RET_OK != ret) { throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index f4f7099dba..c2b08a9de6 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -478,10 +478,10 @@ EventsExecutorEntitiesCollector::set_guard_condition_callback( const rcl_guard_condition_t * guard_condition) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( - associated_executor_, + guard_condition, &EventsExecutor::push_event, + associated_executor_, this, - guard_condition, false /* Discard previous events */); if (ret != RCL_RET_OK) { @@ -494,10 +494,10 @@ EventsExecutorEntitiesCollector::unset_guard_condition_callback( const rcl_guard_condition_t * guard_condition) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( + guard_condition, nullptr, nullptr, nullptr, - guard_condition, false /* Discard previous events */); if (ret != RCL_RET_OK) { diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 5ae3dd1a93..1fbfa6559c 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -74,10 +74,10 @@ QOSEventHandlerBase::set_events_executor_callback( rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_event_set_listener_callback( - executor, + &event_handle_, executor_callback, + executor, this, - &event_handle_, false /* Discard previous events */); if (RCL_RET_OK != ret) { diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 0396e728c5..506370325a 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -91,10 +91,10 @@ ServiceBase::set_events_executor_callback( rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_service_set_listener_callback( - executor, + service_handle_.get(), executor_callback, - this, - service_handle_.get()); + executor, + this); if (RCL_RET_OK != ret) { throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index f272a976a9..e266a3d612 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -295,10 +295,10 @@ SubscriptionBase::set_events_executor_callback( rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_subscription_set_listener_callback( - executor, + subscription_handle_.get(), executor_callback, - this, - subscription_handle_.get()); + executor, + this); if (RCL_RET_OK != ret) { throw std::runtime_error("Couldn't set the EventsExecutor's callback to subscription"); diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 797ddb2453..3aa9dc22b3 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -43,10 +43,10 @@ SubscriptionIntraProcessBase::set_events_executor_callback( rmw_listener_cb_t executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( - executor, + &gc_, executor_callback, + executor, this, - &gc_, true /*Use previous events*/); if (RCL_RET_OK != ret) { diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index e18ea0703c..a8393974d7 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -473,10 +473,10 @@ class TestWaitable : public rclcpp::Waitable rmw_listener_cb_t executor_callback) const override { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( - executor, + &gc_, executor_callback, + executor, this, - &gc_, true /*Use previous events*/); if (RCL_RET_OK != ret) { From aab59a037f2fac6e31098c210168680d5b378dfa Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Wed, 27 Jan 2021 16:52:40 -0300 Subject: [PATCH 130/168] Implement no-op on IPC subscription --- .../experimental/buffers/ring_buffer_implementation.hpp | 7 ++++++- .../rclcpp/experimental/subscription_intra_process.hpp | 9 ++++++++- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp index ad01946193..ece0e379dc 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp @@ -91,7 +91,12 @@ class RingBufferImplementation : public BufferImplementationBase if (!has_data_()) { RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Calling dequeue on empty intra-process buffer"); - throw std::runtime_error("Calling dequeue on empty intra-process buffer"); + // This situation can happen on the EventsExecutor if we have more events in the queue + // than messages in the history cache (set by the qos_policies.depth of the subscription) + // For example if we set depth=1 and we get 2 messages really fast (so no time for processing), + // we could end up with 2 events in the queue but only 1 msg is actually stored on the buffer. + // In this case we return an empty buffer. + return BufferT(); } auto request = std::move(ring_buffer_[read_index_]); diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp index 46e019daee..dfc70eeeed 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp @@ -129,8 +129,14 @@ class SubscriptionIntraProcess : public SubscriptionIntraProcessBase if (any_callback_.use_take_shared_method()) { shared_msg = buffer_->consume_shared(); + if(!shared_msg) { + return nullptr; + } } else { unique_msg = buffer_->consume_unique(); + if(!unique_msg) { + return nullptr; + } } return std::static_pointer_cast( std::make_shared>( @@ -185,7 +191,8 @@ class SubscriptionIntraProcess : public SubscriptionIntraProcessBase execute_impl(std::shared_ptr & data) { if (!data) { - throw std::runtime_error("'data' is empty"); + RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Subscription intra-process: 'data' is empty"); + return; } rmw_message_info_t msg_info; From 8ec85910b61636d10cabe014272221f03362ea19 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 1 Feb 2021 10:13:26 -0300 Subject: [PATCH 131/168] Check if valid msg before dispatch --- rclcpp/include/rclcpp/any_subscription_callback.hpp | 12 ++++++++++++ .../experimental/subscription_intra_process.hpp | 6 ------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/rclcpp/include/rclcpp/any_subscription_callback.hpp b/rclcpp/include/rclcpp/any_subscription_callback.hpp index 640077a1d9..9b6cb8691f 100644 --- a/rclcpp/include/rclcpp/any_subscription_callback.hpp +++ b/rclcpp/include/rclcpp/any_subscription_callback.hpp @@ -185,6 +185,12 @@ class AnySubscriptionCallback ConstMessageSharedPtr message, const rclcpp::MessageInfo & message_info) { TRACEPOINT(callback_start, static_cast(this), true); + + // If the message is not valid, return. + if(!message) { + return; + } + if (const_shared_ptr_callback_) { const_shared_ptr_callback_(message); } else if (const_shared_ptr_with_info_callback_) { @@ -208,6 +214,12 @@ class AnySubscriptionCallback MessageUniquePtr message, const rclcpp::MessageInfo & message_info) { TRACEPOINT(callback_start, static_cast(this), true); + + // If the message is not valid, return. + if(!message) { + return; + } + if (shared_ptr_callback_) { typename std::shared_ptr shared_message = std::move(message); shared_ptr_callback_(shared_message); diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp index dfc70eeeed..3982ff4e48 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp @@ -129,14 +129,8 @@ class SubscriptionIntraProcess : public SubscriptionIntraProcessBase if (any_callback_.use_take_shared_method()) { shared_msg = buffer_->consume_shared(); - if(!shared_msg) { - return nullptr; - } } else { unique_msg = buffer_->consume_unique(); - if(!unique_msg) { - return nullptr; - } } return std::static_pointer_cast( std::make_shared>( From b41655b69a926a297fc662f9a72ea02eb2cf62a5 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 1 Feb 2021 10:22:11 -0300 Subject: [PATCH 132/168] Add log when ring buffer is full --- .../rclcpp/experimental/buffers/ring_buffer_implementation.hpp | 1 + 1 file changed, 1 insertion(+) diff --git a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp index ece0e379dc..8ae466d309 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp @@ -73,6 +73,7 @@ class RingBufferImplementation : public BufferImplementationBase ring_buffer_[write_index_] = std::move(request); if (is_full_()) { + RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Ring buffer is full! Buffer capacity: %d", capacity_); read_index_ = next_(read_index_); } else { size_++; From 02ce34960da6aeeb74d5c4c6a3fbdcf0e5f340cb Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 1 Feb 2021 10:23:42 -0300 Subject: [PATCH 133/168] restore line. This shouldn't happen any more --- .../include/rclcpp/experimental/subscription_intra_process.hpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp index 3982ff4e48..46e019daee 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp @@ -185,8 +185,7 @@ class SubscriptionIntraProcess : public SubscriptionIntraProcessBase execute_impl(std::shared_ptr & data) { if (!data) { - RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Subscription intra-process: 'data' is empty"); - return; + throw std::runtime_error("'data' is empty"); } rmw_message_info_t msg_info; From 08fd037b43e266738a7a5d64dcddbb7f255778cf Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 1 Feb 2021 10:26:05 -0300 Subject: [PATCH 134/168] Fix size_t log type --- .../rclcpp/experimental/buffers/ring_buffer_implementation.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp index 8ae466d309..5f35d24a4d 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp @@ -73,7 +73,7 @@ class RingBufferImplementation : public BufferImplementationBase ring_buffer_[write_index_] = std::move(request); if (is_full_()) { - RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Ring buffer is full! Buffer capacity: %d", capacity_); + RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Ring buffer is full! Buffer capacity: %lu", capacity_); read_index_ = next_(read_index_); } else { size_++; From 51bd25cc0b3484193b72446fae78cc1e32ca5eb6 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 1 Feb 2021 11:34:42 -0300 Subject: [PATCH 135/168] remove log about full ring buffer --- .../rclcpp/experimental/buffers/ring_buffer_implementation.hpp | 1 - 1 file changed, 1 deletion(-) diff --git a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp index 5f35d24a4d..ece0e379dc 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp @@ -73,7 +73,6 @@ class RingBufferImplementation : public BufferImplementationBase ring_buffer_[write_index_] = std::move(request); if (is_full_()) { - RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Ring buffer is full! Buffer capacity: %lu", capacity_); read_index_ = next_(read_index_); } else { size_++; From a9bfc77973abdc169c90a0782cc13fb8cbc9ddab Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 4 Feb 2021 17:51:59 -0300 Subject: [PATCH 136/168] Add events queue abstract class and implem --- .../rclcpp/executors/events_executor.hpp | 10 +- .../experimental/buffers/events_queue.hpp | 98 +++++++++++++++ .../buffers/performance_events_queue.hpp | 116 ++++++++++++++++++ .../src/rclcpp/executors/events_executor.cpp | 42 ++++--- 4 files changed, 242 insertions(+), 24 deletions(-) create mode 100644 rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp create mode 100644 rclcpp/include/rclcpp/experimental/buffers/performance_events_queue.hpp diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index df63a96535..32317897b1 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -24,6 +24,7 @@ #include "rclcpp/executors/events_executor_entities_collector.hpp" #include "rclcpp/executors/events_executor_notify_waitable.hpp" #include "rclcpp/executors/timers_manager.hpp" +#include "rclcpp/experimental/buffers/events_queue.hpp" #include "rclcpp/node.hpp" #include "rmw/listener_event_types.h" @@ -59,6 +60,7 @@ class EventsExecutor : public rclcpp::Executor */ RCLCPP_PUBLIC explicit EventsExecutor( + rclcpp::experimental::buffers::EventsQueue::UniquePtr events_queue, const rclcpp::ExecutorOptions & options = rclcpp::ExecutorOptions()); /// Default destrcutor. @@ -190,10 +192,10 @@ class EventsExecutor : public rclcpp::Executor { std::unique_lock lock(this_executor->push_mutex_); - this_executor->event_queue_.push(event); + this_executor->events_queue_->push(event); } // Notify that the event queue has some events in it. - this_executor->event_queue_cv_.notify_one(); + this_executor->events_queue_cv_.notify_one(); } /// Extract and execute events from the queue until it is empty @@ -207,7 +209,7 @@ class EventsExecutor : public rclcpp::Executor execute_event(const rmw_listener_event_t & event); // Queue where entities can push events - EventQueue event_queue_; + rclcpp::experimental::buffers::EventsQueue::SharedPtr events_queue_; EventsExecutorEntitiesCollector::SharedPtr entities_collector_; EventsExecutorNotifyWaitable::SharedPtr executor_notifier_; @@ -215,7 +217,7 @@ class EventsExecutor : public rclcpp::Executor // Mutex to protect the insertion of events in the queue std::mutex push_mutex_; // Variable used to notify when an event is added to the queue - std::condition_variable event_queue_cv_; + std::condition_variable events_queue_cv_; // Timers manager std::shared_ptr timers_manager_; }; diff --git a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp new file mode 100644 index 0000000000..ef9b61a5f2 --- /dev/null +++ b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp @@ -0,0 +1,98 @@ +// Copyright 2021 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXPERIMENTAL__BUFFERS__EVENTS_QUEUE_HPP_ +#define RCLCPP__EXPERIMENTAL__BUFFERS__EVENTS_QUEUE_HPP_ + +#include + +#include "rclcpp/macros.hpp" + +#include "rmw/listener_event_types.h" + +namespace rclcpp +{ +namespace experimental +{ +namespace buffers +{ + +/** + * @brief This abstract class is intended to be used as + * a wrapper around a queue. The derived classes should chose + * which container to use and the strategies for push and prune + * events from the queue. + */ +class EventsQueue +{ +public: + RCLCPP_SMART_PTR_DEFINITIONS(EventsQueue) + + /** + * @brief Destruct the object. + */ + RCLCPP_PUBLIC + virtual ~EventsQueue() = default; + + /** + * @brief push event into the queue + * @param event The event to push into the queue + */ + RCLCPP_PUBLIC + virtual + void + push(rmw_listener_event_t event) = 0; + + /** + * @brief removes front element from the queue + * @return iterator + */ + RCLCPP_PUBLIC + virtual + void + pop() = 0; + + /** + * @brief gets the front event from the queue + * @return the front event + */ + RCLCPP_PUBLIC + virtual + rmw_listener_event_t + front() = 0; + + /** + * @brief Test whether queue is empty + * @return true if the queue's size is 0, false otherwise. + */ + RCLCPP_PUBLIC + virtual + bool + empty() = 0; + + /** + * @brief gets a queue with all events accumulated on it + * @return queue with events + */ + RCLCPP_PUBLIC + virtual + std::queue + get_all_events() = 0; +}; + +} // namespace buffers +} // namespace experimental +} // namespace rclcpp + +#endif // RCLCPP__EXPERIMENTAL__BUFFERS__EVENTS_QUEUE_HPP_ diff --git a/rclcpp/include/rclcpp/experimental/buffers/performance_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/performance_events_queue.hpp new file mode 100644 index 0000000000..feeaff059d --- /dev/null +++ b/rclcpp/include/rclcpp/experimental/buffers/performance_events_queue.hpp @@ -0,0 +1,116 @@ +// Copyright 2021 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXPERIMENTAL__BUFFERS__PERFORMANCE_EVENTS_QUEUE_HPP_ +#define RCLCPP__EXPERIMENTAL__BUFFERS__PERFORMANCE_EVENTS_QUEUE_HPP_ + +#include + +#include "rclcpp/executors/events_queue.hpp" + +namespace rclcpp +{ +namespace experimental +{ +namespace buffers +{ + +/** + * @brief This class provides a simple queue implementation + * based on a std::queue. As the objective is having a CPU peformant + * queue, it does not performs any checks about the size of + * the queue, so the queue size could grow unbounded. + * It does not implement any pruning mechanisms. + */ +class PerformanceEventsQueue : public EventsQueue +{ +public: + + using EventQueue = std::queue; + + RCLCPP_PUBLIC + ~PerformanceEventsQueue() = default; + + /** + * @brief push event into the queue + * @param event The event to push into the queue + */ + RCLCPP_PUBLIC + virtual + void + push(rmw_listener_event_t event) + { + event_queue_.push(event); + } + + /** + * @brief removes front element from the queue + * @return iterator + */ + RCLCPP_PUBLIC + virtual + void + pop() + { + event_queue_.pop(); + } + + /** + * @brief gets the front event from the queue + * @return the front event + */ + RCLCPP_PUBLIC + virtual + rmw_listener_event_t + front() + { + return event_queue_.front(); + } + + /** + * @brief Test whether queue is empty + * @return true if the queue's size is 0, false otherwise. + */ + RCLCPP_PUBLIC + virtual + bool + empty() + { + return event_queue_.empty(); + } + + /** + * @brief gets a queue with all events on it + * @return queue with events + */ + RCLCPP_PUBLIC + virtual + EventQueue + get_all_events() + { + EventQueue local_queue; + std::swap(event_queue_, local_queue); + return local_queue; + } + +private: + EventQueue event_queue_; +}; + +} // namespace buffers +} // namespace experimental +} // namespace rclcpp + + +#endif // RCLCPP__EXPERIMENTAL__BUFFERS__PERFORMANCE_EVENTS_QUEUE_HPP_ diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 97a673c5cf..be688134bf 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -24,8 +24,10 @@ using namespace std::chrono_literals; using rclcpp::executors::EventsExecutor; +using rclcpp::experimental::buffers::EventsQueue; EventsExecutor::EventsExecutor( + EventsQueue::UniquePtr events_queue, const rclcpp::ExecutorOptions & options) : rclcpp::Executor(options) { @@ -33,6 +35,8 @@ EventsExecutor::EventsExecutor( entities_collector_ = std::make_shared(this); entities_collector_->init(); + events_queue_ = std::move(events_queue); + // Setup the executor notifier to wake up the executor when some guard conditions are tiggered. // The added guard conditions are guaranteed to not go out of scope before the executor itself. executor_notifier_ = std::make_shared(); @@ -51,7 +55,7 @@ EventsExecutor::spin() RCLCPP_SCOPE_EXIT(this->spinning.store(false);); // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() {return !event_queue_.empty();}; + auto has_event_predicate = [this]() {return !events_queue_->empty();}; // Local event queue to allow entities to push events while we execute them EventQueue execution_event_queue; @@ -61,10 +65,10 @@ EventsExecutor::spin() while (rclcpp::ok(context_) && spinning.load()) { std::unique_lock push_lock(push_mutex_); // We wait here until something has been pushed to the event queue - event_queue_cv_.wait(push_lock, has_event_predicate); - // We got an event! Swap queues - std::swap(execution_event_queue, event_queue_); - // After swapping the queues, we don't need the lock anymore + events_queue_cv_.wait(push_lock, has_event_predicate); + // Local event queue to allow entities to push events while we execute them + execution_event_queue = events_queue_->get_all_events(); + // Unlock the mutex push_lock.unlock(); // Consume all available events, this queue will be empty at the end of the function this->consume_all_events(execution_event_queue); @@ -91,10 +95,8 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) // - An executor event is received and processed // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() {return !event_queue_.empty();}; + auto has_event_predicate = [this]() {return !events_queue_->empty();}; - // Local event queue to allow entities to push events while we execute them - EventQueue execution_event_queue; // Select the smallest between input max_duration and timer timeout auto next_timer_timeout = timers_manager_->get_head_timeout(); @@ -104,10 +106,10 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) std::unique_lock push_lock(push_mutex_); // Wait until timeout or event - event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); - // Time to swap queues as the wait is over - std::swap(execution_event_queue, event_queue_); - // After swapping the queues, we don't need the lock anymore + events_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); + // Local event queue to allow entities to push events while we execute them + EventQueue execution_event_queue = events_queue_->get_all_events(); + // We don't need the lock anymore push_lock.unlock(); // Execute all ready timers @@ -129,7 +131,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) RCLCPP_SCOPE_EXIT(this->spinning.store(false);); // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() {return !event_queue_.empty();}; + auto has_event_predicate = [this]() {return !events_queue_->empty();}; // Local event queue to allow entities to push events while we execute them EventQueue execution_event_queue; @@ -149,7 +151,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) { // Wait once until timeout or event std::unique_lock push_lock(push_mutex_); - event_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); + events_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); } auto timeout = timers_manager_->get_head_timeout(); @@ -157,7 +159,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Keep executing until no more work to do or timeout expired while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { std::unique_lock push_lock(push_mutex_); - std::swap(execution_event_queue, event_queue_); + execution_event_queue = events_queue_->get_all_events(); push_lock.unlock(); // Exit if there is no more work to do @@ -188,7 +190,7 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) } // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() {return !event_queue_.empty();}; + auto has_event_predicate = [this]() {return !events_queue_->empty();}; rmw_listener_event_t event; bool has_event = false; @@ -196,13 +198,13 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) { // Wait until timeout or event arrives std::unique_lock lock(push_mutex_); - event_queue_cv_.wait_for(lock, timeout, has_event_predicate); + events_queue_cv_.wait_for(lock, timeout, has_event_predicate); // Grab first event from queue if it exists - has_event = !event_queue_.empty(); + has_event = !events_queue_->empty(); if (has_event) { - event = event_queue_.front(); - event_queue_.pop(); + event = events_queue_->front(); + events_queue_->pop(); } } From 5b568fb8124f86b939790bc16934ffd3e43ca44e Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 5 Feb 2021 12:47:22 -0300 Subject: [PATCH 137/168] Apply PR suggestions --- .../rclcpp/executors/events_executor.hpp | 5 +- .../experimental/buffers/events_queue.hpp | 22 +++++++-- ...ents_queue.hpp => simple_events_queue.hpp} | 49 +++++++++++++------ .../src/rclcpp/executors/events_executor.cpp | 9 ++-- 4 files changed, 60 insertions(+), 25 deletions(-) rename rclcpp/include/rclcpp/experimental/buffers/{performance_events_queue.hpp => simple_events_queue.hpp} (60%) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 32317897b1..a584908b88 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -25,6 +25,7 @@ #include "rclcpp/executors/events_executor_notify_waitable.hpp" #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/experimental/buffers/events_queue.hpp" +#include "rclcpp/experimental/buffers/simple_events_queue.hpp" #include "rclcpp/node.hpp" #include "rmw/listener_event_types.h" @@ -56,11 +57,13 @@ class EventsExecutor : public rclcpp::Executor /// Default constructor. See the default constructor for Executor. /** + * \param[in] events_queue The queue used to store events. * \param[in] options Options used to configure the executor. */ RCLCPP_PUBLIC explicit EventsExecutor( - rclcpp::experimental::buffers::EventsQueue::UniquePtr events_queue, + rclcpp::experimental::buffers::EventsQueue::UniquePtr events_queue = + std::make_unique(), const rclcpp::ExecutorOptions & options = rclcpp::ExecutorOptions()); /// Default destrcutor. diff --git a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp index ef9b61a5f2..492183ccff 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp @@ -17,6 +17,7 @@ #include +#include "rclcpp/executors/events_executor_entities_collector.hpp" #include "rclcpp/macros.hpp" #include "rmw/listener_event_types.h" @@ -52,11 +53,12 @@ class EventsQueue RCLCPP_PUBLIC virtual void - push(rmw_listener_event_t event) = 0; + push(const rmw_listener_event_t & event) = 0; /** * @brief removes front element from the queue - * @return iterator + * The element removed is the "oldest" element in the queue whose + * value can be retrieved by calling member front(). */ RCLCPP_PUBLIC virtual @@ -70,7 +72,7 @@ class EventsQueue RCLCPP_PUBLIC virtual rmw_listener_event_t - front() = 0; + front() const = 0; /** * @brief Test whether queue is empty @@ -79,10 +81,20 @@ class EventsQueue RCLCPP_PUBLIC virtual bool - empty() = 0; + empty() const = 0; /** - * @brief gets a queue with all events accumulated on it + * @brief Initializes the queue + * @param entities_collector The entities collector associated with the executor + */ + RCLCPP_PUBLIC + virtual + void + init(rclcpp::executors::EventsExecutorEntitiesCollector::SharedPtr entities_collector) = 0; + + /** + * @brief gets a queue with all events accumulated on it since + * the last call. The member queue is empty when the call returns. * @return queue with events */ RCLCPP_PUBLIC diff --git a/rclcpp/include/rclcpp/experimental/buffers/performance_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp similarity index 60% rename from rclcpp/include/rclcpp/experimental/buffers/performance_events_queue.hpp rename to rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp index feeaff059d..1854ab0810 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/performance_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#ifndef RCLCPP__EXPERIMENTAL__BUFFERS__PERFORMANCE_EVENTS_QUEUE_HPP_ -#define RCLCPP__EXPERIMENTAL__BUFFERS__PERFORMANCE_EVENTS_QUEUE_HPP_ +#ifndef RCLCPP__EXPERIMENTAL__BUFFERS__SIMPLE_EVENTS_QUEUE_HPP_ +#define RCLCPP__EXPERIMENTAL__BUFFERS__SIMPLE_EVENTS_QUEUE_HPP_ #include @@ -33,14 +33,12 @@ namespace buffers * the queue, so the queue size could grow unbounded. * It does not implement any pruning mechanisms. */ -class PerformanceEventsQueue : public EventsQueue +class SimpleEventsQueue : public EventsQueue { public: - using EventQueue = std::queue; - RCLCPP_PUBLIC - ~PerformanceEventsQueue() = default; + ~SimpleEventsQueue() = default; /** * @brief push event into the queue @@ -49,14 +47,15 @@ class PerformanceEventsQueue : public EventsQueue RCLCPP_PUBLIC virtual void - push(rmw_listener_event_t event) + push(const rmw_listener_event_t & event) { event_queue_.push(event); } /** * @brief removes front element from the queue - * @return iterator + * The element removed is the "oldest" element in the queue whose + * value can be retrieved by calling member front(). */ RCLCPP_PUBLIC virtual @@ -73,7 +72,7 @@ class PerformanceEventsQueue : public EventsQueue RCLCPP_PUBLIC virtual rmw_listener_event_t - front() + front() const { return event_queue_.front(); } @@ -85,27 +84,45 @@ class PerformanceEventsQueue : public EventsQueue RCLCPP_PUBLIC virtual bool - empty() + empty() const { return event_queue_.empty(); } /** - * @brief gets a queue with all events on it - * @return queue with events + * @brief Initializes the queue + * @param entities_collector The entities collector associated with the executor + */ + RCLCPP_PUBLIC + virtual + void + init(rclcpp::executors::EventsExecutorEntitiesCollector::SharedPtr entities_collector) + { + // Entities collector not used in this queue implementation + (void)entities_collector; + // Make sure the queue is empty when we start + std::queue local_queue; + std::swap(event_queue_, local_queue); + } + + + /** + * @brief gets a queue with all events accumulated on it since + * the last call. The member queue is empty when the call returns. + * @return std::queue with events */ RCLCPP_PUBLIC virtual - EventQueue + std::queue get_all_events() { - EventQueue local_queue; + std::queue local_queue; std::swap(event_queue_, local_queue); return local_queue; } private: - EventQueue event_queue_; + std::queue event_queue_; }; } // namespace buffers @@ -113,4 +130,4 @@ class PerformanceEventsQueue : public EventsQueue } // namespace rclcpp -#endif // RCLCPP__EXPERIMENTAL__BUFFERS__PERFORMANCE_EVENTS_QUEUE_HPP_ +#endif // RCLCPP__EXPERIMENTAL__BUFFERS__SIMPLE_EVENTS_QUEUE_HPP_ diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index be688134bf..d0857d949e 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -24,10 +24,9 @@ using namespace std::chrono_literals; using rclcpp::executors::EventsExecutor; -using rclcpp::experimental::buffers::EventsQueue; EventsExecutor::EventsExecutor( - EventsQueue::UniquePtr events_queue, + rclcpp::experimental::buffers::EventsQueue::UniquePtr events_queue, const rclcpp::ExecutorOptions & options) : rclcpp::Executor(options) { @@ -35,7 +34,6 @@ EventsExecutor::EventsExecutor( entities_collector_ = std::make_shared(this); entities_collector_->init(); - events_queue_ = std::move(events_queue); // Setup the executor notifier to wake up the executor when some guard conditions are tiggered. // The added guard conditions are guaranteed to not go out of scope before the executor itself. @@ -44,6 +42,11 @@ EventsExecutor::EventsExecutor( executor_notifier_->add_guard_condition(&interrupt_guard_condition_); executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); entities_collector_->add_waitable(executor_notifier_); + + // Get ownership of the queue used to store events. + events_queue_ = std::move(events_queue); + // Init the events queue + events_queue_->init(entities_collector_); } void From f73cbddd3730bce88c58a536cfe94fa5a053c02b Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 5 Feb 2021 14:13:30 -0300 Subject: [PATCH 138/168] Fix ctests --- rclcpp/include/rclcpp/executors/events_executor.hpp | 4 ++-- .../rclcpp/experimental/buffers/simple_events_queue.hpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index a584908b88..6b5ef5ec11 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -62,8 +62,8 @@ class EventsExecutor : public rclcpp::Executor */ RCLCPP_PUBLIC explicit EventsExecutor( - rclcpp::experimental::buffers::EventsQueue::UniquePtr events_queue = - std::make_unique(), + rclcpp::experimental::buffers::EventsQueue::UniquePtr events_queue = std::make_unique< + rclcpp::experimental::buffers::SimpleEventsQueue>(), const rclcpp::ExecutorOptions & options = rclcpp::ExecutorOptions()); /// Default destrcutor. diff --git a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp index 1854ab0810..cb656a67e9 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp @@ -16,6 +16,7 @@ #define RCLCPP__EXPERIMENTAL__BUFFERS__SIMPLE_EVENTS_QUEUE_HPP_ #include +#include #include "rclcpp/executors/events_queue.hpp" @@ -36,7 +37,6 @@ namespace buffers class SimpleEventsQueue : public EventsQueue { public: - RCLCPP_PUBLIC ~SimpleEventsQueue() = default; @@ -74,7 +74,7 @@ class SimpleEventsQueue : public EventsQueue rmw_listener_event_t front() const { - return event_queue_.front(); + return event_queue_.front(); } /** From 9c5f037ad98a78613c278ba11a9b414ae84ded9b Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 5 Feb 2021 16:04:42 -0300 Subject: [PATCH 139/168] EntitiesCollector not needed on events_queue --- rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp | 3 +-- .../rclcpp/experimental/buffers/simple_events_queue.hpp | 5 +---- rclcpp/src/rclcpp/executors/events_executor.cpp | 2 +- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp index 492183ccff..d00a00705d 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp @@ -85,12 +85,11 @@ class EventsQueue /** * @brief Initializes the queue - * @param entities_collector The entities collector associated with the executor */ RCLCPP_PUBLIC virtual void - init(rclcpp::executors::EventsExecutorEntitiesCollector::SharedPtr entities_collector) = 0; + init() = 0; /** * @brief gets a queue with all events accumulated on it since diff --git a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp index cb656a67e9..b0685e7ad1 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp @@ -91,15 +91,12 @@ class SimpleEventsQueue : public EventsQueue /** * @brief Initializes the queue - * @param entities_collector The entities collector associated with the executor */ RCLCPP_PUBLIC virtual void - init(rclcpp::executors::EventsExecutorEntitiesCollector::SharedPtr entities_collector) + init() { - // Entities collector not used in this queue implementation - (void)entities_collector; // Make sure the queue is empty when we start std::queue local_queue; std::swap(event_queue_, local_queue); diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index d0857d949e..f70cd2772f 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -46,7 +46,7 @@ EventsExecutor::EventsExecutor( // Get ownership of the queue used to store events. events_queue_ = std::move(events_queue); // Init the events queue - events_queue_->init(entities_collector_); + events_queue_->init(); } void From 780af848f1aca6080a4c0333b42c5c665d998976 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 5 Feb 2021 16:36:13 -0300 Subject: [PATCH 140/168] Add BoundedEventsQueue --- .../buffers/bounded_events_queue.hpp | 146 ++++++++++++++++++ 1 file changed, 146 insertions(+) create mode 100644 rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp diff --git a/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp new file mode 100644 index 0000000000..2fb2970ba5 --- /dev/null +++ b/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp @@ -0,0 +1,146 @@ +// Copyright 2021 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXPERIMENTAL__BUFFERS__BOUNDED_EVENTS_QUEUE_HPP_ +#define RCLCPP__EXPERIMENTAL__BUFFERS__BOUNDED_EVENTS_QUEUE_HPP_ + +#include +#include + +#include "rclcpp/executors/events_queue.hpp" + +namespace rclcpp +{ +namespace experimental +{ +namespace buffers +{ + +/** + * @brief This class provides a bounded queue implementation + * based on a std::queue. Before pushing events into the queue + * checks the queue size. In case of exceeding the size it performs + * a prune of the queue. + */ +class BoundedEventsQueue : public EventsQueue +{ +public: + RCLCPP_PUBLIC + ~BoundedEventsQueue() = default; + + /** + * @brief push event into the queue + * @param event The event to push into the queue + */ + RCLCPP_PUBLIC + virtual + void + push(const rmw_listener_event_t & event) + { + if (event_queue_.size() >= queue_size_limit_) { + // Simple prune strategy: Remove all elements in the queue. + this->init(); + } + event_queue_.push(event); + } + + /** + * @brief removes front element from the queue + * The element removed is the "oldest" element in the queue whose + * value can be retrieved by calling member front(). + */ + RCLCPP_PUBLIC + virtual + void + pop() + { + event_queue_.pop(); + } + + /** + * @brief gets the front event from the queue + * @return the front event + */ + RCLCPP_PUBLIC + virtual + rmw_listener_event_t + front() const + { + return event_queue_.front(); + } + + /** + * @brief Test whether queue is empty + * @return true if the queue's size is 0, false otherwise. + */ + RCLCPP_PUBLIC + virtual + bool + empty() const + { + return event_queue_.empty(); + } + + /** + * @brief Initializes the queue + */ + RCLCPP_PUBLIC + virtual + void + init() + { + // Make sure the queue is empty when we start + std::queue local_queue; + std::swap(event_queue_, local_queue); + } + + + /** + * @brief gets a queue with all events accumulated on it since + * the last call. The member queue is empty when the call returns. + * @return std::queue with events + */ + RCLCPP_PUBLIC + virtual + std::queue + get_all_events() + { + std::queue local_queue; + std::swap(event_queue_, local_queue); + return local_queue; + } + + /** + * @brief sets the queue size limit + * @param limit The queue size limit + */ + RCLCPP_PUBLIC + void + set_queue_size_limit(size_t queue_size_limit) + { + queue_size_limit_ = queue_size_limit; + } + +private: + std::queue event_queue_; + + size_t queue_size_limit_ = 1000; +}; + +} // namespace buffers +} // namespace experimental +} // namespace rclcpp + + +#endif // RCLCPP__EXPERIMENTAL__BUFFERS__BOUNDED_EVENTS_QUEUE_HPP_ From 0317e7774a8f0ea40f074936c9bfc6801194eb46 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 5 Feb 2021 18:31:58 -0300 Subject: [PATCH 141/168] Add unit tests for Simple and Bounded queues --- rclcpp/test/rclcpp/CMakeLists.txt | 9 ++ .../rclcpp/executors/test_events_queue.cpp | 147 ++++++++++++++++++ 2 files changed, 156 insertions(+) create mode 100644 rclcpp/test/rclcpp/executors/test_events_queue.cpp diff --git a/rclcpp/test/rclcpp/CMakeLists.txt b/rclcpp/test/rclcpp/CMakeLists.txt index b950c9b354..13d41a7fca 100644 --- a/rclcpp/test/rclcpp/CMakeLists.txt +++ b/rclcpp/test/rclcpp/CMakeLists.txt @@ -568,6 +568,15 @@ if(TARGET test_events_executor) target_link_libraries(test_events_executor ${PROJECT_NAME} mimick) endif() +ament_add_gtest(test_events_queue executors/test_events_queue.cpp + APPEND_LIBRARY_DIRS "${append_library_dirs}") +if(TARGET test_events_queue) + ament_target_dependencies(test_events_queue + "rcl" + "test_msgs") + target_link_libraries(test_events_queue ${PROJECT_NAME} mimick) +endif() + ament_add_gtest(test_events_executor_entities_collector executors/test_events_executor_entities_collector.cpp APPEND_LIBRARY_DIRS "${append_library_dirs}") if(TARGET test_events_executor_entities_collector) diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp new file mode 100644 index 0000000000..5cf5db7b8e --- /dev/null +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -0,0 +1,147 @@ +// Copyright 2021 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +#include +#include +#include + +#include "rclcpp/executors/events_executor.hpp" +#include "rclcpp/experimental/buffers/bounded_events_queue.hpp" + +#include "test_msgs/msg/empty.hpp" + +using namespace std::chrono_literals; + +using rclcpp::executors::EventsExecutor; + +class TestEventsQueue : public ::testing::Test +{ +public: + void SetUp() + { + rclcpp::init(0, nullptr); + } + + void TearDown() + { + rclcpp::shutdown(); + } +}; + +TEST_F(TestEventsQueue, BoundedQueue) +{ + // Create BoundedEventsQueue and set limit to 10 events. + auto bounded_queue = std::make_unique(); + bounded_queue->set_queue_size_limit(10); + + // Create an events executor using the bounded queue + EventsExecutor executor_sub(std::move(bounded_queue)); + + // Create a subscription node + auto node_sub = std::make_shared("node_sub"); + + size_t callback_count = 0; + + auto subscription = + node_sub->create_subscription( + "topic", + rclcpp::QoS(10), + [&](test_msgs::msg::Empty::SharedPtr) { + callback_count++; + }); + + // Add susbscription node to the executor, so the queue can start getting events + executor_sub.add_node(node_sub); + + // Create a publisher node + auto node_pub = std::make_shared("node_pub"); + auto publisher = node_pub->create_publisher("topic", rclcpp::QoS(10)); + + // Let subscriptions executor spin to execute any previous events + // not related to the subscription, so we start with an empty queue + executor_sub.spin_some(10ms); + + // Publish 11 messages, the eleventh msg should prune the queue + // and we should end up with only one event on it + for (int i = 0; i < 11; i++) { + publisher->publish(std::make_unique()); + std::this_thread::sleep_for(1ms); + } + + // Let subscriptions executor spin + executor_sub.spin_some(10ms); + + // The callback count should be 1 + EXPECT_EQ(1u, callback_count); + + // Reset callback count + callback_count = 0; + + // Publish 5 messages + for (int i = 0; i < 5; i++) { + publisher->publish(std::make_unique()); + std::this_thread::sleep_for(1ms); + } + + // Let subscriptions executor spin + executor_sub.spin_some(10ms); + + // The callback count should be 5, the queue shouldn't have been pruned + EXPECT_EQ(5u, callback_count); +} + +TEST_F(TestEventsQueue, SimpleQueue) +{ + // Create SimpleEventsQueue + auto simple_queue = std::make_unique(); + + // Create an events executor using the simple queue + EventsExecutor executor_sub(std::move(simple_queue)); + + // Create a subscription node with QoS->depth = 10 + auto node_sub = std::make_shared("node_sub"); + + size_t callback_count = 0; + + auto subscription = + node_sub->create_subscription( + "topic", + rclcpp::QoS(10), + [&](test_msgs::msg::Empty::SharedPtr) { + callback_count++; + }); + + // Add susbscription node to the executor, so the queue can start getting events + executor_sub.add_node(node_sub); + + // Create a publisher node + auto node_pub = std::make_shared("node_pub"); + auto publisher = node_pub->create_publisher("topic", rclcpp::QoS(10)); + + // Let subscriptions executor spin to execute any previous events + // not related to the subscription, so we start with an empty queue + executor_sub.spin_some(10ms); + + // Publish 11 messages, but as the subscription has only 10 as queue depth, + // we should lost a message (the events queue finish with 11 messages but one is not valid) + for (int i = 0; i < 11; i++) { + publisher->publish(std::make_unique()); + std::this_thread::sleep_for(1ms); + } + + // Let subscriptions executor spin + executor_sub.spin_some(10ms); + + // The callback count should be 10 + EXPECT_EQ(10u, callback_count); +} From 5780ccf0256d72c19be1b37006a95845a1c47389 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 5 Feb 2021 18:38:51 -0300 Subject: [PATCH 142/168] Set size limit in constructor --- .../buffers/bounded_events_queue.hpp | 20 +++++++------------ .../rclcpp/executors/test_events_queue.cpp | 3 +-- 2 files changed, 8 insertions(+), 15 deletions(-) diff --git a/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp index 2fb2970ba5..147389da5f 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp @@ -36,6 +36,12 @@ namespace buffers class BoundedEventsQueue : public EventsQueue { public: + RCLCPP_PUBLIC + explicit BoundedEventsQueue(size_t queue_size_limit) + { + queue_size_limit_ = queue_size_limit; + } + RCLCPP_PUBLIC ~BoundedEventsQueue() = default; @@ -121,21 +127,9 @@ class BoundedEventsQueue : public EventsQueue return local_queue; } - /** - * @brief sets the queue size limit - * @param limit The queue size limit - */ - RCLCPP_PUBLIC - void - set_queue_size_limit(size_t queue_size_limit) - { - queue_size_limit_ = queue_size_limit; - } - private: std::queue event_queue_; - - size_t queue_size_limit_ = 1000; + size_t queue_size_limit_; }; } // namespace buffers diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp index 5cf5db7b8e..bcdb2c6144 100644 --- a/rclcpp/test/rclcpp/executors/test_events_queue.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -41,8 +41,7 @@ class TestEventsQueue : public ::testing::Test TEST_F(TestEventsQueue, BoundedQueue) { // Create BoundedEventsQueue and set limit to 10 events. - auto bounded_queue = std::make_unique(); - bounded_queue->set_queue_size_limit(10); + auto bounded_queue = std::make_unique(10); // Create an events executor using the bounded queue EventsExecutor executor_sub(std::move(bounded_queue)); From 6fb2463ee4e436e319209f1c789f78a85e606c4f Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 8 Feb 2021 12:53:56 -0300 Subject: [PATCH 143/168] Test all events queue APIs --- .../buffers/bounded_events_queue.hpp | 2 +- .../buffers/simple_events_queue.hpp | 2 +- .../rclcpp/executors/test_events_queue.cpp | 159 +++++++++--------- 3 files changed, 77 insertions(+), 86 deletions(-) diff --git a/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp index 147389da5f..f153dfbdfb 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp @@ -18,7 +18,7 @@ #include #include -#include "rclcpp/executors/events_queue.hpp" +#include "rclcpp/experimental/buffers/events_queue.hpp" namespace rclcpp { diff --git a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp index b0685e7ad1..53c99dba7d 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp @@ -18,7 +18,7 @@ #include #include -#include "rclcpp/executors/events_queue.hpp" +#include "rclcpp/experimental/buffers/events_queue.hpp" namespace rclcpp { diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp index bcdb2c6144..4888d62c32 100644 --- a/rclcpp/test/rclcpp/executors/test_events_queue.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -12,18 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. #include -#include +#include + #include -#include "rclcpp/executors/events_executor.hpp" +#include "rclcpp/experimental/buffers/simple_events_queue.hpp" #include "rclcpp/experimental/buffers/bounded_events_queue.hpp" -#include "test_msgs/msg/empty.hpp" - using namespace std::chrono_literals; -using rclcpp::executors::EventsExecutor; - class TestEventsQueue : public ::testing::Test { public: @@ -38,109 +35,103 @@ class TestEventsQueue : public ::testing::Test } }; -TEST_F(TestEventsQueue, BoundedQueue) +TEST_F(TestEventsQueue, SimpleQueueTest) { - // Create BoundedEventsQueue and set limit to 10 events. - auto bounded_queue = std::make_unique(10); + // Create a SimpleEventsQueue and a local queue + auto simple_queue = std::make_unique(); + std::queue local_events_queue; + + // Make sure the queue is empty after init + simple_queue->init(); + EXPECT_TRUE(simple_queue->empty()); - // Create an events executor using the bounded queue - EventsExecutor executor_sub(std::move(bounded_queue)); + // Push 11 messages + for (int i = 0; i < 11; i++) { + rmw_listener_event_t stub_event; + simple_queue->push(stub_event); + } - // Create a subscription node - auto node_sub = std::make_shared("node_sub"); + // Pop one message + simple_queue->pop(); - size_t callback_count = 0; + local_events_queue = simple_queue->get_all_events(); - auto subscription = - node_sub->create_subscription( - "topic", - rclcpp::QoS(10), - [&](test_msgs::msg::Empty::SharedPtr) { - callback_count++; - }); + // We should have (11 - 1) events in the local queue + size_t local_queue_size = local_events_queue.size(); - // Add susbscription node to the executor, so the queue can start getting events - executor_sub.add_node(node_sub); + // The local queue size should be 10 + EXPECT_EQ(10u, local_queue_size); - // Create a publisher node - auto node_pub = std::make_shared("node_pub"); - auto publisher = node_pub->create_publisher("topic", rclcpp::QoS(10)); + // The simple queue should be empty after taking all events + EXPECT_TRUE(simple_queue->empty()); - // Let subscriptions executor spin to execute any previous events - // not related to the subscription, so we start with an empty queue - executor_sub.spin_some(10ms); + // Lets push an event into the queue and get it back + rmw_listener_event_t push_event = {simple_queue.get(), SUBSCRIPTION_EVENT}; - // Publish 11 messages, the eleventh msg should prune the queue - // and we should end up with only one event on it - for (int i = 0; i < 11; i++) { - publisher->publish(std::make_unique()); - std::this_thread::sleep_for(1ms); - } + simple_queue->push(push_event); - // Let subscriptions executor spin - executor_sub.spin_some(10ms); + rmw_listener_event_t front_event = simple_queue->front(); - // The callback count should be 1 - EXPECT_EQ(1u, callback_count); + // The events should be equal + EXPECT_EQ(push_event.entity, front_event.entity); + EXPECT_EQ(push_event.type, front_event.type); +} - // Reset callback count - callback_count = 0; - // Publish 5 messages - for (int i = 0; i < 5; i++) { - publisher->publish(std::make_unique()); - std::this_thread::sleep_for(1ms); +TEST_F(TestEventsQueue, BoundedQueueTest) +{ + // Create a BoundedEventsQueue with limit of 10 events and a local events queue + auto bounded_queue = std::make_unique(10); + std::queue local_events_queue; + + // Make sure the queue is empty after init + bounded_queue->init(); + EXPECT_TRUE(bounded_queue->empty()); + + // Push 11 messages, the eleventh msg should prune the queue + // and we should end up with only one event on it + for (int i = 0; i < 11; i++) { + rmw_listener_event_t stub_event; + bounded_queue->push(stub_event); } - // Let subscriptions executor spin - executor_sub.spin_some(10ms); + local_events_queue = bounded_queue->get_all_events(); - // The callback count should be 5, the queue shouldn't have been pruned - EXPECT_EQ(5u, callback_count); -} + size_t local_queue_size = local_events_queue.size(); -TEST_F(TestEventsQueue, SimpleQueue) -{ - // Create SimpleEventsQueue - auto simple_queue = std::make_unique(); + // The queue size should be 1 + EXPECT_EQ(1u, local_queue_size); + + // The bounded queue should be empty after taking all events + EXPECT_TRUE(bounded_queue->empty()); - // Create an events executor using the simple queue - EventsExecutor executor_sub(std::move(simple_queue)); + // Push 5 messages + for (int i = 0; i < 5; i++) { + rmw_listener_event_t stub_event; + bounded_queue->push(stub_event); + } - // Create a subscription node with QoS->depth = 10 - auto node_sub = std::make_shared("node_sub"); + // Pop one message + bounded_queue->pop(); - size_t callback_count = 0; + local_events_queue = bounded_queue->get_all_events(); - auto subscription = - node_sub->create_subscription( - "topic", - rclcpp::QoS(10), - [&](test_msgs::msg::Empty::SharedPtr) { - callback_count++; - }); + local_queue_size = local_events_queue.size(); - // Add susbscription node to the executor, so the queue can start getting events - executor_sub.add_node(node_sub); + // The local queue size should be 4 as the bounded queue shouldn't have been pruned + EXPECT_EQ(4u, local_queue_size); - // Create a publisher node - auto node_pub = std::make_shared("node_pub"); - auto publisher = node_pub->create_publisher("topic", rclcpp::QoS(10)); + // The bounded queue should be empty after taking all events + EXPECT_TRUE(bounded_queue->empty()); - // Let subscriptions executor spin to execute any previous events - // not related to the subscription, so we start with an empty queue - executor_sub.spin_some(10ms); + // Lets push an event into the queue and get it back + rmw_listener_event_t push_event = {bounded_queue.get(), WAITABLE_EVENT}; - // Publish 11 messages, but as the subscription has only 10 as queue depth, - // we should lost a message (the events queue finish with 11 messages but one is not valid) - for (int i = 0; i < 11; i++) { - publisher->publish(std::make_unique()); - std::this_thread::sleep_for(1ms); - } + bounded_queue->push(push_event); - // Let subscriptions executor spin - executor_sub.spin_some(10ms); + rmw_listener_event_t front_event = bounded_queue->front(); - // The callback count should be 10 - EXPECT_EQ(10u, callback_count); + // The events should be the equal + EXPECT_EQ(push_event.entity, front_event.entity); + EXPECT_EQ(push_event.type, front_event.type); } From c4ac6f9ee7f4836cc8aa0b7ff7a7d1e4093a0005 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 8 Feb 2021 13:55:51 -0300 Subject: [PATCH 144/168] Remove BoundedQueue, not ready yet for this --- .../buffers/bounded_events_queue.hpp | 140 ------------------ .../rclcpp/executors/test_events_queue.cpp | 60 -------- 2 files changed, 200 deletions(-) delete mode 100644 rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp diff --git a/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp deleted file mode 100644 index f153dfbdfb..0000000000 --- a/rclcpp/include/rclcpp/experimental/buffers/bounded_events_queue.hpp +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright 2021 Open Source Robotics Foundation, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#ifndef RCLCPP__EXPERIMENTAL__BUFFERS__BOUNDED_EVENTS_QUEUE_HPP_ -#define RCLCPP__EXPERIMENTAL__BUFFERS__BOUNDED_EVENTS_QUEUE_HPP_ - -#include -#include - -#include "rclcpp/experimental/buffers/events_queue.hpp" - -namespace rclcpp -{ -namespace experimental -{ -namespace buffers -{ - -/** - * @brief This class provides a bounded queue implementation - * based on a std::queue. Before pushing events into the queue - * checks the queue size. In case of exceeding the size it performs - * a prune of the queue. - */ -class BoundedEventsQueue : public EventsQueue -{ -public: - RCLCPP_PUBLIC - explicit BoundedEventsQueue(size_t queue_size_limit) - { - queue_size_limit_ = queue_size_limit; - } - - RCLCPP_PUBLIC - ~BoundedEventsQueue() = default; - - /** - * @brief push event into the queue - * @param event The event to push into the queue - */ - RCLCPP_PUBLIC - virtual - void - push(const rmw_listener_event_t & event) - { - if (event_queue_.size() >= queue_size_limit_) { - // Simple prune strategy: Remove all elements in the queue. - this->init(); - } - event_queue_.push(event); - } - - /** - * @brief removes front element from the queue - * The element removed is the "oldest" element in the queue whose - * value can be retrieved by calling member front(). - */ - RCLCPP_PUBLIC - virtual - void - pop() - { - event_queue_.pop(); - } - - /** - * @brief gets the front event from the queue - * @return the front event - */ - RCLCPP_PUBLIC - virtual - rmw_listener_event_t - front() const - { - return event_queue_.front(); - } - - /** - * @brief Test whether queue is empty - * @return true if the queue's size is 0, false otherwise. - */ - RCLCPP_PUBLIC - virtual - bool - empty() const - { - return event_queue_.empty(); - } - - /** - * @brief Initializes the queue - */ - RCLCPP_PUBLIC - virtual - void - init() - { - // Make sure the queue is empty when we start - std::queue local_queue; - std::swap(event_queue_, local_queue); - } - - - /** - * @brief gets a queue with all events accumulated on it since - * the last call. The member queue is empty when the call returns. - * @return std::queue with events - */ - RCLCPP_PUBLIC - virtual - std::queue - get_all_events() - { - std::queue local_queue; - std::swap(event_queue_, local_queue); - return local_queue; - } - -private: - std::queue event_queue_; - size_t queue_size_limit_; -}; - -} // namespace buffers -} // namespace experimental -} // namespace rclcpp - - -#endif // RCLCPP__EXPERIMENTAL__BUFFERS__BOUNDED_EVENTS_QUEUE_HPP_ diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp index 4888d62c32..7996731ea2 100644 --- a/rclcpp/test/rclcpp/executors/test_events_queue.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -17,7 +17,6 @@ #include #include "rclcpp/experimental/buffers/simple_events_queue.hpp" -#include "rclcpp/experimental/buffers/bounded_events_queue.hpp" using namespace std::chrono_literals; @@ -76,62 +75,3 @@ TEST_F(TestEventsQueue, SimpleQueueTest) EXPECT_EQ(push_event.entity, front_event.entity); EXPECT_EQ(push_event.type, front_event.type); } - - -TEST_F(TestEventsQueue, BoundedQueueTest) -{ - // Create a BoundedEventsQueue with limit of 10 events and a local events queue - auto bounded_queue = std::make_unique(10); - std::queue local_events_queue; - - // Make sure the queue is empty after init - bounded_queue->init(); - EXPECT_TRUE(bounded_queue->empty()); - - // Push 11 messages, the eleventh msg should prune the queue - // and we should end up with only one event on it - for (int i = 0; i < 11; i++) { - rmw_listener_event_t stub_event; - bounded_queue->push(stub_event); - } - - local_events_queue = bounded_queue->get_all_events(); - - size_t local_queue_size = local_events_queue.size(); - - // The queue size should be 1 - EXPECT_EQ(1u, local_queue_size); - - // The bounded queue should be empty after taking all events - EXPECT_TRUE(bounded_queue->empty()); - - // Push 5 messages - for (int i = 0; i < 5; i++) { - rmw_listener_event_t stub_event; - bounded_queue->push(stub_event); - } - - // Pop one message - bounded_queue->pop(); - - local_events_queue = bounded_queue->get_all_events(); - - local_queue_size = local_events_queue.size(); - - // The local queue size should be 4 as the bounded queue shouldn't have been pruned - EXPECT_EQ(4u, local_queue_size); - - // The bounded queue should be empty after taking all events - EXPECT_TRUE(bounded_queue->empty()); - - // Lets push an event into the queue and get it back - rmw_listener_event_t push_event = {bounded_queue.get(), WAITABLE_EVENT}; - - bounded_queue->push(push_event); - - rmw_listener_event_t front_event = bounded_queue->front(); - - // The events should be the equal - EXPECT_EQ(push_event.entity, front_event.entity); - EXPECT_EQ(push_event.type, front_event.type); -} From 46e3b48ba8cebf67c4bfe32e243b98c018e54f93 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 8 Feb 2021 14:22:25 -0300 Subject: [PATCH 145/168] rename rmw_listener_cb_t->rmw_listener_callback_t --- rclcpp/include/rclcpp/client.hpp | 2 +- .../rclcpp/executors/events_executor_notify_waitable.hpp | 2 +- .../rclcpp/experimental/subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/qos_event.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- rclcpp/src/rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- 14 files changed, 14 insertions(+), 14 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 44b2724b7b..e5e7bd50a3 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -160,7 +160,7 @@ class ClientBase void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const; + rmw_listener_callback_t executor_callback) const; protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index ee7ed4a6dc..3e868314ab 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -62,7 +62,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const override + rmw_listener_callback_t executor_callback) const override { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 4c5713f13b..cca9a6cc46 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -78,7 +78,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const override; + rmw_listener_callback_t executor_callback) const override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 9afcecfbf0..4883c7f184 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -112,7 +112,7 @@ class QOSEventHandlerBase : public Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const override; + rmw_listener_callback_t executor_callback) const override; protected: rcl_event_t event_handle_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 8ffbef820f..35efd0cc8d 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -130,7 +130,7 @@ class ServiceBase void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const; + rmw_listener_callback_t executor_callback) const; protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 81d3be4bec..ea60a062cc 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -273,7 +273,7 @@ class SubscriptionBase : public std::enable_shared_from_this void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const; + rmw_listener_callback_t executor_callback) const; protected: template diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index e4d61f1732..8e6608ee62 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -213,7 +213,7 @@ class Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const; + rmw_listener_callback_t executor_callback) const; private: std::atomic in_use_by_wait_set_{false}; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 14f8e02b42..789919c0b6 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -202,7 +202,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const + rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_client_set_listener_callback( client_handle_.get(), diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 1fbfa6559c..82b4ed7eb9 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -71,7 +71,7 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) void QOSEventHandlerBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const + rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_event_set_listener_callback( &event_handle_, diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 506370325a..766de70826 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -88,7 +88,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const + rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_service_set_listener_callback( service_handle_.get(), diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index e266a3d612..0eb2b3a62e 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -292,7 +292,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const + rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_subscription_set_listener_callback( subscription_handle_.get(), diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 3aa9dc22b3..876e73ff8b 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -40,7 +40,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const + rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index e30ff05f4e..040716c40f 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -63,7 +63,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const + rmw_listener_callback_t executor_callback) const { (void)executor; (void)executor_callback; diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index a8393974d7..bbb68298ee 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -470,7 +470,7 @@ class TestWaitable : public rclcpp::Waitable void set_events_executor_callback( const rclcpp::executors::EventsExecutor * executor, - rmw_listener_cb_t executor_callback) const override + rmw_listener_callback_t executor_callback) const override { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, From a32211990a1f3f16675af100c528ec2f836044ca Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 11 Feb 2021 14:30:33 -0300 Subject: [PATCH 146/168] Ctests fixes --- rclcpp/include/rclcpp/any_subscription_callback.hpp | 4 ++-- .../experimental/buffers/ring_buffer_implementation.hpp | 6 +++--- rclcpp/test/rclcpp/executors/test_events_queue.cpp | 5 +++-- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/rclcpp/include/rclcpp/any_subscription_callback.hpp b/rclcpp/include/rclcpp/any_subscription_callback.hpp index 9b6cb8691f..5c8b6460b1 100644 --- a/rclcpp/include/rclcpp/any_subscription_callback.hpp +++ b/rclcpp/include/rclcpp/any_subscription_callback.hpp @@ -187,7 +187,7 @@ class AnySubscriptionCallback TRACEPOINT(callback_start, static_cast(this), true); // If the message is not valid, return. - if(!message) { + if (!message) { return; } @@ -216,7 +216,7 @@ class AnySubscriptionCallback TRACEPOINT(callback_start, static_cast(this), true); // If the message is not valid, return. - if(!message) { + if (!message) { return; } diff --git a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp index ece0e379dc..4ed7245c48 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/ring_buffer_implementation.hpp @@ -93,9 +93,9 @@ class RingBufferImplementation : public BufferImplementationBase RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "Calling dequeue on empty intra-process buffer"); // This situation can happen on the EventsExecutor if we have more events in the queue // than messages in the history cache (set by the qos_policies.depth of the subscription) - // For example if we set depth=1 and we get 2 messages really fast (so no time for processing), - // we could end up with 2 events in the queue but only 1 msg is actually stored on the buffer. - // In this case we return an empty buffer. + // For example if we set depth=1 and we get 2 messages really fast (so no time for + // processing), we could end up with 2 events in the queue but only 1 msg is + // actually stored on the buffer. In this case we return an empty buffer. return BufferT(); } diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp index 7996731ea2..4d22a5eb1e 100644 --- a/rclcpp/test/rclcpp/executors/test_events_queue.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -11,11 +11,12 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. +#include + #include +#include #include -#include - #include "rclcpp/experimental/buffers/simple_events_queue.hpp" using namespace std::chrono_literals; From 1bc6294e99acfac9adc04239488a0219462cf150 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 15 Feb 2021 12:02:14 -0300 Subject: [PATCH 147/168] Check validity of executor pointer --- rclcpp/include/rclcpp/executors/events_executor.hpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 6b5ef5ec11..cdf357f9dd 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -187,6 +187,11 @@ class EventsExecutor : public rclcpp::Executor static void push_event(const void * executor_ptr, rmw_listener_event_t event) { + // Check if the executor pointer is not valid + if (!executor_ptr) { + throw std::runtime_error("The executor pointer is not valid."); + } + // Cast executor_ptr to this (need to remove constness) auto this_executor = const_cast( static_cast(executor_ptr)); From f13ffb25d30b03f751e97faa992cdfa886942cea Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 15 Feb 2021 15:14:29 -0300 Subject: [PATCH 148/168] use void * to pass executor ptr --- rclcpp/include/rclcpp/client.hpp | 2 +- rclcpp/include/rclcpp/executors/events_executor.hpp | 6 ++---- .../rclcpp/executors/events_executor_notify_waitable.hpp | 2 +- .../rclcpp/experimental/subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/qos_event.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- rclcpp/src/rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- 15 files changed, 16 insertions(+), 18 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index e5e7bd50a3..47b657fe5c 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -159,7 +159,7 @@ class ClientBase RCLCPP_PUBLIC void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const; protected: diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index cdf357f9dd..b257bf9789 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -185,16 +185,14 @@ class EventsExecutor : public rclcpp::Executor // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. static void - push_event(const void * executor_ptr, rmw_listener_event_t event) + push_event(void * executor_ptr, rmw_listener_event_t event) { // Check if the executor pointer is not valid if (!executor_ptr) { throw std::runtime_error("The executor pointer is not valid."); } - // Cast executor_ptr to this (need to remove constness) - auto this_executor = const_cast( - static_cast(executor_ptr)); + auto this_executor = static_cast(executor_ptr); // Event queue mutex scope { diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 3e868314ab..b9987d4bd4 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -61,7 +61,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable RCLCPP_PUBLIC void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const override { for (auto gc : notify_guard_conditions_) { diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index cca9a6cc46..645d6b65cc 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -77,7 +77,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const override; protected: diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 4883c7f184..9a3716528e 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -111,7 +111,7 @@ class QOSEventHandlerBase : public Waitable RCLCPP_PUBLIC void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const override; protected: diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 35efd0cc8d..00d4227e62 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -129,7 +129,7 @@ class ServiceBase RCLCPP_PUBLIC void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const; protected: diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index ea60a062cc..fa7789ceb6 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -272,7 +272,7 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const; protected: diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 8e6608ee62..449fcb5b46 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -212,7 +212,7 @@ class Waitable virtual void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const; private: diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index 789919c0b6..dd9e7d4e7e 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -201,7 +201,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_client_set_listener_callback( diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 82b4ed7eb9..d9e27dc32f 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -70,7 +70,7 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) void QOSEventHandlerBase::set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_event_set_listener_callback( diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 766de70826..e0afdbcad9 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -87,7 +87,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_service_set_listener_callback( diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 0eb2b3a62e..5f37848726 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -291,7 +291,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_subscription_set_listener_callback( diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 876e73ff8b..1c906e95cd 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -39,7 +39,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 040716c40f..4c5bd672ce 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -62,7 +62,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const { (void)executor; diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index bbb68298ee..0be4691152 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -469,7 +469,7 @@ class TestWaitable : public rclcpp::Waitable void set_events_executor_callback( - const rclcpp::executors::EventsExecutor * executor, + rclcpp::executors::EventsExecutor * executor, rmw_listener_callback_t executor_callback) const override { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( From 76a833dcd6c1426c9b21548a80492d1a05e76b8c Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Wed, 17 Feb 2021 15:13:27 -0300 Subject: [PATCH 149/168] Proper spin_some behaviour on EventsExecutor --- .../rclcpp/executors/events_executor.hpp | 13 ++-- .../src/rclcpp/executors/events_executor.cpp | 67 ++++++++++--------- .../rclcpp/executors/test_events_executor.cpp | 6 ++ .../test/rclcpp/executors/test_executors.cpp | 5 -- 4 files changed, 51 insertions(+), 40 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index b257bf9789..a3f7c87c71 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -82,10 +82,15 @@ class EventsExecutor : public rclcpp::Executor /// Events executor implementation of spin some /** - * executor.provide_callbacks(); - * while(condition) { - * executor.spin_some(); - * } + * This function will execute ready timers and events + * until timeout or no more work available. + * + * Example: + * while(condition) { + * spin_some(); + * sleep(); // User should have some sync work or + * // sleep to avoid a 100% CPU usage + * } */ RCLCPP_PUBLIC void diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index f70cd2772f..2febbeb212 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -60,9 +60,6 @@ EventsExecutor::spin() // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !events_queue_->empty();}; - // Local event queue to allow entities to push events while we execute them - EventQueue execution_event_queue; - timers_manager_->start(); while (rclcpp::ok(context_) && spinning.load()) { @@ -70,7 +67,7 @@ EventsExecutor::spin() // We wait here until something has been pushed to the event queue events_queue_cv_.wait(push_lock, has_event_predicate); // Local event queue to allow entities to push events while we execute them - execution_event_queue = events_queue_->get_all_events(); + EventQueue execution_event_queue = events_queue_->get_all_events(); // Unlock the mutex push_lock.unlock(); // Consume all available events, this queue will be empty at the end of the function @@ -92,33 +89,44 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) max_duration = timers_manager_->MAX_TIME; } - // This function will wait until the first of the following events occur: - // - The input max_duration is elapsed - // - A timer triggers - // - An executor event is received and processed + auto start = std::chrono::steady_clock::now(); - // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() {return !events_queue_->empty();}; + auto max_duration_not_elapsed = [max_duration, start]() { + auto elapsed_time = std::chrono::steady_clock::now() - start; + return elapsed_time < max_duration; + }; + // Execute events until timeout or no more work available + while (max_duration_not_elapsed()) { + bool work_available = false; - // Select the smallest between input max_duration and timer timeout - auto next_timer_timeout = timers_manager_->get_head_timeout(); - if (next_timer_timeout < max_duration) { - max_duration = next_timer_timeout; - } + // Check for ready timers + auto next_timer_timeout = timers_manager_->get_head_timeout(); + + bool timer_ready = next_timer_timeout < 0ns; + + if (timer_ready) { + timers_manager_->execute_head_timer(); + work_available = true; + } else { + // Execute first event from queue if it exists + std::unique_lock lock(push_mutex_); - std::unique_lock push_lock(push_mutex_); - // Wait until timeout or event - events_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); - // Local event queue to allow entities to push events while we execute them - EventQueue execution_event_queue = events_queue_->get_all_events(); - // We don't need the lock anymore - push_lock.unlock(); - - // Execute all ready timers - timers_manager_->execute_ready_timers(); - // Consume all available events, this queue will be empty at the end of the function - this->consume_all_events(execution_event_queue); + bool has_event = !events_queue_->empty(); + + if (has_event) { + rmw_listener_event_t event = events_queue_->front(); + events_queue_->pop(); + this->execute_event(event); + work_available = true; + } + } + + // If there's no more work available, exit + if (!work_available) { + break; + } + } } void @@ -136,9 +144,6 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !events_queue_->empty();}; - // Local event queue to allow entities to push events while we execute them - EventQueue execution_event_queue; - auto start = std::chrono::steady_clock::now(); auto max_duration_not_elapsed = [max_duration, start]() { auto elapsed_time = std::chrono::steady_clock::now() - start; @@ -162,7 +167,7 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) // Keep executing until no more work to do or timeout expired while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { std::unique_lock push_lock(push_mutex_); - execution_event_queue = events_queue_->get_all_events(); + EventQueue execution_event_queue = events_queue_->get_all_events(); push_lock.unlock(); // Exit if there is no more work to do diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index 8fad5b40b0..ffae94eb3c 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -187,6 +187,9 @@ TEST_F(TestEventsExecutor, spin_some_max_duration) t_runs++; }); + // Sleep some time for the timer to be ready when spin + std::this_thread::sleep_for(10ms); + EventsExecutor executor; executor.add_node(node); @@ -209,6 +212,9 @@ TEST_F(TestEventsExecutor, spin_some_zero_duration) t_runs++; }); + // Sleep some time for the timer to be ready when spin + std::this_thread::sleep_for(20ms); + EventsExecutor executor; executor.add_node(node); executor.spin_some(0ms); diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 0be4691152..b6b45244fb 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -547,11 +547,6 @@ TYPED_TEST(TestExecutorsStable, spinSome) { spin_exited = true; }); - // Give some time for executor to start spinning - // otherwise when it will start looking for work to do it will already find - // more than 1 notification - std::this_thread::sleep_for(10ms); - // Do some work until sufficient calls to the waitable occur, but keep going until either // count becomes too large, spin exits, or the 1 second timeout completes. auto start = std::chrono::steady_clock::now(); From 00873929a4ab4f74e0ee0a79e8734ddad7767f83 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 18 Feb 2021 17:41:53 -0300 Subject: [PATCH 150/168] rework spin_some & spin_all --- .../rclcpp/executors/events_executor.hpp | 20 +++- .../rclcpp/executors/timers_manager.hpp | 8 ++ .../experimental/buffers/events_queue.hpp | 9 ++ .../buffers/simple_events_queue.hpp | 12 +++ .../src/rclcpp/executors/events_executor.cpp | 97 ++++++++++--------- .../src/rclcpp/executors/timers_manager.cpp | 37 +++++++ .../rclcpp/executors/test_events_executor.cpp | 4 +- 7 files changed, 138 insertions(+), 49 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index a3f7c87c71..c71f7dfff9 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -82,8 +82,10 @@ class EventsExecutor : public rclcpp::Executor /// Events executor implementation of spin some /** - * This function will execute ready timers and events - * until timeout or no more work available. + * This non-blocking function will execute the timers and events + * that were ready when this API was called, until timeout or no + * more work available. New ready-timers/events arrived while + * executing work, won't be taken into account here. * * Example: * while(condition) { @@ -96,6 +98,20 @@ class EventsExecutor : public rclcpp::Executor void spin_some(std::chrono::nanoseconds max_duration = std::chrono::nanoseconds(0)) override; + /// Events executor implementation of spin all + /** + * This non-blocking function will execute timers and events + * until timeout or no more work available. If new ready-timers/events + * arrive while executing work available, they will be executed + * as long as the timeout hasn't expired. + * + * Example: + * while(condition) { + * spin_all(); + * sleep(); // User should have some sync work or + * // sleep to avoid a 100% CPU usage + * } + */ RCLCPP_PUBLIC void spin_all(std::chrono::nanoseconds max_duration) override; diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index b4be64aee3..eb34dfcd57 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -127,6 +127,14 @@ class TimersManager */ void remove_timer(rclcpp::TimerBase::SharedPtr timer); + /** + * @brief Executes head timer if ready at time point. + * @param tp the timepoint to check for + * @return true if head timer was ready at tp + */ + bool execute_head_timer_if_ready_at_tp( + std::chrono::time_point tp); + // This is what the TimersManager uses to denote a duration forever. // We don't use std::chrono::nanoseconds::max because it will overflow. // See https://en.cppreference.com/w/cpp/thread/condition_variable/wait_for diff --git a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp index d00a00705d..77a5efd695 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp @@ -83,6 +83,15 @@ class EventsQueue bool empty() const = 0; + /** + * @brief Returns the number of elements in the queue. + * @return the number of elements in the queue. + */ + RCLCPP_PUBLIC + virtual + size_t + size() const = 0; + /** * @brief Initializes the queue */ diff --git a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp index 53c99dba7d..5bb0a85b51 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp @@ -89,6 +89,18 @@ class SimpleEventsQueue : public EventsQueue return event_queue_.empty(); } + /** + * @brief Returns the number of elements in the queue. + * @return the number of elements in the queue. + */ + RCLCPP_PUBLIC + virtual + size_t + size() const + { + return event_queue_.size(); + } + /** * @brief Initializes the queue */ diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 2febbeb212..0723d5fd8d 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -96,20 +96,23 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) return elapsed_time < max_duration; }; - // Execute events until timeout or no more work available - while (max_duration_not_elapsed()) { - bool work_available = false; + // Get the number of events ready at this time point + std::unique_lock lock(push_mutex_); + size_t available_events_at_tp = events_queue_->size(); + lock.unlock(); - // Check for ready timers - auto next_timer_timeout = timers_manager_->get_head_timeout(); + size_t executed_events = 0; - bool timer_ready = next_timer_timeout < 0ns; + // Checks if all events that were ready when spin_some was called, were executed. + auto executed_ready_events_at_tp = [executed_events, available_events_at_tp]() { + return executed_events >= available_events_at_tp; + }; - if (timer_ready) { - timers_manager_->execute_head_timer(); - work_available = true; - } else { - // Execute first event from queue if it exists + // Execute events and timers ready when spin_some was called, + // until timeout or no more work available. + while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { + // Execute first event from queue if it exists + if (!executed_ready_events_at_tp()) { std::unique_lock lock(push_mutex_); bool has_event = !events_queue_->empty(); @@ -117,69 +120,71 @@ EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) if (has_event) { rmw_listener_event_t event = events_queue_->front(); events_queue_->pop(); + // std::cout << "Execute event" << std::endl; this->execute_event(event); - work_available = true; + executed_events++; + continue; } } - // If there's no more work available, exit - if (!work_available) { - break; + // Execute timer, if was ready at start + if (timers_manager_->execute_head_timer_if_ready_at_tp(start)) { + continue; } + + // If there's no more work available, exit + break; } } void EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) { - if (max_duration <= 0ns) { - throw std::invalid_argument("max_duration must be positive"); + if (spinning.exchange(true)) { + throw std::runtime_error("spin_all() called while already spinning"); } - if (spinning.exchange(true)) { - throw std::runtime_error("spin_some() called while already spinning"); + if (max_duration < 0ns) { + throw std::invalid_argument("max_duration must be positive"); } + RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - // When condition variable is notified, check this predicate to proceed - auto has_event_predicate = [this]() {return !events_queue_->empty();}; + // In this context a 0 input max_duration means no duration limit + if (std::chrono::nanoseconds(0) == max_duration) { + max_duration = timers_manager_->MAX_TIME; + } auto start = std::chrono::steady_clock::now(); + auto max_duration_not_elapsed = [max_duration, start]() { auto elapsed_time = std::chrono::steady_clock::now() - start; return elapsed_time < max_duration; }; - // Select the smallest between input max duration and timer timeout - auto next_timer_timeout = timers_manager_->get_head_timeout(); - if (next_timer_timeout < max_duration) { - max_duration = next_timer_timeout; - } - - { - // Wait once until timeout or event - std::unique_lock push_lock(push_mutex_); - events_queue_cv_.wait_for(push_lock, max_duration, has_event_predicate); - } + // Execute timer and events until timeout or no more work available + while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { + // Execute first event from queue if it exists + { + std::unique_lock lock(push_mutex_); - auto timeout = timers_manager_->get_head_timeout(); + bool has_event = !events_queue_->empty(); - // Keep executing until no more work to do or timeout expired - while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { - std::unique_lock push_lock(push_mutex_); - EventQueue execution_event_queue = events_queue_->get_all_events(); - push_lock.unlock(); + if (has_event) { + rmw_listener_event_t event = events_queue_->front(); + events_queue_->pop(); + this->execute_event(event); + continue; + } + } - // Exit if there is no more work to do - const bool ready_timer = timeout < 0ns; - const bool has_events = !execution_event_queue.empty(); - if (!ready_timer && !has_events) { - break; + // Execute timer, if was ready + if (timers_manager_->execute_head_timer()) { + continue; } - // Execute all ready work - timeout = timers_manager_->execute_ready_timers(); - this->consume_all_events(execution_event_queue); + // If there's no more work available, exit + break; } } diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 7e29ec0f23..e556935800 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -230,3 +230,40 @@ void TimersManager::remove_timer(TimerPtr timer) timers_cv_.notify_one(); } } + +bool TimersManager::execute_head_timer_if_ready_at_tp( + std::chrono::time_point timepoint) +{ + // Do not allow to interfere with the thread running + if (running_) { + throw std::runtime_error( + "TimersManager::execute_head_timer_if_ready_at_tp() can't" + "be used while timers thread is running"); + } + + std::unique_lock lock(timers_mutex_); + + TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); + + // Nothing to do if we don't have any timer + if (timers_heap.empty()) { + return false; + } + + TimerPtr head = timers_heap.front(); + + // A ready timer will return a negative duration when calling time_until_trigger + auto timepoint_timer_ready = std::chrono::steady_clock::now() + head->time_until_trigger(); + + auto timer_was_ready_at_tp = timepoint_timer_ready < timepoint; + + if (timer_was_ready_at_tp) { + // Head timer is ready, execute + head->execute_callback(); + timers_heap.heapify_root(); + weak_timers_heap_.store(timers_heap); + return true; + } else { + return false; + } +} diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index ffae94eb3c..e2bec72756 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -254,6 +254,9 @@ TEST_F(TestEventsExecutor, spin_all_max_duration) t_runs++; }); + // Sleep some time for the timer to be ready when spin + std::this_thread::sleep_for(10ms); + EventsExecutor executor; executor.add_node(node); @@ -265,7 +268,6 @@ TEST_F(TestEventsExecutor, spin_all_max_duration) } EventsExecutor executor; - EXPECT_THROW(executor.spin_all(0ms), std::invalid_argument); EXPECT_THROW(executor.spin_all(-5ms), std::invalid_argument); } From d0e8e43980ab61ef6b7ac852a2541f051c549b54 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 19 Feb 2021 12:49:21 -0300 Subject: [PATCH 151/168] spin_some and spin_all call spin_some_impl --- .../rclcpp/executors/events_executor.hpp | 4 + .../rclcpp/executors/timers_manager.hpp | 19 ++-- .../src/rclcpp/executors/events_executor.cpp | 100 ++++++------------ .../src/rclcpp/executors/timers_manager.cpp | 56 +++------- .../rclcpp/executors/test_events_executor.cpp | 1 + .../test/rclcpp/executors/test_executors.cpp | 1 + 6 files changed, 64 insertions(+), 117 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index c71f7dfff9..d6ded52be8 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -197,6 +197,10 @@ class EventsExecutor : public rclcpp::Executor void spin_once_impl(std::chrono::nanoseconds timeout) override; + RCLCPP_PUBLIC + void + spin_some_impl(std::chrono::nanoseconds max_duration, bool exhaustive); + private: RCLCPP_DISABLE_COPY(EventsExecutor) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index eb34dfcd57..4e50b2922e 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -101,11 +101,12 @@ class TimersManager std::chrono::nanoseconds execute_ready_timers(); /** - * @brief Executes one ready timer if available - * - * @return true if there was a timer ready + * @brief Executes head timer if ready at time point. + * @param tp the timepoint to check for + * @return true if head timer was ready at tp */ - bool execute_head_timer(); + bool execute_head_timer( + std::chrono::time_point tp = TimePoint::max()); /** * @brief Get the amount of time before the next timer expires. @@ -127,19 +128,13 @@ class TimersManager */ void remove_timer(rclcpp::TimerBase::SharedPtr timer); - /** - * @brief Executes head timer if ready at time point. - * @param tp the timepoint to check for - * @return true if head timer was ready at tp - */ - bool execute_head_timer_if_ready_at_tp( - std::chrono::time_point tp); - // This is what the TimersManager uses to denote a duration forever. // We don't use std::chrono::nanoseconds::max because it will overflow. // See https://en.cppreference.com/w/cpp/thread/condition_variable/wait_for static constexpr std::chrono::nanoseconds MAX_TIME = std::chrono::hours(90); + using TimePoint = std::chrono::time_point; + private: RCLCPP_DISABLE_COPY(TimersManager) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 0723d5fd8d..d4dcdd8b90 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -79,82 +79,32 @@ EventsExecutor::spin() void EventsExecutor::spin_some(std::chrono::nanoseconds max_duration) { - if (spinning.exchange(true)) { - throw std::runtime_error("spin_some() called while already spinning"); - } - RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - // In this context a 0 input max_duration means no duration limit if (std::chrono::nanoseconds(0) == max_duration) { max_duration = timers_manager_->MAX_TIME; } - auto start = std::chrono::steady_clock::now(); - - auto max_duration_not_elapsed = [max_duration, start]() { - auto elapsed_time = std::chrono::steady_clock::now() - start; - return elapsed_time < max_duration; - }; - - // Get the number of events ready at this time point - std::unique_lock lock(push_mutex_); - size_t available_events_at_tp = events_queue_->size(); - lock.unlock(); - - size_t executed_events = 0; - - // Checks if all events that were ready when spin_some was called, were executed. - auto executed_ready_events_at_tp = [executed_events, available_events_at_tp]() { - return executed_events >= available_events_at_tp; - }; - - // Execute events and timers ready when spin_some was called, - // until timeout or no more work available. - while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { - // Execute first event from queue if it exists - if (!executed_ready_events_at_tp()) { - std::unique_lock lock(push_mutex_); - - bool has_event = !events_queue_->empty(); - - if (has_event) { - rmw_listener_event_t event = events_queue_->front(); - events_queue_->pop(); - // std::cout << "Execute event" << std::endl; - this->execute_event(event); - executed_events++; - continue; - } - } - - // Execute timer, if was ready at start - if (timers_manager_->execute_head_timer_if_ready_at_tp(start)) { - continue; - } - - // If there's no more work available, exit - break; - } + return this->spin_some_impl(max_duration, false); } void EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) { - if (spinning.exchange(true)) { - throw std::runtime_error("spin_all() called while already spinning"); + if (max_duration <= 0ns) { + throw std::invalid_argument("max_duration must be positive"); } + return this->spin_some_impl(max_duration, true); +} - if (max_duration < 0ns) { - throw std::invalid_argument("max_duration must be positive"); +void +EventsExecutor::spin_some_impl(std::chrono::nanoseconds max_duration, bool exhaustive) +{ + if (spinning.exchange(true)) { + throw std::runtime_error("spin_some() called while already spinning"); } RCLCPP_SCOPE_EXIT(this->spinning.store(false);); - // In this context a 0 input max_duration means no duration limit - if (std::chrono::nanoseconds(0) == max_duration) { - max_duration = timers_manager_->MAX_TIME; - } - auto start = std::chrono::steady_clock::now(); auto max_duration_not_elapsed = [max_duration, start]() { @@ -162,24 +112,42 @@ EventsExecutor::spin_all(std::chrono::nanoseconds max_duration) return elapsed_time < max_duration; }; - // Execute timer and events until timeout or no more work available + size_t ready_events_at_start = 0; + size_t executed_events = 0; + + if (!exhaustive) { + // Get the number of events ready at start + std::unique_lock lock(push_mutex_); + ready_events_at_start = events_queue_->size(); + lock.unlock(); + } + while (rclcpp::ok(context_) && spinning.load() && max_duration_not_elapsed()) { - // Execute first event from queue if it exists - { + // Execute first ready event from queue if exists + if (exhaustive || (executed_events < ready_events_at_start)) { std::unique_lock lock(push_mutex_); - bool has_event = !events_queue_->empty(); if (has_event) { rmw_listener_event_t event = events_queue_->front(); events_queue_->pop(); this->execute_event(event); + executed_events++; continue; } } - // Execute timer, if was ready - if (timers_manager_->execute_head_timer()) { + bool timer_executed; + + if (exhaustive) { + // Execute timer if is ready + timer_executed = timers_manager_->execute_head_timer(); + } else { + // Execute timer if was ready at start + timer_executed = timers_manager_->execute_head_timer(start); + } + + if (timer_executed) { continue; } diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index e556935800..7026f8bd90 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -116,7 +116,8 @@ std::chrono::nanoseconds TimersManager::execute_ready_timers() return this->get_head_timeout_unsafe(timers_heap); } -bool TimersManager::execute_head_timer() +bool TimersManager::execute_head_timer( + std::chrono::time_point timepoint) { // Do not allow to interfere with the thread running if (running_) { @@ -127,13 +128,27 @@ bool TimersManager::execute_head_timer() std::unique_lock lock(timers_mutex_); TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); + // Nothing to do if we don't have any timer if (timers_heap.empty()) { return false; } TimerPtr head = timers_heap.front(); - if (head->is_ready()) { + + bool timer_ready = false; + + if (timepoint != TimePoint::max()) { + // A ready timer will return a negative duration when calling time_until_trigger + auto timepoint_timer_ready = std::chrono::steady_clock::now() + head->time_until_trigger(); + + // Here we check if the timer was already ready at timepoint + timer_ready = timepoint_timer_ready < timepoint; + } else { + timer_ready = head->is_ready(); + } + + if (timer_ready) { // Head timer is ready, execute head->execute_callback(); timers_heap.heapify_root(); @@ -230,40 +245,3 @@ void TimersManager::remove_timer(TimerPtr timer) timers_cv_.notify_one(); } } - -bool TimersManager::execute_head_timer_if_ready_at_tp( - std::chrono::time_point timepoint) -{ - // Do not allow to interfere with the thread running - if (running_) { - throw std::runtime_error( - "TimersManager::execute_head_timer_if_ready_at_tp() can't" - "be used while timers thread is running"); - } - - std::unique_lock lock(timers_mutex_); - - TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); - - // Nothing to do if we don't have any timer - if (timers_heap.empty()) { - return false; - } - - TimerPtr head = timers_heap.front(); - - // A ready timer will return a negative duration when calling time_until_trigger - auto timepoint_timer_ready = std::chrono::steady_clock::now() + head->time_until_trigger(); - - auto timer_was_ready_at_tp = timepoint_timer_ready < timepoint; - - if (timer_was_ready_at_tp) { - // Head timer is ready, execute - head->execute_callback(); - timers_heap.heapify_root(); - weak_timers_heap_.store(timers_heap); - return true; - } else { - return false; - } -} diff --git a/rclcpp/test/rclcpp/executors/test_events_executor.cpp b/rclcpp/test/rclcpp/executors/test_events_executor.cpp index e2bec72756..949b0ee994 100644 --- a/rclcpp/test/rclcpp/executors/test_events_executor.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_executor.cpp @@ -268,6 +268,7 @@ TEST_F(TestEventsExecutor, spin_all_max_duration) } EventsExecutor executor; + EXPECT_THROW(executor.spin_all(0ms), std::invalid_argument); EXPECT_THROW(executor.spin_all(-5ms), std::invalid_argument); } diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index b6b45244fb..76e6e66b3c 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -501,6 +501,7 @@ TYPED_TEST(TestExecutorsStable, spinAll) { // executor. bool spin_exited = false; std::thread spinner([&spin_exited, &executor, this]() { + std::this_thread::sleep_for(10ms); executor.spin_all(1s); executor.remove_node(this->node, true); spin_exited = true; From feef4e2ff1ca903a80a762fc630655e04a8cc84b Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 19 Feb 2021 14:58:11 -0300 Subject: [PATCH 152/168] Use timer_was_ready_at_tp --- rclcpp/include/rclcpp/executors/timers_manager.hpp | 5 ++--- rclcpp/src/rclcpp/executors/timers_manager.cpp | 8 +++----- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 4e50b2922e..fb58576255 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -106,7 +106,8 @@ class TimersManager * @return true if head timer was ready at tp */ bool execute_head_timer( - std::chrono::time_point tp = TimePoint::max()); + std::chrono::time_point tp = + std::chrono::time_point::max()); /** * @brief Get the amount of time before the next timer expires. @@ -133,8 +134,6 @@ class TimersManager // See https://en.cppreference.com/w/cpp/thread/condition_variable/wait_for static constexpr std::chrono::nanoseconds MAX_TIME = std::chrono::hours(90); - using TimePoint = std::chrono::time_point; - private: RCLCPP_DISABLE_COPY(TimersManager) diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 7026f8bd90..1afcd09f5d 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -138,12 +138,10 @@ bool TimersManager::execute_head_timer( bool timer_ready = false; - if (timepoint != TimePoint::max()) { - // A ready timer will return a negative duration when calling time_until_trigger - auto timepoint_timer_ready = std::chrono::steady_clock::now() + head->time_until_trigger(); + auto max_time = std::chrono::time_point::max(); - // Here we check if the timer was already ready at timepoint - timer_ready = timepoint_timer_ready < timepoint; + if (timepoint != max_time) { + timer_ready = timer_was_ready_at_tp(head, timepoint); } else { timer_ready = head->is_ready(); } From f978a489b6296271ef2dde515b5ada1ba7ea0551 Mon Sep 17 00:00:00 2001 From: Alberto Soragna Date: Fri, 19 Feb 2021 18:48:46 +0000 Subject: [PATCH 153/168] remove smart pointer definitions for virtual class Signed-off-by: Alberto Soragna --- rclcpp/include/rclcpp/executors/event_waitable.hpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/event_waitable.hpp b/rclcpp/include/rclcpp/executors/event_waitable.hpp index 8f4b4d27ca..d52b664b7f 100644 --- a/rclcpp/include/rclcpp/executors/event_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/event_waitable.hpp @@ -33,8 +33,6 @@ namespace executors class EventWaitable : public rclcpp::Waitable { public: - RCLCPP_SMART_PTR_DEFINITIONS(EventWaitable) - // Constructor RCLCPP_PUBLIC EventWaitable() = default; From 191f47a3183c1263b5b8efb4ae1294c2c0b5d66c Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Sat, 20 Feb 2021 10:26:45 +0000 Subject: [PATCH 154/168] cleanup events executor class Signed-off-by: Alberto Soragna --- .../rclcpp/executors/events_executor.hpp | 19 +++++++------- .../experimental/buffers/events_queue.hpp | 23 +++++++++-------- .../buffers/simple_events_queue.hpp | 16 +++++------- .../src/rclcpp/executors/events_executor.cpp | 25 ++++++++----------- .../rclcpp/executors/test_events_queue.cpp | 18 ++----------- 5 files changed, 39 insertions(+), 62 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index d6ded52be8..49e3b24c7d 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -37,10 +37,16 @@ namespace executors /// Events executor implementation /** - * This executor is a variation of the standard one that does not uses a waitset. - * The executor uses an events queue and a timers manager to execute entities from its + * This executor uses an events queue and a timers manager to execute entities from its * associated nodes and callback groups. - * This provides improved performance as it allows to skip all the waitset maintenance operations. + * The RMW listener APIs are used to collect new events. + * + * This executor tries to reduce as much as possible the amount of maintenance operations. + * This allows to use customized `EventsQueue` classes to achieve different goals such + * as very low CPU usage, bounded memory requirement, determinism, etc. + * + * The executor uses a weak ownership model and it locks entities only while executing + * their related events. * * To run this executor: * rclcpp::executors::EventsExecutor executor; @@ -204,8 +210,6 @@ class EventsExecutor : public rclcpp::Executor private: RCLCPP_DISABLE_COPY(EventsExecutor) - using EventQueue = std::queue; - // Executor callback: Push new events into the queue and trigger cv. // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. @@ -229,11 +233,6 @@ class EventsExecutor : public rclcpp::Executor this_executor->events_queue_cv_.notify_one(); } - /// Extract and execute events from the queue until it is empty - RCLCPP_PUBLIC - void - consume_all_events(EventQueue & queue); - // Execute a single event RCLCPP_PUBLIC void diff --git a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp index 77a5efd695..625d8c7651 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp @@ -17,8 +17,8 @@ #include -#include "rclcpp/executors/events_executor_entities_collector.hpp" #include "rclcpp/macros.hpp" +#include "rclcpp/visibility_control.hpp" #include "rmw/listener_event_types.h" @@ -30,10 +30,14 @@ namespace buffers { /** - * @brief This abstract class is intended to be used as - * a wrapper around a queue. The derived classes should chose - * which container to use and the strategies for push and prune - * events from the queue. + * @brief This abstract class can be used to implement different types of queues + * where `rmw_listener_event_t` can be stored. + * The derived classes should choose which underlying container to use and + * the strategy for pushing and popping events. + * For example a queue implementation may be bounded or unbounded and have + * different pruning strategies. + * Implementations may or may not check the validity of events and decide how to handle + * the situation where an event is not valid anymore (e.g. a subscription history cache overruns) */ class EventsQueue { @@ -56,9 +60,7 @@ class EventsQueue push(const rmw_listener_event_t & event) = 0; /** - * @brief removes front element from the queue - * The element removed is the "oldest" element in the queue whose - * value can be retrieved by calling member front(). + * @brief removes front element from the queue. */ RCLCPP_PUBLIC virtual @@ -101,14 +103,13 @@ class EventsQueue init() = 0; /** - * @brief gets a queue with all events accumulated on it since - * the last call. The member queue is empty when the call returns. + * @brief pops out all events stored in the object into an output queue. * @return queue with events */ RCLCPP_PUBLIC virtual std::queue - get_all_events() = 0; + pop_all_events() = 0; }; } // namespace buffers diff --git a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp index 5bb0a85b51..278e50d7be 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp @@ -28,11 +28,10 @@ namespace buffers { /** - * @brief This class provides a simple queue implementation - * based on a std::queue. As the objective is having a CPU peformant - * queue, it does not performs any checks about the size of - * the queue, so the queue size could grow unbounded. - * It does not implement any pruning mechanisms. + * @brief This class implements an EventsQueue as a simple wrapper around a std::queue. + * It does not perform any checks about the size of queue, which can grow + * unbounded without being pruned. + * The simplicity of this implementation makes it suitable for optimizing CPU usage. */ class SimpleEventsQueue : public EventsQueue { @@ -53,9 +52,7 @@ class SimpleEventsQueue : public EventsQueue } /** - * @brief removes front element from the queue - * The element removed is the "oldest" element in the queue whose - * value can be retrieved by calling member front(). + * @brief removes front event from the queue. */ RCLCPP_PUBLIC virtual @@ -114,7 +111,6 @@ class SimpleEventsQueue : public EventsQueue std::swap(event_queue_, local_queue); } - /** * @brief gets a queue with all events accumulated on it since * the last call. The member queue is empty when the call returns. @@ -123,7 +119,7 @@ class SimpleEventsQueue : public EventsQueue RCLCPP_PUBLIC virtual std::queue - get_all_events() + pop_all_events() { std::queue local_queue; std::swap(event_queue_, local_queue); diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index d4dcdd8b90..31f8039a43 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -66,13 +66,19 @@ EventsExecutor::spin() std::unique_lock push_lock(push_mutex_); // We wait here until something has been pushed to the event queue events_queue_cv_.wait(push_lock, has_event_predicate); - // Local event queue to allow entities to push events while we execute them - EventQueue execution_event_queue = events_queue_->get_all_events(); + // Move all events into a local events queue to allow entities to push while we execute them + std::queue execution_events_queue = events_queue_->pop_all_events(); // Unlock the mutex push_lock.unlock(); - // Consume all available events, this queue will be empty at the end of the function - this->consume_all_events(execution_event_queue); + // Consume all available events + while (!execution_events_queue.empty()) { + rmw_listener_event_t event = execution_events_queue.front(); + execution_events_queue.pop(); + this->execute_event(event); + } } + + // Stop the timers manager thread when we are done spinning timers_manager_->stop(); } @@ -235,17 +241,6 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) this->remove_node(node_ptr->get_node_base_interface(), notify); } -void -EventsExecutor::consume_all_events(EventQueue & event_queue) -{ - while (!event_queue.empty()) { - rmw_listener_event_t event = event_queue.front(); - event_queue.pop(); - - this->execute_event(event); - } -} - void EventsExecutor::execute_event(const rmw_listener_event_t & event) { diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp index 4d22a5eb1e..9f540687d0 100644 --- a/rclcpp/test/rclcpp/executors/test_events_queue.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -21,21 +21,7 @@ using namespace std::chrono_literals; -class TestEventsQueue : public ::testing::Test -{ -public: - void SetUp() - { - rclcpp::init(0, nullptr); - } - - void TearDown() - { - rclcpp::shutdown(); - } -}; - -TEST_F(TestEventsQueue, SimpleQueueTest) +TEST(TestEventsQueue, SimpleQueueTest) { // Create a SimpleEventsQueue and a local queue auto simple_queue = std::make_unique(); @@ -54,7 +40,7 @@ TEST_F(TestEventsQueue, SimpleQueueTest) // Pop one message simple_queue->pop(); - local_events_queue = simple_queue->get_all_events(); + local_events_queue = simple_queue->pop_all_events(); // We should have (11 - 1) events in the local queue size_t local_queue_size = local_events_queue.size(); From 6a99496bb8e32837a46a3eea986e506a23bab52f Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Sat, 20 Feb 2021 11:41:51 +0000 Subject: [PATCH 155/168] rework events executor constructor Signed-off-by: Alberto Soragna --- rclcpp/src/rclcpp/executors/events_executor.cpp | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 31f8039a43..2737f76e94 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -30,11 +30,20 @@ EventsExecutor::EventsExecutor( const rclcpp::ExecutorOptions & options) : rclcpp::Executor(options) { + // Get ownership of the queue used to store events. + if (!events_queue) { + throw std::invalid_argument("events_queue can't be a null pointer"); + } + events_queue_ = std::move(events_queue); + events_queue_->init(); + + // Create timers manager timers_manager_ = std::make_shared(context_); + + // Create entities collector entities_collector_ = std::make_shared(this); entities_collector_->init(); - // Setup the executor notifier to wake up the executor when some guard conditions are tiggered. // The added guard conditions are guaranteed to not go out of scope before the executor itself. executor_notifier_ = std::make_shared(); @@ -42,11 +51,6 @@ EventsExecutor::EventsExecutor( executor_notifier_->add_guard_condition(&interrupt_guard_condition_); executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); entities_collector_->add_waitable(executor_notifier_); - - // Get ownership of the queue used to store events. - events_queue_ = std::move(events_queue); - // Init the events queue - events_queue_->init(); } void From c99d764cfd19f8dd5b7163148440a7c2072f8b64 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Sat, 20 Feb 2021 10:30:54 +0000 Subject: [PATCH 156/168] cleanup timers manager Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/timers_manager.hpp | 153 ++++++++++-------- .../src/rclcpp/executors/timers_manager.cpp | 21 +-- 2 files changed, 94 insertions(+), 80 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index fb58576255..8be1a97172 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -40,11 +40,11 @@ namespace executors * * Timers management * This class provides APIs to add and remove timers. - * It keeps a list of weak pointers from added timers, and owns them only when - * have expired and need to be executed. + * It keeps a list of weak pointers from added timers, and locks them only when + * they need to be executed or modified. * Timers are kept ordered in a binary-heap priority queue. * Calls to add/remove APIs will temporarily block the execution of the timers and - * will require to reorder the internal priority queue of timers. + * will require to reorder the internal priority queue. * Because of this, they have a not-negligible impact on the performance. * * Timers execution @@ -52,10 +52,10 @@ namespace executors * to spawn a thread where timers are monitored and periodically executed. * Besides this, other APIs allow to either execute a single timer or all the * currently ready ones. - * This class assumes that the execute_callback API of the stored timer is never - * called by other entities, but can only be called from here. - * If this assumption is not respected, the heap property will be invalidated, - * so timers may be executed out of order. + * This class assumes that the `execute_callback()` API of the stored timers is never + * called by other entities, but it can only be called from here. + * If this assumption is not respected, the heap property may be invalidated, + * so timers may be executed out of order, without this object noticing it. * */ class TimersManager @@ -65,45 +65,65 @@ class TimersManager /** * @brief Construct a new TimersManager object + * + * @param context custom context to be used. + * Shared ownership of the context is held until destruction. */ explicit TimersManager(std::shared_ptr context); /** - * @brief Destruct the object making sure to stop thread and release memory. + * @brief Destruct the TimersManager object making sure to stop thread and release memory. */ ~TimersManager(); /** - * @brief Adds a new TimerBase::WeakPtr to the storage. - * This object will store a weak pointer of the added timer - * in a heap data structure. - * @param timer the timer to be added + * @brief Adds a new timer to the storage, maintaining weak ownership of it. + * Function is thread safe and it can be called regardless of the state of the timers thread. + * + * @param timer the timer to add. */ void add_timer(rclcpp::TimerBase::SharedPtr timer); /** - * @brief Starts a thread that takes care of executing timers added to this object. + * @brief Remove a single timer from the object storage. + * Will do nothing if the timer was not being stored here. + * Function is thread safe and it can be called regardless of the state of the timers thread. + * + * @param timer the timer to remove. + */ + void remove_timer(rclcpp::TimerBase::SharedPtr timer); + + /** + * @brief Remove all the timers stored in the object. + * Function is thread safe and it can be called regardless of the state of the timers thread. + */ + void clear(); + + /** + * @brief Starts a thread that takes care of executing the timers stored in this object. + * Function will throw an error if the timers thread was already running. */ void start(); /** * @brief Stops the timers thread. + * Will do nothing if the timer thread was not running. */ void stop(); /** - * @brief Executes all the timers currently ready when the function is invoked - * while keeping the heap correctly sorted. - * @return std::chrono::nanoseconds for next timer to expire, - * the returned value could be negative if the timer is already expired - * or MAX_TIME if the heap is empty. + * @brief Executes all the timers currently ready when the function was invoked. + * This function will lock all the stored timers throughout its duration. + * Function is thread safe, but it will throw an error if the timers thread is running. */ - std::chrono::nanoseconds execute_ready_timers(); + void execute_ready_timers(); /** * @brief Executes head timer if ready at time point. - * @param tp the timepoint to check for - * @return true if head timer was ready at tp + * Function is thread safe, but it will throw an error if the timers thread is running. + * + * @param tp the time point to check for, where `max()` denotes that no check will be performed. + * @return true if head timer was ready at time point. */ bool execute_head_timer( std::chrono::time_point tp = @@ -111,24 +131,14 @@ class TimersManager /** * @brief Get the amount of time before the next timer expires. - * + * Function is thread safe, but it will throw an error if the timers thread is running. + * * @return std::chrono::nanoseconds to wait, * the returned value could be negative if the timer is already expired - * or MAX_TIME if the heap is empty. + * or MAX_TIME if there are no timers stored in the object. */ std::chrono::nanoseconds get_head_timeout(); - /** - * @brief Remove all the timers stored in the object. - */ - void clear(); - - /** - * @brief Remove a single timer stored in the object, passed as a shared_ptr. - * @param timer the timer to remove. - */ - void remove_timer(rclcpp::TimerBase::SharedPtr timer); - // This is what the TimersManager uses to denote a duration forever. // We don't use std::chrono::nanoseconds::max because it will overflow. // See https://en.cppreference.com/w/cpp/thread/condition_variable/wait_for @@ -144,20 +154,22 @@ class TimersManager class TimersHeap; /** - * @brief This class allows to store weak pointers to timers in a heap data structure. + * @brief This class allows to store weak pointers to timers in a heap-like data structure. + * The root of the heap is the timer that expires first. + * Since this class uses weak ownership, it is not guaranteed that it represents a valid heap + * at any point in time as timers could go out of scope, thus invalidating it. * The "validate_and_lock" API allows to get ownership of the timers and also makes sure that * the heap property is respected. - * The root of the heap is the timer that expires first. * This class is not thread safe and requires external mutexes to protect its usage. */ class WeakTimersHeap { public: /** - * @brief Try to add a new timer to the heap. - * After the addition, the heap property is preserved. - * @param timer new timer to add - * @return true if timer has been added, false if it was already there + * @brief Add a new timer to the heap. After the addition, the heap property is enforced. + * + * @param timer new timer to add. + * @return true if timer has been added, false if it was already there. */ bool add_timer(TimerPtr timer) { @@ -173,10 +185,10 @@ class TimersManager } /** - * @brief Try to remove a timer from the heap. - * After the removal, the heap property is preserved. - * @param timer timer to remove - * @return true if timer has been removed, false if it was not there + * @brief Remove a timer from the heap. After the removal, the heap property is enforced. + * + * @param timer timer to remove. + * @return true if timer has been removed, false if it was not there. */ bool remove_timer(TimerPtr timer) { @@ -193,7 +205,9 @@ class TimersManager /** * @brief This function restores the current object as a valid heap - * and it also returns a locked version of it + * and it returns a locked version of it. + * It is the only public API to access the stored timers. + * * @return TimersHeap owned timers corresponding to the current object */ TimersHeap validate_and_lock() @@ -205,8 +219,9 @@ class TimersManager while (it != weak_heap_.end()) { if (auto timer_shared_ptr = it->lock()) { - // This timer is valid, add it to the vector - locked_heap.push_back(std::move(timer_shared_ptr)); + // This timer is valid, add it to the locked heap + // Note: we access private `owned_heap_` member field. + locked_heap.owned_heap_.push_back(std::move(timer_shared_ptr)); it++; } else { // This timer went out of scope, remove it @@ -229,11 +244,15 @@ class TimersManager /** * @brief This function allows to recreate the heap of weak pointers * from an heap of owned pointers. + * It is required to be called after a locked TimersHeap generated from this object + * has been modified in any way (e.g. timers triggered, added, removed). + * * @param heap timers heap to store as weak pointers */ void store(const TimersHeap & heap) { weak_heap_.clear(); + // Note: we access private `owned_heap_` member field. for (auto t : heap.owned_heap_) { weak_heap_.push_back(t); } @@ -254,7 +273,8 @@ class TimersManager /** * @brief This class is the equivalent of WeakTimersHeap but with ownership of the timers. * It can be generated by locking the weak version. - * It provides operations to manipulate the heap + * It provides operations to manipulate the heap. + * This class is not thread safe and requires external mutexes to protect its usage. */ class TimersHeap { @@ -324,13 +344,15 @@ class TimersManager } /** - * @brief Restore a valid heap after the root value has been replaced. + * @brief Restore a valid heap after the root value has been replaced (e.g. timer triggered). */ void heapify_root() { // The following code is a more efficient version of doing - // - pop_heap; pop_back; - // - push_back; push_heap; + // pop_heap(); + // pop_back(); + // push_back(); + // push_heap(); // as it removes the need for the last push_heap // Push the modified element (i.e. the current root) at the bottom of the heap @@ -350,26 +372,21 @@ class TimersManager } /** - * @brief Convenience function that allows to push an element at the bottom of the heap. - * It will not perform any check on whether the heap remains valid or not. - * Those checks are responsibility of the calling code. - * - * @param timer timer to push at the back of the data structure representing the heap + * @brief Friend declaration to allow the `validate_and_lock()` function to access the + * underlying heap container */ - void push_back(TimerPtr timer) - { - owned_heap_.push_back(timer); - } + friend TimersHeap WeakTimersHeap::validate_and_lock(); /** - * @brief Friend declaration to allow the store function to access the underlying - * heap container + * @brief Friend declaration to allow the `store()` function to access the + * underlying heap container */ friend void WeakTimersHeap::store(const TimersHeap & heap); private: /** * @brief Comparison function between timers. + * Returns true if `a` expires after `b`. */ static bool timer_greater(TimerPtr a, TimerPtr b) { @@ -392,6 +409,7 @@ class TimersManager * @return std::chrono::nanoseconds to wait, * the returned value could be negative if the timer is already expired * or MAX_TIME if the heap is empty. + * This function is not thread safe, acquire the timers_mutex_ before calling it. */ std::chrono::nanoseconds get_head_timeout_unsafe(const TimersHeap & heap) { @@ -404,15 +422,16 @@ class TimersManager /** * @brief Executes all the timers currently ready when the function is invoked * while keeping the heap correctly sorted. - * This function is not thread safe, acquire a mutex before calling it. + * This function is not thread safe, acquire the timers_mutex_ before calling it. */ - void execute_ready_timers_unsafe(TimersHeap & heap); + void execute_ready_timers_unsafe(); /** * @brief Helper function that checks whether a timer was already ready - * at a specific timepoint + * at a specific time point. + * @param timer a pointer to the timer to check for - * @param tp the timepoint to check for + * @param tp the time point to check for * @return true if timer was ready at tp */ bool timer_was_ready_at_tp( @@ -424,7 +443,7 @@ class TimersManager return time_ready < tp; } - // Thread used to run the timers monitoring and execution task + // Thread used to run the timers execution task std::thread timers_thread_; // Protects access to timers std::mutex timers_mutex_; diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index 1afcd09f5d..f7911153e4 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -68,7 +68,7 @@ void TimersManager::start() void TimersManager::stop() { // Nothing to do if the timers thread is not running - // or if another thred already signaled to stop. + // or if another thread already signaled to stop. if (!running_.exchange(false)) { return; } @@ -99,7 +99,7 @@ std::chrono::nanoseconds TimersManager::get_head_timeout() return this->get_head_timeout_unsafe(timers_heap); } -std::chrono::nanoseconds TimersManager::execute_ready_timers() +void TimersManager::execute_ready_timers() { // Do not allow to interfere with the thread running if (running_) { @@ -112,12 +112,10 @@ std::chrono::nanoseconds TimersManager::execute_ready_timers() TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); this->execute_ready_timers_unsafe(timers_heap); weak_timers_heap_.store(timers_heap); - - return this->get_head_timeout_unsafe(timers_heap); } bool TimersManager::execute_head_timer( - std::chrono::time_point timepoint) + std::chrono::time_point tp) { // Do not allow to interfere with the thread running if (running_) { @@ -137,11 +135,8 @@ bool TimersManager::execute_head_timer( TimerPtr head = timers_heap.front(); bool timer_ready = false; - - auto max_time = std::chrono::time_point::max(); - - if (timepoint != max_time) { - timer_ready = timer_was_ready_at_tp(head, timepoint); + if (tp != std::chrono::time_point::max()) { + timer_ready = timer_was_ready_at_tp(head, tp); } else { timer_ready = head->is_ready(); } @@ -152,10 +147,10 @@ bool TimersManager::execute_head_timer( timers_heap.heapify_root(); weak_timers_heap_.store(timers_heap); return true; - } else { - // Head timer was not ready yet - return false; } + + // Head timer was not ready yet + return false; } void TimersManager::execute_ready_timers_unsafe(TimersHeap & heap) From b72d124af12182070e9b4907cab660854b2cddf0 Mon Sep 17 00:00:00 2001 From: "Soragna, Alberto" Date: Sat, 20 Feb 2021 10:32:44 +0000 Subject: [PATCH 157/168] new implementation of get_head_timeout_unsafe Signed-off-by: Soragna, Alberto --- .../rclcpp/executors/timers_manager.hpp | 36 ++++++---- .../src/rclcpp/executors/timers_manager.cpp | 68 ++++++++++++------- 2 files changed, 66 insertions(+), 38 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 8be1a97172..2630fdaf2d 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -65,7 +65,7 @@ class TimersManager /** * @brief Construct a new TimersManager object - * + * * @param context custom context to be used. * Shared ownership of the context is held until destruction. */ @@ -79,7 +79,7 @@ class TimersManager /** * @brief Adds a new timer to the storage, maintaining weak ownership of it. * Function is thread safe and it can be called regardless of the state of the timers thread. - * + * * @param timer the timer to add. */ void add_timer(rclcpp::TimerBase::SharedPtr timer); @@ -88,7 +88,7 @@ class TimersManager * @brief Remove a single timer from the object storage. * Will do nothing if the timer was not being stored here. * Function is thread safe and it can be called regardless of the state of the timers thread. - * + * * @param timer the timer to remove. */ void remove_timer(rclcpp::TimerBase::SharedPtr timer); @@ -121,7 +121,7 @@ class TimersManager /** * @brief Executes head timer if ready at time point. * Function is thread safe, but it will throw an error if the timers thread is running. - * + * * @param tp the time point to check for, where `max()` denotes that no check will be performed. * @return true if head timer was ready at time point. */ @@ -132,7 +132,7 @@ class TimersManager /** * @brief Get the amount of time before the next timer expires. * Function is thread safe, but it will throw an error if the timers thread is running. - * + * * @return std::chrono::nanoseconds to wait, * the returned value could be negative if the timer is already expired * or MAX_TIME if there are no timers stored in the object. @@ -203,10 +203,26 @@ class TimersManager return removed; } + /** + * @brief Returns a const reference to the front element + */ + const WeakTimerPtr & front() const + { + return weak_heap_.front(); + } + + /** + * @brief Returns whether the heap is empty or not + */ + bool empty() const + { + return weak_heap_.empty(); + } + /** * @brief This function restores the current object as a valid heap * and it returns a locked version of it. - * It is the only public API to access the stored timers. + * It is the only public API to access and manipulate the stored timers. * * @return TimersHeap owned timers corresponding to the current object */ @@ -411,13 +427,7 @@ class TimersManager * or MAX_TIME if the heap is empty. * This function is not thread safe, acquire the timers_mutex_ before calling it. */ - std::chrono::nanoseconds get_head_timeout_unsafe(const TimersHeap & heap) - { - if (heap.empty()) { - return MAX_TIME; - } - return (heap.front())->time_until_trigger(); - } + std::chrono::nanoseconds get_head_timeout_unsafe(); /** * @brief Executes all the timers currently ready when the function is invoked diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index f7911153e4..be6309886c 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -95,8 +95,7 @@ std::chrono::nanoseconds TimersManager::get_head_timeout() } std::unique_lock lock(timers_mutex_); - TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); - return this->get_head_timeout_unsafe(timers_heap); + return this->get_head_timeout_unsafe(); } void TimersManager::execute_ready_timers() @@ -108,10 +107,7 @@ void TimersManager::execute_ready_timers() } std::unique_lock lock(timers_mutex_); - - TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); - this->execute_ready_timers_unsafe(timers_heap); - weak_timers_heap_.store(timers_heap); + this->execute_ready_timers_unsafe(); } bool TimersManager::execute_head_timer( @@ -153,10 +149,40 @@ bool TimersManager::execute_head_timer( return false; } -void TimersManager::execute_ready_timers_unsafe(TimersHeap & heap) +std::chrono::nanoseconds TimersManager::get_head_timeout_unsafe() { + // If we don't have any weak pointer, then we just return maximum timeout + if (weak_timers_heap_.empty()) { + return MAX_TIME; + } + + // Weak heap is not empty, so try to lock the first element + TimerPtr head_timer = weak_timers_heap_.front().lock(); + // If it is still a valid pointer, it is guaranteed to be the correct head + if (head_timer != nullptr) { + return head_timer->time_until_trigger(); + } + + // If the first elements has expired, we can't make other assumptions on the heap + // and we need to entirely validate it. + TimersHeap locked_heap = weak_timers_heap_.validate_and_lock(); + + // NOTE: the following operations will not modify any element in the heap, so we + // don't have to call `weak_timers_heap_.store(locked_heap)` at the end. + + if (locked_heap.empty()) { + return MAX_TIME; + } + return locked_heap.front()->time_until_trigger(); +} + +void TimersManager::execute_ready_timers_unsafe() +{ + // We start by locking the timers + TimersHeap locked_heap = weak_timers_heap_.validate_and_lock(); + // Nothing to do if we don't have any timer - if (heap.empty()) { + if (locked_heap.empty()) { return; } @@ -164,44 +190,36 @@ void TimersManager::execute_ready_timers_unsafe(TimersHeap & heap) // The second check prevents this function from blocking indefinitely if the // time required for executing the timers is longer than their period. - TimerPtr head = heap.front(); + TimerPtr head = locked_heap.front(); auto start_time = std::chrono::steady_clock::now(); while (head->is_ready() && this->timer_was_ready_at_tp(head, start_time)) { // Execute head timer head->execute_callback(); // Executing a timer will result in updating its time_until_trigger, so re-heapify - heap.heapify_root(); + locked_heap.heapify_root(); // Get new head timer - head = heap.front(); + head = locked_heap.front(); } + + // After having performed work on the locked heap we reflect the changes to weak one. + // Timers will be already sorted the next time we need them if none went out of scope. + weak_timers_heap_.store(locked_heap); } void TimersManager::run_timers() { - std::chrono::nanoseconds time_to_sleep; - { - std::unique_lock lock(timers_mutex_); - TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); - time_to_sleep = this->get_head_timeout_unsafe(timers_heap); - } - while (rclcpp::ok(context_) && running_) { // Lock mutex std::unique_lock timers_lock(timers_mutex_); + std::chrono::nanoseconds time_to_sleep = get_head_timeout_unsafe(); // Wait until timeout or notification that timers have been updated timers_cv_.wait_for(timers_lock, time_to_sleep, [this]() {return timers_updated_;}); // Reset timers updated flag timers_updated_ = false; - // Get ownership of timers - TimersHeap timers_heap = weak_timers_heap_.validate_and_lock(); // Execute timers - this->execute_ready_timers_unsafe(timers_heap); - // Store updated order of elements to efficiently re-use it next iteration - weak_timers_heap_.store(timers_heap); - // Get next timeout - time_to_sleep = this->get_head_timeout_unsafe(timers_heap); + this->execute_ready_timers_unsafe(); } // Make sure the running flag is set to false when we exit from this function From 80b6ccc69dc09de73fe731f9a94e7a12882cff91 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 26 Feb 2021 15:42:21 -0300 Subject: [PATCH 158/168] Rework executor callback data --- rclcpp/include/rclcpp/client.hpp | 4 +- .../rclcpp/executors/events_executor.hpp | 17 ++-- .../events_executor_entities_collector.hpp | 9 ++ .../executors/events_executor_event_types.hpp | 81 ++++++++++++++++ .../events_executor_notify_waitable.hpp | 7 +- .../experimental/buffers/events_queue.hpp | 10 +- .../buffers/simple_events_queue.hpp | 12 +-- .../subscription_intra_process_base.hpp | 4 +- rclcpp/include/rclcpp/qos_event.hpp | 4 +- rclcpp/include/rclcpp/service.hpp | 4 +- rclcpp/include/rclcpp/subscription_base.hpp | 4 +- rclcpp/include/rclcpp/waitable.hpp | 4 +- rclcpp/src/rclcpp/client.cpp | 7 +- .../src/rclcpp/executors/events_executor.cpp | 20 ++-- .../events_executor_entities_collector.cpp | 93 +++++++++++++++---- rclcpp/src/rclcpp/qos_event.cpp | 7 +- rclcpp/src/rclcpp/service.cpp | 7 +- rclcpp/src/rclcpp/subscription_base.cpp | 7 +- .../subscription_intra_process_base.cpp | 7 +- rclcpp/src/rclcpp/waitable.cpp | 6 +- rclcpp/test/rclcpp/CMakeLists.txt | 2 +- .../rclcpp/executors/test_events_queue.cpp | 11 ++- .../test/rclcpp/executors/test_executors.cpp | 7 +- 23 files changed, 240 insertions(+), 94 deletions(-) create mode 100644 rclcpp/include/rclcpp/executors/events_executor_event_types.hpp diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 47b657fe5c..1a66d128e6 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -159,8 +159,8 @@ class ClientBase RCLCPP_PUBLIC void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const; + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const; protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 49e3b24c7d..a5ceffd81a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -22,6 +22,7 @@ #include "rclcpp/executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" +#include "rclcpp/executors/events_executor_event_types.hpp" #include "rclcpp/executors/events_executor_notify_waitable.hpp" #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/experimental/buffers/events_queue.hpp" @@ -214,20 +215,20 @@ class EventsExecutor : public rclcpp::Executor // This function is called by the DDS entities when an event happened, // like a subscription receiving a message. static void - push_event(void * executor_ptr, rmw_listener_event_t event) + push_event(const void * event_data) { - // Check if the executor pointer is not valid - if (!executor_ptr) { - throw std::runtime_error("The executor pointer is not valid."); + if (!event_data) { + throw std::runtime_error("Executor event data not valid."); } - auto this_executor = static_cast(executor_ptr); + auto data = static_cast(event_data); + + executors::EventsExecutor * this_executor = data->executor; // Event queue mutex scope { std::unique_lock lock(this_executor->push_mutex_); - - this_executor->events_queue_->push(event); + this_executor->events_queue_->push({data->entity_id, data->event_type}); } // Notify that the event queue has some events in it. this_executor->events_queue_cv_.notify_one(); @@ -236,7 +237,7 @@ class EventsExecutor : public rclcpp::Executor // Execute a single event RCLCPP_PUBLIC void - execute_event(const rmw_listener_event_t & event); + execute_event(const ExecutorEvent & event); // Queue where entities can push events rclcpp::experimental::buffers::EventsQueue::SharedPtr events_queue_; diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 7a07d61375..75716d236b 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -22,6 +22,7 @@ #include #include "rclcpp/executors/event_waitable.hpp" +#include "rclcpp/executors/events_executor_event_types.hpp" #include "rclcpp/executors/timers_manager.hpp" #include "rclcpp/node_interfaces/node_base_interface.hpp" @@ -213,6 +214,12 @@ class EventsExecutorEntitiesCollector final void unset_guard_condition_callback(const rcl_guard_condition_t * guard_condition); + void + remove_callback_data(void * entity_id, ExecutorEventType type); + + const EventsExecutorCallbackData * + get_callback_data(void * entity_id, ExecutorEventType type); + /// Return true if the node belongs to the collector /** * \param[in] group_ptr a node base interface shared pointer @@ -262,6 +269,8 @@ class EventsExecutorEntitiesCollector final EventsExecutor * associated_executor_ = nullptr; /// Instance of the timers manager used by the associated executor TimersManager::SharedPtr timers_manager_; + /// Callback data from entities mapped to a counter of each + std::unordered_map callback_data_map_; }; } // namespace executors diff --git a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp new file mode 100644 index 0000000000..d1b6e150d1 --- /dev/null +++ b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp @@ -0,0 +1,81 @@ +// Copyright 2021 Open Source Robotics Foundation, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef RCLCPP__EXECUTORS__EVENTS_EXECUTOR_EVENT_TYPES_HPP_ +#define RCLCPP__EXECUTORS__EVENTS_EXECUTOR_EVENT_TYPES_HPP_ + +namespace rclcpp +{ +namespace executors +{ + +// forward declaration of EventsExecutor to avoid circular dependency +class EventsExecutor; + +enum ExecutorEventType +{ + SUBSCRIPTION_EVENT, + SERVICE_EVENT, + CLIENT_EVENT, + WAITABLE_EVENT +}; + +struct ExecutorEvent +{ + const void * entity_id; + ExecutorEventType type; +}; + +struct EventsExecutorCallbackData +{ + EventsExecutorCallbackData( + EventsExecutor * exec, + void * id, + ExecutorEventType type) + { + executor = exec; + entity_id = id; + event_type = type; + } + + // Equal operator + bool operator==(const EventsExecutorCallbackData & other) const + { + return (executor == other.executor) && + (entity_id == other.entity_id) && + (event_type == other.event_type); + } + + // Struct members + EventsExecutor * executor; + void * entity_id; + ExecutorEventType event_type; +}; + +// To be able to use std::unordered_map with an EventsExecutorCallbackData +// as key, we need a hasher: +struct KeyHasher +{ + size_t operator()(const EventsExecutorCallbackData & k) const + { + return ((std::hash()(k.executor) ^ + (std::hash()(k.entity_id) << 1)) >> 1) ^ + (std::hash()(k.event_type) << 1); + } +}; + +} // namespace executors +} // namespace rclcpp + +#endif // RCLCPP__EXECUTORS__EVENTS_EXECUTOR_EVENT_TYPES_HPP_ diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index b9987d4bd4..5c00a8ed7a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -61,15 +61,14 @@ class EventsExecutorNotifyWaitable final : public EventWaitable RCLCPP_PUBLIC void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const override + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const override { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( gc, executor_callback, - executor, - this, + executor_callback_data, false); if (RCL_RET_OK != ret) { diff --git a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp index 625d8c7651..92d6e5e6c8 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp @@ -20,7 +20,7 @@ #include "rclcpp/macros.hpp" #include "rclcpp/visibility_control.hpp" -#include "rmw/listener_event_types.h" +#include "rclcpp/executors/events_executor_event_types.hpp" namespace rclcpp { @@ -31,7 +31,7 @@ namespace buffers /** * @brief This abstract class can be used to implement different types of queues - * where `rmw_listener_event_t` can be stored. + * where `ExecutorEvent` can be stored. * The derived classes should choose which underlying container to use and * the strategy for pushing and popping events. * For example a queue implementation may be bounded or unbounded and have @@ -57,7 +57,7 @@ class EventsQueue RCLCPP_PUBLIC virtual void - push(const rmw_listener_event_t & event) = 0; + push(const rclcpp::executors::ExecutorEvent & event) = 0; /** * @brief removes front element from the queue. @@ -73,7 +73,7 @@ class EventsQueue */ RCLCPP_PUBLIC virtual - rmw_listener_event_t + rclcpp::executors::ExecutorEvent front() const = 0; /** @@ -108,7 +108,7 @@ class EventsQueue */ RCLCPP_PUBLIC virtual - std::queue + std::queue pop_all_events() = 0; }; diff --git a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp index 278e50d7be..4f560a91cf 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/simple_events_queue.hpp @@ -46,7 +46,7 @@ class SimpleEventsQueue : public EventsQueue RCLCPP_PUBLIC virtual void - push(const rmw_listener_event_t & event) + push(const rclcpp::executors::ExecutorEvent & event) { event_queue_.push(event); } @@ -68,7 +68,7 @@ class SimpleEventsQueue : public EventsQueue */ RCLCPP_PUBLIC virtual - rmw_listener_event_t + rclcpp::executors::ExecutorEvent front() const { return event_queue_.front(); @@ -107,7 +107,7 @@ class SimpleEventsQueue : public EventsQueue init() { // Make sure the queue is empty when we start - std::queue local_queue; + std::queue local_queue; std::swap(event_queue_, local_queue); } @@ -118,16 +118,16 @@ class SimpleEventsQueue : public EventsQueue */ RCLCPP_PUBLIC virtual - std::queue + std::queue pop_all_events() { - std::queue local_queue; + std::queue local_queue; std::swap(event_queue_, local_queue); return local_queue; } private: - std::queue event_queue_; + std::queue event_queue_; }; } // namespace buffers diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 645d6b65cc..9227e21623 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -77,8 +77,8 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const override; + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 9a3716528e..4d886a99ff 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -111,8 +111,8 @@ class QOSEventHandlerBase : public Waitable RCLCPP_PUBLIC void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const override; + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const override; protected: rcl_event_t event_handle_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 00d4227e62..93a7669dbd 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -129,8 +129,8 @@ class ServiceBase RCLCPP_PUBLIC void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const; + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const; protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index fa7789ceb6..9ae9c30a1b 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -272,8 +272,8 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const; + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const; protected: template diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 449fcb5b46..4c89af80ac 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -212,8 +212,8 @@ class Waitable virtual void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const; + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const; private: std::atomic in_use_by_wait_set_{false}; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index dd9e7d4e7e..fc80044525 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -201,14 +201,13 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const { rcl_ret_t ret = rcl_client_set_listener_callback( client_handle_.get(), executor_callback, - executor, - this); + executor_callback_data); if (RCL_RET_OK != ret) { throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index 2737f76e94..eb53c2e3c7 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -49,7 +49,7 @@ EventsExecutor::EventsExecutor( executor_notifier_ = std::make_shared(); executor_notifier_->add_guard_condition(&shutdown_guard_condition_->get_rcl_guard_condition()); executor_notifier_->add_guard_condition(&interrupt_guard_condition_); - executor_notifier_->set_events_executor_callback(this, &EventsExecutor::push_event); + entities_collector_->add_waitable(executor_notifier_); } @@ -71,12 +71,12 @@ EventsExecutor::spin() // We wait here until something has been pushed to the event queue events_queue_cv_.wait(push_lock, has_event_predicate); // Move all events into a local events queue to allow entities to push while we execute them - std::queue execution_events_queue = events_queue_->pop_all_events(); + std::queue execution_events_queue = events_queue_->pop_all_events(); // Unlock the mutex push_lock.unlock(); // Consume all available events while (!execution_events_queue.empty()) { - rmw_listener_event_t event = execution_events_queue.front(); + ExecutorEvent event = execution_events_queue.front(); execution_events_queue.pop(); this->execute_event(event); } @@ -139,7 +139,7 @@ EventsExecutor::spin_some_impl(std::chrono::nanoseconds max_duration, bool exhau bool has_event = !events_queue_->empty(); if (has_event) { - rmw_listener_event_t event = events_queue_->front(); + ExecutorEvent event = events_queue_->front(); events_queue_->pop(); this->execute_event(event); executed_events++; @@ -183,7 +183,7 @@ EventsExecutor::spin_once_impl(std::chrono::nanoseconds timeout) // When condition variable is notified, check this predicate to proceed auto has_event_predicate = [this]() {return !events_queue_->empty();}; - rmw_listener_event_t event; + ExecutorEvent event; bool has_event = false; { @@ -246,12 +246,12 @@ EventsExecutor::remove_node(std::shared_ptr node_ptr, bool notify) } void -EventsExecutor::execute_event(const rmw_listener_event_t & event) +EventsExecutor::execute_event(const ExecutorEvent & event) { switch (event.type) { case SUBSCRIPTION_EVENT: { - auto subscription = entities_collector_->get_subscription(event.entity); + auto subscription = entities_collector_->get_subscription(event.entity_id); if (subscription) { execute_subscription(subscription); @@ -261,7 +261,7 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) case SERVICE_EVENT: { - auto service = entities_collector_->get_service(event.entity); + auto service = entities_collector_->get_service(event.entity_id); if (service) { execute_service(service); @@ -271,7 +271,7 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) case CLIENT_EVENT: { - auto client = entities_collector_->get_client(event.entity); + auto client = entities_collector_->get_client(event.entity_id); if (client) { execute_client(client); @@ -281,7 +281,7 @@ EventsExecutor::execute_event(const rmw_listener_event_t & event) case WAITABLE_EVENT: { - auto waitable = entities_collector_->get_waitable(event.entity); + auto waitable = entities_collector_->get_waitable(event.entity_id); if (waitable) { auto data = waitable->take_data(); diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index c2b08a9de6..c6bf4df55d 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -20,6 +20,7 @@ #include "rclcpp/executors/events_executor.hpp" #include "rclcpp/executors/events_executor_entities_collector.hpp" +using rclcpp::executors::EventsExecutorCallbackData; using rclcpp::executors::EventsExecutorEntitiesCollector; EventsExecutorEntitiesCollector::EventsExecutorEntitiesCollector( @@ -75,6 +76,7 @@ EventsExecutorEntitiesCollector::~EventsExecutorEntitiesCollector() weak_nodes_.clear(); weak_clients_map_.clear(); weak_services_map_.clear(); + callback_data_map_.clear(); weak_waitables_map_.clear(); weak_subscriptions_map_.clear(); weak_nodes_to_guard_conditions_.clear(); @@ -86,7 +88,7 @@ void EventsExecutorEntitiesCollector::init() { // Add the EventsExecutorEntitiesCollector shared_ptr to waitables map - add_waitable(this->shared_from_this()); + weak_waitables_map_.emplace(this, this->shared_from_this()); } void @@ -234,40 +236,44 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); weak_subscriptions_map_.emplace(subscription.get(), subscription); + + subscription->set_events_executor_callback( + &EventsExecutor::push_event, + get_callback_data(subscription.get(), SUBSCRIPTION_EVENT)); } return false; }); group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); weak_services_map_.emplace(service.get(), service); + + service->set_events_executor_callback( + &EventsExecutor::push_event, + get_callback_data(service.get(), SERVICE_EVENT)); } return false; }); group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); weak_clients_map_.emplace(client.get(), client); + + client->set_events_executor_callback( + &EventsExecutor::push_event, + get_callback_data(client.get(), CLIENT_EVENT)); } return false; }); group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_events_executor_callback( - associated_executor_, - &EventsExecutor::push_event); weak_waitables_map_.emplace(waitable.get(), waitable); + + waitable->set_events_executor_callback( + &EventsExecutor::push_event, + get_callback_data(waitable.get(), WAITABLE_EVENT)); } return false; }); @@ -292,6 +298,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( if (subscription) { subscription->set_events_executor_callback(nullptr, nullptr); weak_subscriptions_map_.erase(subscription.get()); + remove_callback_data(subscription.get(), SUBSCRIPTION_EVENT); } return false; }); @@ -300,6 +307,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( if (service) { service->set_events_executor_callback(nullptr, nullptr); weak_services_map_.erase(service.get()); + remove_callback_data(service.get(), SERVICE_EVENT); } return false; }); @@ -308,6 +316,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( if (client) { client->set_events_executor_callback(nullptr, nullptr); weak_clients_map_.erase(client.get()); + remove_callback_data(client.get(), CLIENT_EVENT); } return false; }); @@ -316,6 +325,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( if (waitable) { waitable->set_events_executor_callback(nullptr, nullptr); weak_waitables_map_.erase(waitable.get()); + remove_callback_data(waitable.get(), WAITABLE_EVENT); } return false; }); @@ -480,8 +490,7 @@ EventsExecutorEntitiesCollector::set_guard_condition_callback( rcl_ret_t ret = rcl_guard_condition_set_listener_callback( guard_condition, &EventsExecutor::push_event, - associated_executor_, - this, + get_callback_data(this, WAITABLE_EVENT), false /* Discard previous events */); if (ret != RCL_RET_OK) { @@ -497,12 +506,13 @@ EventsExecutorEntitiesCollector::unset_guard_condition_callback( guard_condition, nullptr, nullptr, - nullptr, false /* Discard previous events */); if (ret != RCL_RET_OK) { throw std::runtime_error("Couldn't unset guard condition event callback"); } + + remove_callback_data(this, WAITABLE_EVENT); } rclcpp::SubscriptionBase::SharedPtr @@ -585,4 +595,55 @@ void EventsExecutorEntitiesCollector::add_waitable(rclcpp::Waitable::SharedPtr waitable) { weak_waitables_map_.emplace(waitable.get(), waitable); + + waitable->set_events_executor_callback( + &EventsExecutor::push_event, + get_callback_data(waitable.get(), WAITABLE_EVENT)); +} + +const EventsExecutorCallbackData * +EventsExecutorEntitiesCollector::get_callback_data( + void * entity_id, ExecutorEventType event_type) +{ + // Create an entity callback data object and check if + // we already have stored one like it + EventsExecutorCallbackData data(associated_executor_, entity_id, event_type); + + auto it = callback_data_map_.find(data); + + if (it != callback_data_map_.end()) { + // We found a callback data matching entity ID and type. + // Increment callback data counter and return pointer to data + it->second++; + return &it->first; + } + + // There was no callback data object matching ID and type, + // create one and set counter to 1. + callback_data_map_.emplace(data, 1); + + // Return a pointer to the just added entity callback data. + it = callback_data_map_.find(data); + return &it->first; +} + +void +EventsExecutorEntitiesCollector::remove_callback_data( + void * entity_id, ExecutorEventType event_type) +{ + // Create an entity callback data object and check if + // we already have stored one like it + EventsExecutorCallbackData data(associated_executor_, entity_id, event_type); + + auto it = callback_data_map_.find(data); + + if (it != callback_data_map_.end()) { + // We found a callback data matching entity ID and type. + // If we have more than 1 decrement counter, otherwise remove it. + if (it->second > 1) { + it->second--; + } else { + callback_data_map_.erase(it); + } + } } diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index d9e27dc32f..055cab306e 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -70,14 +70,13 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) void QOSEventHandlerBase::set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const { rcl_ret_t ret = rcl_event_set_listener_callback( &event_handle_, executor_callback, - executor, - this, + executor_callback_data, false /* Discard previous events */); if (RCL_RET_OK != ret) { diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index e0afdbcad9..0bf08e0dbc 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -87,14 +87,13 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const { rcl_ret_t ret = rcl_service_set_listener_callback( service_handle_.get(), executor_callback, - executor, - this); + executor_callback_data); if (RCL_RET_OK != ret) { throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 5f37848726..ef1ad4e10b 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -291,14 +291,13 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const { rcl_ret_t ret = rcl_subscription_set_listener_callback( subscription_handle_.get(), executor_callback, - executor, - this); + executor_callback_data); if (RCL_RET_OK != ret) { throw std::runtime_error("Couldn't set the EventsExecutor's callback to subscription"); diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 1c906e95cd..83c51d3bbb 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -39,14 +39,13 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, executor_callback, - executor, - this, + executor_callback_data, true /*Use previous events*/); if (RCL_RET_OK != ret) { diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 4c5bd672ce..61c2cf3f18 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -62,11 +62,11 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const { - (void)executor; (void)executor_callback; + (void)executor_callback_data; throw std::runtime_error( "Custom waitables should override set_events_executor_callback() to use events executor"); diff --git a/rclcpp/test/rclcpp/CMakeLists.txt b/rclcpp/test/rclcpp/CMakeLists.txt index 13d41a7fca..dc04ec5e43 100644 --- a/rclcpp/test/rclcpp/CMakeLists.txt +++ b/rclcpp/test/rclcpp/CMakeLists.txt @@ -574,7 +574,7 @@ if(TARGET test_events_queue) ament_target_dependencies(test_events_queue "rcl" "test_msgs") - target_link_libraries(test_events_queue ${PROJECT_NAME} mimick) + target_link_libraries(test_events_queue ${PROJECT_NAME}) endif() ament_add_gtest(test_events_executor_entities_collector executors/test_events_executor_entities_collector.cpp diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp index 9f540687d0..9d600e61f7 100644 --- a/rclcpp/test/rclcpp/executors/test_events_queue.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -25,7 +25,7 @@ TEST(TestEventsQueue, SimpleQueueTest) { // Create a SimpleEventsQueue and a local queue auto simple_queue = std::make_unique(); - std::queue local_events_queue; + std::queue local_events_queue; // Make sure the queue is empty after init simple_queue->init(); @@ -33,7 +33,7 @@ TEST(TestEventsQueue, SimpleQueueTest) // Push 11 messages for (int i = 0; i < 11; i++) { - rmw_listener_event_t stub_event; + rclcpp::executors::ExecutorEvent stub_event; simple_queue->push(stub_event); } @@ -52,13 +52,14 @@ TEST(TestEventsQueue, SimpleQueueTest) EXPECT_TRUE(simple_queue->empty()); // Lets push an event into the queue and get it back - rmw_listener_event_t push_event = {simple_queue.get(), SUBSCRIPTION_EVENT}; + rclcpp::executors::ExecutorEvent push_event = {simple_queue.get(), + rclcpp::executors::ExecutorEventType::SUBSCRIPTION_EVENT}; simple_queue->push(push_event); - rmw_listener_event_t front_event = simple_queue->front(); + rclcpp::executors::ExecutorEvent front_event = simple_queue->front(); // The events should be equal - EXPECT_EQ(push_event.entity, front_event.entity); + EXPECT_EQ(push_event.entity_id, front_event.entity_id); EXPECT_EQ(push_event.type, front_event.type); } diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index 76e6e66b3c..c00c8b441b 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -469,14 +469,13 @@ class TestWaitable : public rclcpp::Waitable void set_events_executor_callback( - rclcpp::executors::EventsExecutor * executor, - rmw_listener_callback_t executor_callback) const override + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) const override { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, executor_callback, - executor, - this, + executor_callback_data, true /*Use previous events*/); if (RCL_RET_OK != ret) { From 36e66ed0a06592c7eb68a8ee8e72800dfb2fce8b Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 26 Feb 2021 18:20:12 -0300 Subject: [PATCH 159/168] Use entity_id to compute hash. Add comments --- .../events_executor_entities_collector.hpp | 11 +++++++- .../executors/events_executor_event_types.hpp | 25 +++++++++++-------- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 75716d236b..2781026492 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -269,7 +269,16 @@ class EventsExecutorEntitiesCollector final EventsExecutor * associated_executor_ = nullptr; /// Instance of the timers manager used by the associated executor TimersManager::SharedPtr timers_manager_; - /// Callback data from entities mapped to a counter of each + + /// Callback data objects mapped to the number of listeners sharing the same object. + /// When no more listeners use the object, it can be removed from the map. + /// For example, the entities collector holds every node's guard condition, which + /// share the same EventsExecutorCallbackData object ptr to use as their callback arg: + /// cb_data_object = {executor_ptr, entities_collector_ptr, WAITABLE_EVENT}; + /// Node1->gc(&cb_data_object) + /// Node2->gc(&cb_data_object) + /// So the maps has: (cb_data_object, 2) + /// When both nodes are removed (counter = 0), the cb_data_object can be destroyed. std::unordered_map callback_data_map_; }; diff --git a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp index d1b6e150d1..e03adab2f5 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp @@ -37,16 +37,20 @@ struct ExecutorEvent ExecutorEventType type; }; +// The EventsExecutorCallbackData struct is what the listeners +// will use as argument when calling their callbacks from the +// RMW implementation. The listeners get a (void *) of this struct, +// and the executor is in charge to cast it back and use the data. struct EventsExecutorCallbackData { EventsExecutorCallbackData( - EventsExecutor * exec, - void * id, - ExecutorEventType type) + EventsExecutor * _executor, + void * _entity_id, + ExecutorEventType _event_type) { - executor = exec; - entity_id = id; - event_type = type; + executor = _executor; + entity_id = _entity_id; + event_type = _event_type; } // Equal operator @@ -64,14 +68,13 @@ struct EventsExecutorCallbackData }; // To be able to use std::unordered_map with an EventsExecutorCallbackData -// as key, we need a hasher: +// as key, we need a hasher. We use the entity ID as hash, since it is +// unique for each EventsExecutorCallbackData object. struct KeyHasher { - size_t operator()(const EventsExecutorCallbackData & k) const + size_t operator()(const EventsExecutorCallbackData & key) const { - return ((std::hash()(k.executor) ^ - (std::hash()(k.entity_id) << 1)) >> 1) ^ - (std::hash()(k.event_type) << 1); + return std::hash()(key.entity_id); } }; From 8e137140f477176375131a6ad3125fa1134bb871 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 26 Feb 2021 18:23:57 -0300 Subject: [PATCH 160/168] Use RMW renamed file --- rclcpp/include/rclcpp/executors/events_executor.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index a5ceffd81a..35b7271399 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -29,7 +29,7 @@ #include "rclcpp/experimental/buffers/simple_events_queue.hpp" #include "rclcpp/node.hpp" -#include "rmw/listener_event_types.h" +#include "rmw/listener_callback_type.h" namespace rclcpp { From 2b9b3ad05d22258d65189bc05df7fa2f07281e05 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Fri, 26 Feb 2021 18:29:07 -0300 Subject: [PATCH 161/168] fix hash type --- rclcpp/include/rclcpp/executors/events_executor_event_types.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp index e03adab2f5..2557af107f 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp @@ -74,7 +74,7 @@ struct KeyHasher { size_t operator()(const EventsExecutorCallbackData & key) const { - return std::hash()(key.entity_id); + return std::hash()(key.entity_id); } }; From 930d9a7cf17b4f65e37387d85f0b7f310ec9eb1d Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Wed, 10 Mar 2021 17:51:41 -0300 Subject: [PATCH 162/168] Update EventsExecutorCallbackData --- .../include/rclcpp/executors/events_executor.hpp | 2 +- .../executors/events_executor_event_types.hpp | 15 +++++---------- .../events_executor_entities_collector.cpp | 6 ++++-- 3 files changed, 10 insertions(+), 13 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 35b7271399..375206fca0 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -228,7 +228,7 @@ class EventsExecutor : public rclcpp::Executor // Event queue mutex scope { std::unique_lock lock(this_executor->push_mutex_); - this_executor->events_queue_->push({data->entity_id, data->event_type}); + this_executor->events_queue_->push(data->event); } // Notify that the event queue has some events in it. this_executor->events_queue_cv_.notify_one(); diff --git a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp index 2557af107f..045cc51d57 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp @@ -45,26 +45,21 @@ struct EventsExecutorCallbackData { EventsExecutorCallbackData( EventsExecutor * _executor, - void * _entity_id, - ExecutorEventType _event_type) + ExecutorEvent _event) { executor = _executor; - entity_id = _entity_id; - event_type = _event_type; + event = _event; } // Equal operator bool operator==(const EventsExecutorCallbackData & other) const { - return (executor == other.executor) && - (entity_id == other.entity_id) && - (event_type == other.event_type); + return (event.entity_id == other.event.entity_id); } // Struct members EventsExecutor * executor; - void * entity_id; - ExecutorEventType event_type; + ExecutorEvent event; }; // To be able to use std::unordered_map with an EventsExecutorCallbackData @@ -74,7 +69,7 @@ struct KeyHasher { size_t operator()(const EventsExecutorCallbackData & key) const { - return std::hash()(key.entity_id); + return std::hash()(key.event.entity_id); } }; diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index c6bf4df55d..4198547366 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -607,7 +607,8 @@ EventsExecutorEntitiesCollector::get_callback_data( { // Create an entity callback data object and check if // we already have stored one like it - EventsExecutorCallbackData data(associated_executor_, entity_id, event_type); + ExecutorEvent event = {entity_id, event_type}; + EventsExecutorCallbackData data(associated_executor_, event); auto it = callback_data_map_.find(data); @@ -633,7 +634,8 @@ EventsExecutorEntitiesCollector::remove_callback_data( { // Create an entity callback data object and check if // we already have stored one like it - EventsExecutorCallbackData data(associated_executor_, entity_id, event_type); + ExecutorEvent event = {entity_id, event_type}; + EventsExecutorCallbackData data(associated_executor_, event); auto it = callback_data_map_.find(data); From f487f98eeb53771835a49ffe89fa5be20fe7f58f Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 25 Feb 2021 12:03:28 -0300 Subject: [PATCH 163/168] Add support for actions on EventsExecutor --- rclcpp/include/rclcpp/client.hpp | 2 +- .../rclcpp/executors/events_executor.hpp | 2 +- .../events_executor_entities_collector.hpp | 3 +- .../executors/events_executor_event_types.hpp | 16 +-- .../events_executor_notify_waitable.hpp | 5 +- .../static_executor_entities_collector.hpp | 5 +- .../subscription_intra_process.hpp | 3 +- .../subscription_intra_process_base.hpp | 4 +- rclcpp/include/rclcpp/qos_event.hpp | 5 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/timer.hpp | 12 +- rclcpp/include/rclcpp/waitable.hpp | 10 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/executor.cpp | 2 +- .../src/rclcpp/executors/events_executor.cpp | 2 +- .../events_executor_entities_collector.cpp | 10 +- .../static_executor_entities_collector.cpp | 3 +- .../static_single_threaded_executor.cpp | 2 +- .../src/rclcpp/executors/timers_manager.cpp | 5 +- rclcpp/src/rclcpp/qos_event.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- .../subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/benchmark/benchmark_executor.cpp | 2 +- .../rclcpp/executors/test_events_queue.cpp | 4 +- .../test/rclcpp/executors/test_executors.cpp | 5 +- ...est_static_executor_entities_collector.cpp | 9 +- .../node_interfaces/test_node_timers.cpp | 2 +- .../node_interfaces/test_node_waitables.cpp | 3 +- .../test_allocator_memory_strategy.cpp | 3 +- rclcpp/test/rclcpp/test_memory_strategy.cpp | 2 +- rclcpp/test/rclcpp/test_publisher.cpp | 2 +- rclcpp/test/rclcpp/test_qos_event.cpp | 4 +- .../test_dynamic_storage.cpp | 2 +- .../wait_set_policies/test_static_storage.cpp | 2 +- .../test_storage_policy_common.cpp | 2 +- .../test_thread_safe_synchronization.cpp | 2 +- .../include/rclcpp_action/client.hpp | 12 +- .../include/rclcpp_action/server.hpp | 13 +- rclcpp_action/src/client.cpp | 124 +++++++++++++++++- rclcpp_action/src/server.cpp | 102 +++++++++++++- 43 files changed, 332 insertions(+), 70 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 1a66d128e6..91a49404ab 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -160,7 +160,7 @@ class ClientBase void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + const void * executor_callback_data); protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor.hpp b/rclcpp/include/rclcpp/executors/events_executor.hpp index 35b7271399..375206fca0 100644 --- a/rclcpp/include/rclcpp/executors/events_executor.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor.hpp @@ -228,7 +228,7 @@ class EventsExecutor : public rclcpp::Executor // Event queue mutex scope { std::unique_lock lock(this_executor->push_mutex_); - this_executor->events_queue_->push({data->entity_id, data->event_type}); + this_executor->events_queue_->push(data->event); } // Notify that the event queue has some events in it. this_executor->events_queue_cv_.notify_one(); diff --git a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp index 2781026492..97c05b4a6a 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_entities_collector.hpp @@ -195,8 +195,9 @@ class EventsExecutorEntitiesCollector final RCLCPP_PUBLIC std::shared_ptr - take_data() + take_data(const void * arg) { + (void)arg; // This waitable doesn't handle any data return nullptr; } diff --git a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp index 2557af107f..850361f48e 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_event_types.hpp @@ -34,6 +34,7 @@ enum ExecutorEventType struct ExecutorEvent { const void * entity_id; + const void * entity_arg; ExecutorEventType type; }; @@ -45,26 +46,21 @@ struct EventsExecutorCallbackData { EventsExecutorCallbackData( EventsExecutor * _executor, - void * _entity_id, - ExecutorEventType _event_type) + ExecutorEvent _event) { executor = _executor; - entity_id = _entity_id; - event_type = _event_type; + event = _event; } // Equal operator bool operator==(const EventsExecutorCallbackData & other) const { - return (executor == other.executor) && - (entity_id == other.entity_id) && - (event_type == other.event_type); + return event.entity_id == other.event.entity_id; } // Struct members EventsExecutor * executor; - void * entity_id; - ExecutorEventType event_type; + ExecutorEvent event; }; // To be able to use std::unordered_map with an EventsExecutorCallbackData @@ -74,7 +70,7 @@ struct KeyHasher { size_t operator()(const EventsExecutorCallbackData & key) const { - return std::hash()(key.entity_id); + return std::hash()(key.event.entity_id); } }; diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 5c00a8ed7a..d8920d8411 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -62,7 +62,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override + const void * executor_callback_data) override { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( @@ -79,9 +79,10 @@ class EventsExecutorNotifyWaitable final : public EventWaitable RCLCPP_PUBLIC std::shared_ptr - take_data() + take_data(const void * arg) { // This waitable doesn't handle any data + (void)arg; return nullptr; } diff --git a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp index 51f80b32c9..86af2c9782 100644 --- a/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp +++ b/rclcpp/include/rclcpp/executors/static_executor_entities_collector.hpp @@ -80,11 +80,12 @@ class StaticExecutorEntitiesCollector final /// Take the data so that it can be consumed with `execute`. /** * For `StaticExecutorEntitiesCollector`, this always return `nullptr`. - * \sa rclcpp::Waitable::take_data() + * \param arg Argument to identify data to take + * \sa rclcpp::Waitable::take_data(const void * arg) */ RCLCPP_PUBLIC std::shared_ptr - take_data() override; + take_data(const void * arg) override; /// Function to add_handles_to_wait_set and wait for work and /** diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp index 46e019daee..23fff468c6 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process.hpp @@ -122,8 +122,9 @@ class SubscriptionIntraProcess : public SubscriptionIntraProcessBase } std::shared_ptr - take_data() + take_data(const void * arg) { + (void)arg; ConstMessageSharedPtr shared_msg; MessageUniquePtr unique_msg; diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 9227e21623..684e745813 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -58,7 +58,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable virtual std::shared_ptr - take_data() = 0; + take_data(const void * arg) = 0; virtual void execute(std::shared_ptr & data) = 0; @@ -78,7 +78,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override; + const void * executor_callback_data) override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 4d886a99ff..0636d32b1d 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -112,7 +112,7 @@ class QOSEventHandlerBase : public Waitable void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override; + const void * executor_callback_data) override; protected: rcl_event_t event_handle_; @@ -147,8 +147,9 @@ class QOSEventHandler : public QOSEventHandlerBase /// Take data so that the callback cannot be scheduled again std::shared_ptr - take_data() override + take_data(const void * arg) override { + (void)arg; EventCallbackInfoT callback_info; rcl_ret_t ret = rcl_take_event(&event_handle_, &callback_info); if (ret != RCL_RET_OK) { diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 93a7669dbd..ed99802c81 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -130,7 +130,7 @@ class ServiceBase void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + const void * executor_callback_data); protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 9ae9c30a1b..367dc1491e 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -273,7 +273,7 @@ class SubscriptionBase : public std::enable_shared_from_this void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + const void * executor_callback_data); protected: template diff --git a/rclcpp/include/rclcpp/timer.hpp b/rclcpp/include/rclcpp/timer.hpp index a2662f4abe..778976da81 100644 --- a/rclcpp/include/rclcpp/timer.hpp +++ b/rclcpp/include/rclcpp/timer.hpp @@ -92,8 +92,12 @@ class TimerBase reset(); /// Call the callback function when the timer signal is emitted. + /** + * \return true if callback was executed, false otherwise + * \throws std::runtime_error if failed to notify timer that callback occurred + */ RCLCPP_PUBLIC - virtual void + virtual bool execute_callback() = 0; RCLCPP_PUBLIC @@ -193,14 +197,15 @@ class GenericTimer : public TimerBase /** * \sa rclcpp::TimerBase::execute_callback + * \return true if callback was executed, false otherwise * \throws std::runtime_error if it failed to notify timer that callback occurred */ - void + bool execute_callback() override { rcl_ret_t ret = rcl_timer_call(timer_handle_.get()); if (ret == RCL_RET_TIMER_CANCELED) { - return; + return false; } if (ret != RCL_RET_OK) { throw std::runtime_error("Failed to notify timer that callback occurred"); @@ -208,6 +213,7 @@ class GenericTimer : public TimerBase TRACEPOINT(callback_start, static_cast(&callback_), false); execute_callback_delegate<>(); TRACEPOINT(callback_end, static_cast(&callback_)); + return true; } // void specialization diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 4c89af80ac..475fa3777d 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -141,6 +141,8 @@ class Waitable * This method takes the data from the underlying data structure and * writes it to the void shared pointer `data` that is passed into the * method. The `data` can then be executed with the `execute` method. + * The argument passed can help to identify which data to take for + * some waitables. * * Before calling this method, the Waitable should be added to a wait set, * waited on, and then updated. @@ -158,13 +160,13 @@ class Waitable * // Update the Waitable * waitable.update(wait_set); * // Execute any entities of the Waitable that may be ready - * std::shared_ptr data = waitable.take_data(); + * std::shared_ptr data = waitable.take_data(nullptr); * ``` */ RCLCPP_PUBLIC virtual std::shared_ptr - take_data() = 0; + take_data(const void * arg) = 0; /// Execute data that is passed in. /** @@ -185,7 +187,7 @@ class Waitable * // Update the Waitable * waitable.update(wait_set); * // Execute any entities of the Waitable that may be ready - * std::shared_ptr data = waitable.take_data(); + * std::shared_ptr data = waitable.take_data(nullptr); * waitable.execute(data); * ``` */ @@ -213,7 +215,7 @@ class Waitable void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + const void * executor_callback_data); private: std::atomic in_use_by_wait_set_{false}; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index fc80044525..9543295223 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -202,7 +202,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const + const void * executor_callback_data) { rcl_ret_t ret = rcl_client_set_listener_callback( client_handle_.get(), diff --git a/rclcpp/src/rclcpp/executor.cpp b/rclcpp/src/rclcpp/executor.cpp index d33df58ee2..c8da7528af 100644 --- a/rclcpp/src/rclcpp/executor.cpp +++ b/rclcpp/src/rclcpp/executor.cpp @@ -848,7 +848,7 @@ Executor::get_next_ready_executable_from_map( // Check the waitables to see if there are any that are ready memory_strategy_->get_next_waitable(any_executable, weak_groups_to_nodes); if (any_executable.waitable) { - any_executable.data = any_executable.waitable->take_data(); + any_executable.data = any_executable.waitable->take_data(nullptr); success = true; } } diff --git a/rclcpp/src/rclcpp/executors/events_executor.cpp b/rclcpp/src/rclcpp/executors/events_executor.cpp index eb53c2e3c7..f7943df487 100644 --- a/rclcpp/src/rclcpp/executors/events_executor.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor.cpp @@ -284,7 +284,7 @@ EventsExecutor::execute_event(const ExecutorEvent & event) auto waitable = entities_collector_->get_waitable(event.entity_id); if (waitable) { - auto data = waitable->take_data(); + auto data = waitable->take_data(event.entity_arg); waitable->execute(data); } break; diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index c6bf4df55d..6286e1708a 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -607,12 +607,13 @@ EventsExecutorEntitiesCollector::get_callback_data( { // Create an entity callback data object and check if // we already have stored one like it - EventsExecutorCallbackData data(associated_executor_, entity_id, event_type); + ExecutorEvent event = {entity_id, nullptr /*entity_arg*/, event_type}; + EventsExecutorCallbackData data(associated_executor_, event); auto it = callback_data_map_.find(data); if (it != callback_data_map_.end()) { - // We found a callback data matching entity ID and type. + // We found a callback data matching entity ID. // Increment callback data counter and return pointer to data it->second++; return &it->first; @@ -633,12 +634,13 @@ EventsExecutorEntitiesCollector::remove_callback_data( { // Create an entity callback data object and check if // we already have stored one like it - EventsExecutorCallbackData data(associated_executor_, entity_id, event_type); + ExecutorEvent event = {entity_id, nullptr /*entity_arg*/, event_type}; + EventsExecutorCallbackData data(associated_executor_, event); auto it = callback_data_map_.find(data); if (it != callback_data_map_.end()) { - // We found a callback data matching entity ID and type. + // We found a callback data matching entity ID. // If we have more than 1 decrement counter, otherwise remove it. if (it->second > 1) { it->second--; diff --git a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp index a0f55d3d5b..4e3d89a01f 100644 --- a/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/static_executor_entities_collector.cpp @@ -89,8 +89,9 @@ StaticExecutorEntitiesCollector::fini() } std::shared_ptr -StaticExecutorEntitiesCollector::take_data() +StaticExecutorEntitiesCollector::take_data(const void * arg) { + (void)arg; return nullptr; } diff --git a/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp b/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp index 4b5afc308f..f4e881b5f3 100644 --- a/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp +++ b/rclcpp/src/rclcpp/executors/static_single_threaded_executor.cpp @@ -177,7 +177,7 @@ StaticSingleThreadedExecutor::execute_ready_executables() for (size_t i = 0; i < entities_collector_->get_number_of_waitables(); ++i) { auto waitable = entities_collector_->get_waitable(i); if (waitable->is_ready(&wait_set_)) { - auto data = waitable->take_data(); + auto data = waitable->take_data(nullptr); waitable->execute(data); } } diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index be6309886c..b4a2ef864e 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -139,10 +139,11 @@ bool TimersManager::execute_head_timer( if (timer_ready) { // Head timer is ready, execute - head->execute_callback(); + bool callback_executed = head->execute_callback(); + timers_heap.heapify_root(); weak_timers_heap_.store(timers_heap); - return true; + return callback_executed; } // Head timer was not ready yet diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 055cab306e..cad9ad6ddd 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -71,7 +71,7 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) void QOSEventHandlerBase::set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const + const void * executor_callback_data) { rcl_ret_t ret = rcl_event_set_listener_callback( &event_handle_, diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 0bf08e0dbc..b50465260a 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -88,7 +88,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const + const void * executor_callback_data) { rcl_ret_t ret = rcl_service_set_listener_callback( service_handle_.get(), diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index ef1ad4e10b..54392e7043 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -292,7 +292,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const + const void * executor_callback_data) { rcl_ret_t ret = rcl_subscription_set_listener_callback( subscription_handle_.get(), diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 83c51d3bbb..a4cdca2c3c 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -40,7 +40,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const + const void * executor_callback_data) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 61c2cf3f18..d1b85d446f 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -63,7 +63,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const + const void * executor_callback_data) { (void)executor_callback; (void)executor_callback_data; diff --git a/rclcpp/test/benchmark/benchmark_executor.cpp b/rclcpp/test/benchmark/benchmark_executor.cpp index 27e6d2ff5b..7707a8c610 100644 --- a/rclcpp/test/benchmark/benchmark_executor.cpp +++ b/rclcpp/test/benchmark/benchmark_executor.cpp @@ -384,7 +384,7 @@ BENCHMARK_F( reset_heap_counters(); for (auto _ : st) { - std::shared_ptr data = entities_collector_->take_data(); + std::shared_ptr data = entities_collector_->take_data(nullptr); entities_collector_->execute(data); } } diff --git a/rclcpp/test/rclcpp/executors/test_events_queue.cpp b/rclcpp/test/rclcpp/executors/test_events_queue.cpp index 9d600e61f7..5fac068c3a 100644 --- a/rclcpp/test/rclcpp/executors/test_events_queue.cpp +++ b/rclcpp/test/rclcpp/executors/test_events_queue.cpp @@ -52,7 +52,9 @@ TEST(TestEventsQueue, SimpleQueueTest) EXPECT_TRUE(simple_queue->empty()); // Lets push an event into the queue and get it back - rclcpp::executors::ExecutorEvent push_event = {simple_queue.get(), + rclcpp::executors::ExecutorEvent push_event = { + simple_queue.get(), + nullptr, rclcpp::executors::ExecutorEventType::SUBSCRIPTION_EVENT}; simple_queue->push(push_event); diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index c00c8b441b..dad285d694 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -445,8 +445,9 @@ class TestWaitable : public rclcpp::Waitable } std::shared_ptr - take_data() override + take_data(const void * arg) override { + (void)arg; return nullptr; } @@ -470,7 +471,7 @@ class TestWaitable : public rclcpp::Waitable void set_events_executor_callback( rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override + const void * executor_callback_data) override { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, diff --git a/rclcpp/test/rclcpp/executors/test_static_executor_entities_collector.cpp b/rclcpp/test/rclcpp/executors/test_static_executor_entities_collector.cpp index e39589c3c6..302372ecc9 100644 --- a/rclcpp/test/rclcpp/executors/test_static_executor_entities_collector.cpp +++ b/rclcpp/test/rclcpp/executors/test_static_executor_entities_collector.cpp @@ -236,8 +236,9 @@ class TestWaitable : public rclcpp::Waitable bool is_ready(rcl_wait_set_t *) override {return true;} std::shared_ptr - take_data() override + take_data(const void * arg) override { + (void)arg; return nullptr; } void @@ -372,7 +373,7 @@ TEST_F(TestStaticExecutorEntitiesCollector, prepare_wait_set_rcl_wait_set_clear_ { auto mock = mocking_utils::patch_and_return("lib:rclcpp", rcl_wait_set_clear, RCL_RET_ERROR); - std::shared_ptr data = entities_collector_->take_data(); + std::shared_ptr data = entities_collector_->take_data(nullptr); RCLCPP_EXPECT_THROW_EQ( entities_collector_->execute(data), std::runtime_error("Couldn't clear wait set")); @@ -403,7 +404,7 @@ TEST_F(TestStaticExecutorEntitiesCollector, prepare_wait_set_rcl_wait_set_resize { auto mock = mocking_utils::patch_and_return("lib:rclcpp", rcl_wait_set_resize, RCL_RET_ERROR); - std::shared_ptr data = entities_collector_->take_data(); + std::shared_ptr data = entities_collector_->take_data(nullptr); RCLCPP_EXPECT_THROW_EQ( entities_collector_->execute(data), std::runtime_error("Couldn't resize the wait set: error not set")); @@ -627,7 +628,7 @@ TEST_F( RCLCPP_SCOPE_EXIT(entities_collector_->fini()); cb_group->get_associated_with_executor_atomic().exchange(false); - std::shared_ptr data = entities_collector_->take_data(); + std::shared_ptr data = entities_collector_->take_data(nullptr); entities_collector_->execute(data); EXPECT_TRUE(entities_collector_->remove_node(node->get_node_base_interface())); diff --git a/rclcpp/test/rclcpp/node_interfaces/test_node_timers.cpp b/rclcpp/test/rclcpp/node_interfaces/test_node_timers.cpp index e206e75b6c..d3358802bd 100644 --- a/rclcpp/test/rclcpp/node_interfaces/test_node_timers.cpp +++ b/rclcpp/test/rclcpp/node_interfaces/test_node_timers.cpp @@ -32,7 +32,7 @@ class TestTimer : public rclcpp::TimerBase : TimerBase(node->get_clock(), std::chrono::nanoseconds(1), node->get_node_base_interface()->get_context()) {} - void execute_callback() override {} + bool execute_callback() override {return false;} bool is_steady() override {return false;} }; diff --git a/rclcpp/test/rclcpp/node_interfaces/test_node_waitables.cpp b/rclcpp/test/rclcpp/node_interfaces/test_node_waitables.cpp index a12ef36eab..c50980ee8f 100644 --- a/rclcpp/test/rclcpp/node_interfaces/test_node_waitables.cpp +++ b/rclcpp/test/rclcpp/node_interfaces/test_node_waitables.cpp @@ -32,8 +32,9 @@ class TestWaitable : public rclcpp::Waitable bool is_ready(rcl_wait_set_t *) override {return false;} std::shared_ptr - take_data() override + take_data(const void * arg) override { + (void)arg; return nullptr; } diff --git a/rclcpp/test/rclcpp/strategies/test_allocator_memory_strategy.cpp b/rclcpp/test/rclcpp/strategies/test_allocator_memory_strategy.cpp index 696b9f319a..a32ac431b4 100644 --- a/rclcpp/test/rclcpp/strategies/test_allocator_memory_strategy.cpp +++ b/rclcpp/test/rclcpp/strategies/test_allocator_memory_strategy.cpp @@ -50,8 +50,9 @@ class TestWaitable : public rclcpp::Waitable } std::shared_ptr - take_data() override + take_data(const void * arg) override { + (void)arg; return nullptr; } diff --git a/rclcpp/test/rclcpp/test_memory_strategy.cpp b/rclcpp/test/rclcpp/test_memory_strategy.cpp index 93fa0c8d5e..26c8aa582a 100644 --- a/rclcpp/test/rclcpp/test_memory_strategy.cpp +++ b/rclcpp/test/rclcpp/test_memory_strategy.cpp @@ -37,7 +37,7 @@ class TestWaitable : public rclcpp::Waitable public: bool add_to_wait_set(rcl_wait_set_t *) override {return true;} bool is_ready(rcl_wait_set_t *) override {return true;} - std::shared_ptr take_data() override {return nullptr;} + std::shared_ptr take_data(const void * arg) override {(void)arg; return nullptr;} void execute(std::shared_ptr & data) override {(void)data;} }; diff --git a/rclcpp/test/rclcpp/test_publisher.cpp b/rclcpp/test/rclcpp/test_publisher.cpp index 5f0712fddc..104f24a183 100644 --- a/rclcpp/test/rclcpp/test_publisher.cpp +++ b/rclcpp/test/rclcpp/test_publisher.cpp @@ -505,7 +505,7 @@ TEST_F(TestPublisher, run_event_handlers) { auto publisher = node->create_publisher("topic", 10); for (const auto & handler : publisher->get_event_handlers()) { - std::shared_ptr data = handler->take_data(); + std::shared_ptr data = handler->take_data(nullptr); handler->execute(data); } } diff --git a/rclcpp/test/rclcpp/test_qos_event.cpp b/rclcpp/test/rclcpp/test_qos_event.cpp index 6c94aaff97..154b7b3797 100644 --- a/rclcpp/test/rclcpp/test_qos_event.cpp +++ b/rclcpp/test/rclcpp/test_qos_event.cpp @@ -285,7 +285,7 @@ TEST_F(TestQosEvent, execute) { rclcpp::QOSEventHandler handler( callback, rcl_publisher_event_init, rcl_handle, event_type); - std::shared_ptr data = handler.take_data(); + std::shared_ptr data = handler.take_data(nullptr); EXPECT_NO_THROW(handler.execute(data)); EXPECT_TRUE(handler_callback_executed); @@ -293,7 +293,7 @@ TEST_F(TestQosEvent, execute) { handler_callback_executed = false; // Logs error and returns early auto mock = mocking_utils::patch_and_return("lib:rclcpp", rcl_take_event, RCL_RET_ERROR); - std::shared_ptr data = handler.take_data(); + std::shared_ptr data = handler.take_data(nullptr); EXPECT_THROW(handler.execute(data), std::runtime_error); EXPECT_FALSE(handler_callback_executed); } diff --git a/rclcpp/test/rclcpp/wait_set_policies/test_dynamic_storage.cpp b/rclcpp/test/rclcpp/wait_set_policies/test_dynamic_storage.cpp index 2282a92bb1..addb619a2c 100644 --- a/rclcpp/test/rclcpp/wait_set_policies/test_dynamic_storage.cpp +++ b/rclcpp/test/rclcpp/wait_set_policies/test_dynamic_storage.cpp @@ -55,7 +55,7 @@ class TestWaitable : public rclcpp::Waitable bool is_ready(rcl_wait_set_t *) override {return is_ready_;} - std::shared_ptr take_data() override {return nullptr;} + std::shared_ptr take_data(const void * arg) override {(void)arg; return nullptr;} void execute(std::shared_ptr & data) override {(void)data;} diff --git a/rclcpp/test/rclcpp/wait_set_policies/test_static_storage.cpp b/rclcpp/test/rclcpp/wait_set_policies/test_static_storage.cpp index 8e406276f1..05c49d64a5 100644 --- a/rclcpp/test/rclcpp/wait_set_policies/test_static_storage.cpp +++ b/rclcpp/test/rclcpp/wait_set_policies/test_static_storage.cpp @@ -54,7 +54,7 @@ class TestWaitable : public rclcpp::Waitable bool is_ready(rcl_wait_set_t *) override {return is_ready_;} - std::shared_ptr take_data() override {return nullptr;} + std::shared_ptr take_data(const void * arg) override {(void)arg; return nullptr;} void execute(std::shared_ptr & data) override {(void)data;} diff --git a/rclcpp/test/rclcpp/wait_set_policies/test_storage_policy_common.cpp b/rclcpp/test/rclcpp/wait_set_policies/test_storage_policy_common.cpp index 130d281e13..f943f15f3d 100644 --- a/rclcpp/test/rclcpp/wait_set_policies/test_storage_policy_common.cpp +++ b/rclcpp/test/rclcpp/wait_set_policies/test_storage_policy_common.cpp @@ -54,7 +54,7 @@ class TestWaitable : public rclcpp::Waitable bool is_ready(rcl_wait_set_t *) override {return is_ready_;} - std::shared_ptr take_data() override {return nullptr;} + std::shared_ptr take_data(const void * arg) override {(void)arg; return nullptr;} void execute(std::shared_ptr & data) override {(void)data;} diff --git a/rclcpp/test/rclcpp/wait_set_policies/test_thread_safe_synchronization.cpp b/rclcpp/test/rclcpp/wait_set_policies/test_thread_safe_synchronization.cpp index 8f4870f156..61bcdaa308 100644 --- a/rclcpp/test/rclcpp/wait_set_policies/test_thread_safe_synchronization.cpp +++ b/rclcpp/test/rclcpp/wait_set_policies/test_thread_safe_synchronization.cpp @@ -54,7 +54,7 @@ class TestWaitable : public rclcpp::Waitable bool is_ready(rcl_wait_set_t *) override {return is_ready_;} - std::shared_ptr take_data() override {return nullptr;} + std::shared_ptr take_data(const void * arg) override {(void)arg; return nullptr;} void execute(std::shared_ptr & data) override {(void)data;} diff --git a/rclcpp_action/include/rclcpp_action/client.hpp b/rclcpp_action/include/rclcpp_action/client.hpp index 8294f2be68..e9c6bff0dc 100644 --- a/rclcpp_action/include/rclcpp_action/client.hpp +++ b/rclcpp_action/include/rclcpp_action/client.hpp @@ -16,6 +16,7 @@ #define RCLCPP_ACTION__CLIENT_HPP_ #include +#include "rclcpp/executors/events_executor_event_types.hpp" #include #include #include @@ -119,13 +120,20 @@ class ClientBase : public rclcpp::Waitable /// \internal RCLCPP_ACTION_PUBLIC std::shared_ptr - take_data() override; + take_data(const void * arg) override; /// \internal RCLCPP_ACTION_PUBLIC void execute(std::shared_ptr & data) override; + /// \internal + RCLCPP_PUBLIC + void + set_events_executor_callback( + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) override; + // End Waitables API // ----------------- @@ -246,6 +254,8 @@ class ClientBase : public rclcpp::Waitable private: std::unique_ptr pimpl_; + std::unordered_map id_to_action_client_event_map_; + std::vector action_client_callback_data_; }; /// Action Client diff --git a/rclcpp_action/include/rclcpp_action/server.hpp b/rclcpp_action/include/rclcpp_action/server.hpp index b3871060cd..acefe86b4a 100644 --- a/rclcpp_action/include/rclcpp_action/server.hpp +++ b/rclcpp_action/include/rclcpp_action/server.hpp @@ -18,6 +18,7 @@ #include #include #include +#include "rclcpp/executors/events_executor_event_types.hpp" #include #include #include @@ -120,7 +121,7 @@ class ServerBase : public rclcpp::Waitable RCLCPP_ACTION_PUBLIC std::shared_ptr - take_data() override; + take_data(const void * arg) override; /// Act on entities in the wait set which are ready to be acted upon. /// \internal @@ -128,6 +129,14 @@ class ServerBase : public rclcpp::Waitable void execute(std::shared_ptr & data) override; + /// Set events executor callback + /// \internal + RCLCPP_PUBLIC + void + set_events_executor_callback( + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) override; + // End Waitables API // ----------------- @@ -256,6 +265,8 @@ class ServerBase : public rclcpp::Waitable /// Private implementation /// \internal std::unique_ptr pimpl_; + std::unordered_map id_to_action_server_event_map_; + std::vector action_server_callback_data_; }; /// Action Server diff --git a/rclcpp_action/src/client.cpp b/rclcpp_action/src/client.cpp index 3a255184bf..ec52dd3f3b 100644 --- a/rclcpp_action/src/client.cpp +++ b/rclcpp_action/src/client.cpp @@ -135,6 +135,8 @@ ClientBase::ClientBase( ClientBase::~ClientBase() { + action_client_callback_data_.clear(); + id_to_action_client_event_map_.clear(); } bool @@ -383,8 +385,55 @@ ClientBase::generate_goal_id() } std::shared_ptr -ClientBase::take_data() +ClientBase::take_data(const void * arg) { + if (arg) { + // Find the action client entity which is ready + auto it = id_to_action_client_event_map_.find(arg); + + if (it != id_to_action_client_event_map_.end()) { + rcl_action_client_entity_type_t ready_entity = it->second; + + switch (ready_entity) { + case GOAL_CLIENT: + { + pimpl_->is_goal_response_ready = true; + break; + } + + case RESULT_CLIENT: + { + pimpl_->is_result_response_ready = true; + break; + } + + case CANCEL_CLIENT: + { + pimpl_->is_cancel_response_ready = true; + break; + } + + case FEEDBACK_SUBSCRIPTION: + { + pimpl_->is_feedback_ready = true; + break; + } + + case STATUS_SUBSCRIPTION: + { + pimpl_->is_status_ready = true; + break; + } + + default: + throw std::runtime_error("default case. Break"); + break; + } + } else { + throw std::runtime_error("ClientBase: Ready entity not found"); + } + } + if (pimpl_->is_feedback_ready) { std::shared_ptr feedback_message = this->create_feedback_message(); rcl_ret_t ret = rcl_action_take_feedback( @@ -496,4 +545,77 @@ ClientBase::execute(std::shared_ptr & data) } } +void +ClientBase::set_events_executor_callback( + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) +{ + if ((executor_callback == nullptr) || (executor_callback_data == nullptr)) { + // Not valid arguments. Unset listeners callback data. + rcl_ret_t ret = rcl_action_client_set_listeners_callback( + pimpl_->client_handle.get(), + nullptr, + nullptr); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("ClientBase: Couldn't unset events executor callbacks."); + } + + return; + } + + auto data = static_cast(executor_callback_data); + + // Create an ExecutorEvent per action client event type + rclcpp::executors::ExecutorEvent goal_response = {data->event.entity_id, nullptr, data->event.type}; + rclcpp::executors::ExecutorEvent cancel_response = {data->event.entity_id, nullptr, data->event.type}; + rclcpp::executors::ExecutorEvent result_response = {data->event.entity_id, nullptr, data->event.type}; + rclcpp::executors::ExecutorEvent feedback_ready = {data->event.entity_id, nullptr, data->event.type}; + rclcpp::executors::ExecutorEvent status_ready = {data->event.entity_id, nullptr, data->event.type}; + + // Get the event ID of each action client event type. + const void * events_id[ACTION_CLIENT_NUM_ENTITIES]; + rcl_action_client_get_events_id(pimpl_->client_handle.get(), events_id); + + // Fill the ExecutorEvents entity_arg field. + goal_response.entity_arg = events_id[GOAL_CLIENT]; + cancel_response.entity_arg = events_id[CANCEL_CLIENT]; + result_response.entity_arg = events_id[RESULT_CLIENT]; + feedback_ready.entity_arg = events_id[FEEDBACK_SUBSCRIPTION]; + status_ready.entity_arg = events_id[STATUS_SUBSCRIPTION]; + + // Map the events ID with the event type + id_to_action_client_event_map_.emplace(events_id[GOAL_CLIENT], GOAL_CLIENT); + id_to_action_client_event_map_.emplace(events_id[CANCEL_CLIENT], CANCEL_CLIENT); + id_to_action_client_event_map_.emplace(events_id[RESULT_CLIENT], RESULT_CLIENT); + id_to_action_client_event_map_.emplace(events_id[FEEDBACK_SUBSCRIPTION], FEEDBACK_SUBSCRIPTION); + id_to_action_client_event_map_.emplace(events_id[STATUS_SUBSCRIPTION], STATUS_SUBSCRIPTION); + + // Store all action clients callback data in the std::vector + action_client_callback_data_.reserve(ACTION_CLIENT_NUM_ENTITIES); + action_client_callback_data_[GOAL_CLIENT] = {data->executor, goal_response}; + action_client_callback_data_[RESULT_CLIENT] = {data->executor, result_response}; + action_client_callback_data_[CANCEL_CLIENT] = {data->executor, cancel_response}; + action_client_callback_data_[FEEDBACK_SUBSCRIPTION] = {data->executor, feedback_ready}; + action_client_callback_data_[STATUS_SUBSCRIPTION] = {data->executor, status_ready}; + + // Create simple array to pass the callback data of events to rcl + const void * action_client_cb_data[ACTION_CLIENT_NUM_ENTITIES]; + + action_client_cb_data[GOAL_CLIENT] = &action_client_callback_data_[GOAL_CLIENT]; + action_client_cb_data[RESULT_CLIENT] = &action_client_callback_data_[RESULT_CLIENT]; + action_client_cb_data[CANCEL_CLIENT] = &action_client_callback_data_[CANCEL_CLIENT]; + action_client_cb_data[FEEDBACK_SUBSCRIPTION] = &action_client_callback_data_[FEEDBACK_SUBSCRIPTION]; + action_client_cb_data[STATUS_SUBSCRIPTION] = &action_client_callback_data_[STATUS_SUBSCRIPTION]; + + // Set listener callbacks to the action client entities + rcl_ret_t ret = rcl_action_client_set_listeners_callback( + pimpl_->client_handle.get(), + executor_callback, + action_client_cb_data); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("ClientBase: Couldn't set events executor callbacks."); + } +} } // namespace rclcpp_action diff --git a/rclcpp_action/src/server.cpp b/rclcpp_action/src/server.cpp index f21133d3ff..61f0955e9f 100644 --- a/rclcpp_action/src/server.cpp +++ b/rclcpp_action/src/server.cpp @@ -130,6 +130,8 @@ ServerBase::ServerBase( ServerBase::~ServerBase() { + id_to_action_server_event_map_.clear(); + action_server_callback_data_.clear(); } size_t @@ -206,8 +208,42 @@ ServerBase::is_ready(rcl_wait_set_t * wait_set) } std::shared_ptr -ServerBase::take_data() +ServerBase::take_data(const void * arg) { + if (arg) { + // Find the action server entity which is ready + auto it = id_to_action_server_event_map_.find(arg); + + if (it != id_to_action_server_event_map_.end()) { + rcl_action_server_entity_type_t ready_entity = it->second; + + switch (ready_entity) { + case GOAL_SERVICE: + { + pimpl_->goal_request_ready_.store(true); + break; + } + + case CANCEL_SERVICE: + { + pimpl_->cancel_request_ready_.store(true); + break; + } + + case RESULT_SERVICE: + { + pimpl_->result_request_ready_.store(true); + break; + } + default: + throw std::runtime_error("default case. Break"); + break; + } + } else { + throw std::runtime_error("ServerBase: Ready entity not found"); + } + } + if (pimpl_->goal_request_ready_.load()) { rcl_ret_t ret; rcl_action_goal_info_t goal_info = rcl_action_get_zero_initialized_goal_info(); @@ -663,3 +699,67 @@ ServerBase::publish_feedback(std::shared_ptr feedback_msg) rclcpp::exceptions::throw_from_rcl_error(ret, "Failed to publish feedback"); } } + +void +ServerBase::set_events_executor_callback( + rmw_listener_callback_t executor_callback, + const void * executor_callback_data) +{ + if ((executor_callback == nullptr) || (executor_callback_data == nullptr)) { + // Not valid arguments. Unset listeners callback data. + rcl_ret_t ret = rcl_action_server_set_listeners_callback( + pimpl_->action_server_.get(), + nullptr, + nullptr); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("ServerBase: Couldn't unset events executor callbacks."); + } + + return; + } + + auto data = static_cast(executor_callback_data); + + // Create an ExecutorEvent per action server event type + rclcpp::executors::ExecutorEvent goal_request = {data->event.entity_id, nullptr, data->event.type}; + rclcpp::executors::ExecutorEvent result_request = {data->event.entity_id, nullptr, data->event.type}; + rclcpp::executors::ExecutorEvent cancel_request = {data->event.entity_id, nullptr, data->event.type}; + + // Get the event ID of each action server event type. + const void * events_id[ACTION_SERVER_NUM_ENTITIES]; + rcl_action_server_get_events_id(pimpl_->action_server_.get(), events_id); + + // Fill the ExecutorEvents entity_arg field. + goal_request.entity_arg = events_id[GOAL_SERVICE]; + result_request.entity_arg = events_id[RESULT_SERVICE]; + cancel_request.entity_arg = events_id[CANCEL_SERVICE]; + + // Map the events ID with the event type + id_to_action_server_event_map_.emplace(events_id[GOAL_SERVICE], GOAL_SERVICE); + id_to_action_server_event_map_.emplace(events_id[RESULT_SERVICE], RESULT_SERVICE); + id_to_action_server_event_map_.emplace(events_id[CANCEL_SERVICE], CANCEL_SERVICE); + + // Store all action server callback data in the std::vector + action_server_callback_data_.reserve(ACTION_SERVER_NUM_ENTITIES); + action_server_callback_data_[GOAL_SERVICE] = {data->executor, goal_request}; + action_server_callback_data_[RESULT_SERVICE] = {data->executor, result_request}; + action_server_callback_data_[CANCEL_SERVICE] = {data->executor, cancel_request}; + + // Create simple array to pass the callbacks data of events to rcl + const void * action_server_cb_data[ACTION_SERVER_NUM_ENTITIES]; + + action_server_cb_data[GOAL_SERVICE] = &action_server_callback_data_[GOAL_SERVICE]; + action_server_cb_data[RESULT_SERVICE] = &action_server_callback_data_[RESULT_SERVICE]; + action_server_cb_data[CANCEL_SERVICE] = &action_server_callback_data_[CANCEL_SERVICE]; + + // Set listener callbacks to the action server entities + rcl_ret_t ret = rcl_action_server_set_listeners_callback( + pimpl_->action_server_.get(), + executor_callback, + action_server_cb_data); + + if (ret != RCL_RET_OK) { + throw std::runtime_error("ServerBase: Couldn't set events executor callbacks."); + } +} From 52afe8168849710279b8b0c198d07a50d7bd1101 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 11 Mar 2021 11:51:59 -0300 Subject: [PATCH 164/168] Use RCLCPP_SMART_PTR_ALIASES_ONLY --- rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp index 92d6e5e6c8..f0958d36bd 100644 --- a/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp +++ b/rclcpp/include/rclcpp/experimental/buffers/events_queue.hpp @@ -42,7 +42,7 @@ namespace buffers class EventsQueue { public: - RCLCPP_SMART_PTR_DEFINITIONS(EventsQueue) + RCLCPP_SMART_PTR_ALIASES_ONLY(EventsQueue) /** * @brief Destruct the object. From df37d31e6b48d37dda2b45bac10d49405209150a Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 11 Mar 2021 12:16:44 -0300 Subject: [PATCH 165/168] Rename set_events_executor_callback->set_listener_callback --- rclcpp/include/rclcpp/client.hpp | 6 +++--- .../events_executor_notify_waitable.hpp | 10 +++++----- .../subscription_intra_process_base.hpp | 6 +++--- rclcpp/include/rclcpp/qos_event.hpp | 7 +++---- rclcpp/include/rclcpp/service.hpp | 6 +++--- rclcpp/include/rclcpp/subscription_base.hpp | 6 +++--- rclcpp/include/rclcpp/waitable.hpp | 6 +++--- rclcpp/src/rclcpp/client.cpp | 12 ++++++------ .../events_executor_entities_collector.cpp | 18 +++++++++--------- rclcpp/src/rclcpp/qos_event.cpp | 12 ++++++------ rclcpp/src/rclcpp/service.cpp | 12 ++++++------ rclcpp/src/rclcpp/subscription_base.cpp | 12 ++++++------ .../rclcpp/subscription_intra_process_base.cpp | 12 ++++++------ rclcpp/src/rclcpp/waitable.cpp | 12 ++++++------ .../test/rclcpp/executors/test_executors.cpp | 10 +++++----- 15 files changed, 73 insertions(+), 74 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index 1a66d128e6..b8a2e484ae 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -158,9 +158,9 @@ class ClientBase RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const; protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 5c00a8ed7a..1a9a72ebff 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -60,15 +60,15 @@ class EventsExecutorNotifyWaitable final : public EventWaitable RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const override { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( gc, - executor_callback, - executor_callback_data, + callback, + user_data, false); if (RCL_RET_OK != ret) { diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 9227e21623..f40f997762 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -76,9 +76,9 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 4d886a99ff..afca761961 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -107,12 +107,11 @@ class QOSEventHandlerBase : public Waitable bool is_ready(rcl_wait_set_t * wait_set) override; - /// Set EventsExecutor's callback RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const override; protected: rcl_event_t event_handle_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index 93a7669dbd..bf0471d4b5 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -128,9 +128,9 @@ class ServiceBase RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const; protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 9ae9c30a1b..2b5653f896 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -271,9 +271,9 @@ class SubscriptionBase : public std::enable_shared_from_this RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const; protected: template diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 4c89af80ac..dc194e549b 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -211,9 +211,9 @@ class Waitable RCLCPP_PUBLIC virtual void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const; private: std::atomic in_use_by_wait_set_{false}; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index fc80044525..c74dd59c8d 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -200,16 +200,16 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ClientBase::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const +ClientBase::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const { rcl_ret_t ret = rcl_client_set_listener_callback( client_handle_.get(), - executor_callback, - executor_callback_data); + callback, + user_data); if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set the EventsExecutor's callback to client"); + throw std::runtime_error("Couldn't set listener callback to client"); } } diff --git a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp index 4198547366..a02c07c641 100644 --- a/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp +++ b/rclcpp/src/rclcpp/executors/events_executor_entities_collector.cpp @@ -238,7 +238,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( if (subscription) { weak_subscriptions_map_.emplace(subscription.get(), subscription); - subscription->set_events_executor_callback( + subscription->set_listener_callback( &EventsExecutor::push_event, get_callback_data(subscription.get(), SUBSCRIPTION_EVENT)); } @@ -249,7 +249,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( if (service) { weak_services_map_.emplace(service.get(), service); - service->set_events_executor_callback( + service->set_listener_callback( &EventsExecutor::push_event, get_callback_data(service.get(), SERVICE_EVENT)); } @@ -260,7 +260,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( if (client) { weak_clients_map_.emplace(client.get(), client); - client->set_events_executor_callback( + client->set_listener_callback( &EventsExecutor::push_event, get_callback_data(client.get(), CLIENT_EVENT)); } @@ -271,7 +271,7 @@ EventsExecutorEntitiesCollector::set_callback_group_entities_callbacks( if (waitable) { weak_waitables_map_.emplace(waitable.get(), waitable); - waitable->set_events_executor_callback( + waitable->set_listener_callback( &EventsExecutor::push_event, get_callback_data(waitable.get(), WAITABLE_EVENT)); } @@ -296,7 +296,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_subscription_ptrs_if( [this](const rclcpp::SubscriptionBase::SharedPtr & subscription) { if (subscription) { - subscription->set_events_executor_callback(nullptr, nullptr); + subscription->set_listener_callback(nullptr, nullptr); weak_subscriptions_map_.erase(subscription.get()); remove_callback_data(subscription.get(), SUBSCRIPTION_EVENT); } @@ -305,7 +305,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_service_ptrs_if( [this](const rclcpp::ServiceBase::SharedPtr & service) { if (service) { - service->set_events_executor_callback(nullptr, nullptr); + service->set_listener_callback(nullptr, nullptr); weak_services_map_.erase(service.get()); remove_callback_data(service.get(), SERVICE_EVENT); } @@ -314,7 +314,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_client_ptrs_if( [this](const rclcpp::ClientBase::SharedPtr & client) { if (client) { - client->set_events_executor_callback(nullptr, nullptr); + client->set_listener_callback(nullptr, nullptr); weak_clients_map_.erase(client.get()); remove_callback_data(client.get(), CLIENT_EVENT); } @@ -323,7 +323,7 @@ EventsExecutorEntitiesCollector::unset_callback_group_entities_callbacks( group->find_waitable_ptrs_if( [this](const rclcpp::Waitable::SharedPtr & waitable) { if (waitable) { - waitable->set_events_executor_callback(nullptr, nullptr); + waitable->set_listener_callback(nullptr, nullptr); weak_waitables_map_.erase(waitable.get()); remove_callback_data(waitable.get(), WAITABLE_EVENT); } @@ -596,7 +596,7 @@ EventsExecutorEntitiesCollector::add_waitable(rclcpp::Waitable::SharedPtr waitab { weak_waitables_map_.emplace(waitable.get(), waitable); - waitable->set_events_executor_callback( + waitable->set_listener_callback( &EventsExecutor::push_event, get_callback_data(waitable.get(), WAITABLE_EVENT)); } diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 055cab306e..6a23b95c2f 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -69,18 +69,18 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) } void -QOSEventHandlerBase::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const +QOSEventHandlerBase::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const { rcl_ret_t ret = rcl_event_set_listener_callback( &event_handle_, - executor_callback, - executor_callback_data, + callback, + user_data, false /* Discard previous events */); if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set EventsExecutor's callback in QOSEventHandlerBase"); + throw std::runtime_error("Couldn't set listener callback to QOSEventHandlerBase"); } } diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index 0bf08e0dbc..e279668bdf 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -86,16 +86,16 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) } void -ServiceBase::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const +ServiceBase::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const { rcl_ret_t ret = rcl_service_set_listener_callback( service_handle_.get(), - executor_callback, - executor_callback_data); + callback, + user_data); if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set the EventsExecutor's callback to service"); + throw std::runtime_error("Couldn't set listener callback to service"); } } diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index ef1ad4e10b..065de5cbbd 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -290,16 +290,16 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( } void -SubscriptionBase::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const +SubscriptionBase::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const { rcl_ret_t ret = rcl_subscription_set_listener_callback( subscription_handle_.get(), - executor_callback, - executor_callback_data); + callback, + user_data); if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set the EventsExecutor's callback to subscription"); + throw std::runtime_error("Couldn't set listener callback to subscription"); } } diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 83c51d3bbb..65e3cbbe83 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -38,17 +38,17 @@ SubscriptionIntraProcessBase::get_actual_qos() const } void -SubscriptionIntraProcessBase::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const +SubscriptionIntraProcessBase::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, - executor_callback, - executor_callback_data, + callback, + user_data, true /*Use previous events*/); if (RCL_RET_OK != ret) { - throw std::runtime_error("Couldn't set guard condition callback"); + throw std::runtime_error("Couldn't set guard condition listener callback"); } } diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 61c2cf3f18..213fc9353a 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -61,13 +61,13 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) } void -Waitable::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const +Waitable::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const { - (void)executor_callback; - (void)executor_callback_data; + (void)callback; + (void)user_data; throw std::runtime_error( - "Custom waitables should override set_events_executor_callback() to use events executor"); + "Custom waitables should override set_listener_callback() to use events executor"); } diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index c00c8b441b..1a3602d3a9 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -468,14 +468,14 @@ class TestWaitable : public rclcpp::Waitable } void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) const override + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) const override { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, - executor_callback, - executor_callback_data, + callback, + user_data, true /*Use previous events*/); if (RCL_RET_OK != ret) { From 4dc6f67cea455d71d8a07edcfca8b5d18d7f0a36 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Thu, 11 Mar 2021 16:36:39 -0300 Subject: [PATCH 166/168] Changes --- rclcpp/include/rclcpp/client.hpp | 2 +- .../executors/events_executor_notify_waitable.hpp | 2 +- .../experimental/subscription_intra_process_base.hpp | 2 +- rclcpp/include/rclcpp/qos_event.hpp | 2 +- rclcpp/include/rclcpp/service.hpp | 2 +- rclcpp/include/rclcpp/subscription_base.hpp | 2 +- rclcpp/include/rclcpp/waitable.hpp | 2 +- rclcpp/src/rclcpp/client.cpp | 2 +- rclcpp/src/rclcpp/qos_event.cpp | 2 +- rclcpp/src/rclcpp/service.cpp | 2 +- rclcpp/src/rclcpp/subscription_base.cpp | 2 +- .../src/rclcpp/subscription_intra_process_base.cpp | 2 +- rclcpp/src/rclcpp/waitable.cpp | 2 +- rclcpp/test/rclcpp/executors/test_executors.cpp | 2 +- rclcpp_action/include/rclcpp_action/client.hpp | 6 +++--- rclcpp_action/include/rclcpp_action/server.hpp | 6 +++--- rclcpp_action/src/client.cpp | 12 ++++++------ rclcpp_action/src/server.cpp | 12 ++++++------ 18 files changed, 32 insertions(+), 32 deletions(-) diff --git a/rclcpp/include/rclcpp/client.hpp b/rclcpp/include/rclcpp/client.hpp index b8a2e484ae..ebed4b025d 100644 --- a/rclcpp/include/rclcpp/client.hpp +++ b/rclcpp/include/rclcpp/client.hpp @@ -160,7 +160,7 @@ class ClientBase void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const; + const void * user_data); protected: RCLCPP_DISABLE_COPY(ClientBase) diff --git a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp index 1c5ba9c8e3..bb4709255b 100644 --- a/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp +++ b/rclcpp/include/rclcpp/executors/events_executor_notify_waitable.hpp @@ -62,7 +62,7 @@ class EventsExecutorNotifyWaitable final : public EventWaitable void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const override + const void * user_data) override { for (auto gc : notify_guard_conditions_) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( diff --git a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp index 21610f8e4c..d1cd7c7402 100644 --- a/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp +++ b/rclcpp/include/rclcpp/experimental/subscription_intra_process_base.hpp @@ -78,7 +78,7 @@ class SubscriptionIntraProcessBase : public rclcpp::Waitable void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const override; + const void * user_data) override; protected: std::recursive_mutex reentrant_mutex_; diff --git a/rclcpp/include/rclcpp/qos_event.hpp b/rclcpp/include/rclcpp/qos_event.hpp index 824d909e8d..cd33d0ba98 100644 --- a/rclcpp/include/rclcpp/qos_event.hpp +++ b/rclcpp/include/rclcpp/qos_event.hpp @@ -112,7 +112,7 @@ class QOSEventHandlerBase : public Waitable void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const override; + const void * user_data) override; protected: rcl_event_t event_handle_; diff --git a/rclcpp/include/rclcpp/service.hpp b/rclcpp/include/rclcpp/service.hpp index bf0471d4b5..c82337dc4d 100644 --- a/rclcpp/include/rclcpp/service.hpp +++ b/rclcpp/include/rclcpp/service.hpp @@ -130,7 +130,7 @@ class ServiceBase void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const; + const void * user_data); protected: RCLCPP_DISABLE_COPY(ServiceBase) diff --git a/rclcpp/include/rclcpp/subscription_base.hpp b/rclcpp/include/rclcpp/subscription_base.hpp index 2b5653f896..f665d18911 100644 --- a/rclcpp/include/rclcpp/subscription_base.hpp +++ b/rclcpp/include/rclcpp/subscription_base.hpp @@ -273,7 +273,7 @@ class SubscriptionBase : public std::enable_shared_from_this void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const; + const void * user_data); protected: template diff --git a/rclcpp/include/rclcpp/waitable.hpp b/rclcpp/include/rclcpp/waitable.hpp index 40c50e5e75..8c7aa901d8 100644 --- a/rclcpp/include/rclcpp/waitable.hpp +++ b/rclcpp/include/rclcpp/waitable.hpp @@ -215,7 +215,7 @@ class Waitable void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const; + const void * user_data); private: std::atomic in_use_by_wait_set_{false}; diff --git a/rclcpp/src/rclcpp/client.cpp b/rclcpp/src/rclcpp/client.cpp index c74dd59c8d..2685751793 100644 --- a/rclcpp/src/rclcpp/client.cpp +++ b/rclcpp/src/rclcpp/client.cpp @@ -202,7 +202,7 @@ ClientBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ClientBase::set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const + const void * user_data) { rcl_ret_t ret = rcl_client_set_listener_callback( client_handle_.get(), diff --git a/rclcpp/src/rclcpp/qos_event.cpp b/rclcpp/src/rclcpp/qos_event.cpp index 6a23b95c2f..e652708f5c 100644 --- a/rclcpp/src/rclcpp/qos_event.cpp +++ b/rclcpp/src/rclcpp/qos_event.cpp @@ -71,7 +71,7 @@ QOSEventHandlerBase::is_ready(rcl_wait_set_t * wait_set) void QOSEventHandlerBase::set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const + const void * user_data) { rcl_ret_t ret = rcl_event_set_listener_callback( &event_handle_, diff --git a/rclcpp/src/rclcpp/service.cpp b/rclcpp/src/rclcpp/service.cpp index e279668bdf..72fd9836dc 100644 --- a/rclcpp/src/rclcpp/service.cpp +++ b/rclcpp/src/rclcpp/service.cpp @@ -88,7 +88,7 @@ ServiceBase::exchange_in_use_by_wait_set_state(bool in_use_state) void ServiceBase::set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const + const void * user_data) { rcl_ret_t ret = rcl_service_set_listener_callback( service_handle_.get(), diff --git a/rclcpp/src/rclcpp/subscription_base.cpp b/rclcpp/src/rclcpp/subscription_base.cpp index 065de5cbbd..ea242a39d0 100644 --- a/rclcpp/src/rclcpp/subscription_base.cpp +++ b/rclcpp/src/rclcpp/subscription_base.cpp @@ -292,7 +292,7 @@ SubscriptionBase::exchange_in_use_by_wait_set_state( void SubscriptionBase::set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const + const void * user_data) { rcl_ret_t ret = rcl_subscription_set_listener_callback( subscription_handle_.get(), diff --git a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp index 65e3cbbe83..e0a9bff806 100644 --- a/rclcpp/src/rclcpp/subscription_intra_process_base.cpp +++ b/rclcpp/src/rclcpp/subscription_intra_process_base.cpp @@ -40,7 +40,7 @@ SubscriptionIntraProcessBase::get_actual_qos() const void SubscriptionIntraProcessBase::set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const + const void * user_data) { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, diff --git a/rclcpp/src/rclcpp/waitable.cpp b/rclcpp/src/rclcpp/waitable.cpp index 213fc9353a..9c50c2d98a 100644 --- a/rclcpp/src/rclcpp/waitable.cpp +++ b/rclcpp/src/rclcpp/waitable.cpp @@ -63,7 +63,7 @@ Waitable::exchange_in_use_by_wait_set_state(bool in_use_state) void Waitable::set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const + const void * user_data) { (void)callback; (void)user_data; diff --git a/rclcpp/test/rclcpp/executors/test_executors.cpp b/rclcpp/test/rclcpp/executors/test_executors.cpp index c814b6d8f3..5686567009 100644 --- a/rclcpp/test/rclcpp/executors/test_executors.cpp +++ b/rclcpp/test/rclcpp/executors/test_executors.cpp @@ -471,7 +471,7 @@ class TestWaitable : public rclcpp::Waitable void set_listener_callback( rmw_listener_callback_t callback, - const void * user_data) const override + const void * user_data) override { rcl_ret_t ret = rcl_guard_condition_set_listener_callback( &gc_, diff --git a/rclcpp_action/include/rclcpp_action/client.hpp b/rclcpp_action/include/rclcpp_action/client.hpp index e9c6bff0dc..ab1e1598a6 100644 --- a/rclcpp_action/include/rclcpp_action/client.hpp +++ b/rclcpp_action/include/rclcpp_action/client.hpp @@ -130,9 +130,9 @@ class ClientBase : public rclcpp::Waitable /// \internal RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) override; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) override; // End Waitables API // ----------------- diff --git a/rclcpp_action/include/rclcpp_action/server.hpp b/rclcpp_action/include/rclcpp_action/server.hpp index acefe86b4a..4b86d75297 100644 --- a/rclcpp_action/include/rclcpp_action/server.hpp +++ b/rclcpp_action/include/rclcpp_action/server.hpp @@ -133,9 +133,9 @@ class ServerBase : public rclcpp::Waitable /// \internal RCLCPP_PUBLIC void - set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) override; + set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) override; // End Waitables API // ----------------- diff --git a/rclcpp_action/src/client.cpp b/rclcpp_action/src/client.cpp index ec52dd3f3b..9162000daf 100644 --- a/rclcpp_action/src/client.cpp +++ b/rclcpp_action/src/client.cpp @@ -546,11 +546,11 @@ ClientBase::execute(std::shared_ptr & data) } void -ClientBase::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) +ClientBase::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) { - if ((executor_callback == nullptr) || (executor_callback_data == nullptr)) { + if ((callback == nullptr) || (user_data == nullptr)) { // Not valid arguments. Unset listeners callback data. rcl_ret_t ret = rcl_action_client_set_listeners_callback( pimpl_->client_handle.get(), @@ -564,7 +564,7 @@ ClientBase::set_events_executor_callback( return; } - auto data = static_cast(executor_callback_data); + auto data = static_cast(user_data); // Create an ExecutorEvent per action client event type rclcpp::executors::ExecutorEvent goal_response = {data->event.entity_id, nullptr, data->event.type}; @@ -611,7 +611,7 @@ ClientBase::set_events_executor_callback( // Set listener callbacks to the action client entities rcl_ret_t ret = rcl_action_client_set_listeners_callback( pimpl_->client_handle.get(), - executor_callback, + callback, action_client_cb_data); if (ret != RCL_RET_OK) { diff --git a/rclcpp_action/src/server.cpp b/rclcpp_action/src/server.cpp index 61f0955e9f..a333ed2e38 100644 --- a/rclcpp_action/src/server.cpp +++ b/rclcpp_action/src/server.cpp @@ -701,11 +701,11 @@ ServerBase::publish_feedback(std::shared_ptr feedback_msg) } void -ServerBase::set_events_executor_callback( - rmw_listener_callback_t executor_callback, - const void * executor_callback_data) +ServerBase::set_listener_callback( + rmw_listener_callback_t callback, + const void * user_data) { - if ((executor_callback == nullptr) || (executor_callback_data == nullptr)) { + if ((callback == nullptr) || (user_data == nullptr)) { // Not valid arguments. Unset listeners callback data. rcl_ret_t ret = rcl_action_server_set_listeners_callback( pimpl_->action_server_.get(), @@ -719,7 +719,7 @@ ServerBase::set_events_executor_callback( return; } - auto data = static_cast(executor_callback_data); + auto data = static_cast(user_data); // Create an ExecutorEvent per action server event type rclcpp::executors::ExecutorEvent goal_request = {data->event.entity_id, nullptr, data->event.type}; @@ -756,7 +756,7 @@ ServerBase::set_events_executor_callback( // Set listener callbacks to the action server entities rcl_ret_t ret = rcl_action_server_set_listeners_callback( pimpl_->action_server_.get(), - executor_callback, + callback, action_server_cb_data); if (ret != RCL_RET_OK) { From 3118251dd4bc4f2670b0acb33daa42ad44eb5307 Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 15 Mar 2021 15:12:10 -0300 Subject: [PATCH 167/168] time_until_trigger return MAX if canceled --- .../rclcpp/executors/timers_manager.hpp | 7 ++++++- rclcpp/include/rclcpp/timer.hpp | 12 +++--------- .../src/rclcpp/executors/timers_manager.cpp | 19 ++++++++++++++----- rclcpp/src/rclcpp/timer.cpp | 10 +++++++++- 4 files changed, 32 insertions(+), 16 deletions(-) diff --git a/rclcpp/include/rclcpp/executors/timers_manager.hpp b/rclcpp/include/rclcpp/executors/timers_manager.hpp index 2630fdaf2d..60ff56d901 100644 --- a/rclcpp/include/rclcpp/executors/timers_manager.hpp +++ b/rclcpp/include/rclcpp/executors/timers_manager.hpp @@ -448,8 +448,13 @@ class TimersManager TimerPtr timer, std::chrono::time_point tp) { + auto time_until_trigger = timer->time_until_trigger(); + // A canceled timer will return a nanoseconds::max duration + if (time_until_trigger == std::chrono::nanoseconds::max()) { + return false; + } // A ready timer will return a negative duration when calling time_until_trigger - auto time_ready = std::chrono::steady_clock::now() + timer->time_until_trigger(); + auto time_ready = std::chrono::steady_clock::now() + time_until_trigger; return time_ready < tp; } diff --git a/rclcpp/include/rclcpp/timer.hpp b/rclcpp/include/rclcpp/timer.hpp index 778976da81..a2662f4abe 100644 --- a/rclcpp/include/rclcpp/timer.hpp +++ b/rclcpp/include/rclcpp/timer.hpp @@ -92,12 +92,8 @@ class TimerBase reset(); /// Call the callback function when the timer signal is emitted. - /** - * \return true if callback was executed, false otherwise - * \throws std::runtime_error if failed to notify timer that callback occurred - */ RCLCPP_PUBLIC - virtual bool + virtual void execute_callback() = 0; RCLCPP_PUBLIC @@ -197,15 +193,14 @@ class GenericTimer : public TimerBase /** * \sa rclcpp::TimerBase::execute_callback - * \return true if callback was executed, false otherwise * \throws std::runtime_error if it failed to notify timer that callback occurred */ - bool + void execute_callback() override { rcl_ret_t ret = rcl_timer_call(timer_handle_.get()); if (ret == RCL_RET_TIMER_CANCELED) { - return false; + return; } if (ret != RCL_RET_OK) { throw std::runtime_error("Failed to notify timer that callback occurred"); @@ -213,7 +208,6 @@ class GenericTimer : public TimerBase TRACEPOINT(callback_start, static_cast(&callback_), false); execute_callback_delegate<>(); TRACEPOINT(callback_end, static_cast(&callback_)); - return true; } // void specialization diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index b4a2ef864e..b3519cae1f 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -62,7 +62,6 @@ void TimersManager::start() } timers_thread_ = std::thread(&TimersManager::run_timers, this); - pthread_setname_np(timers_thread_.native_handle(), "TimersManager"); } void TimersManager::stop() @@ -139,11 +138,10 @@ bool TimersManager::execute_head_timer( if (timer_ready) { // Head timer is ready, execute - bool callback_executed = head->execute_callback(); - + head->execute_callback(); timers_heap.heapify_root(); weak_timers_heap_.store(timers_heap); - return callback_executed; + return true; } // Head timer was not ready yet @@ -161,6 +159,11 @@ std::chrono::nanoseconds TimersManager::get_head_timeout_unsafe() TimerPtr head_timer = weak_timers_heap_.front().lock(); // If it is still a valid pointer, it is guaranteed to be the correct head if (head_timer != nullptr) { + auto time_until_trigger = head_timer->time_until_trigger(); + // A canceled timer will return a nanoseconds::max duration + if (time_until_trigger == std::chrono::nanoseconds::max()) { + return MAX_TIME; + } return head_timer->time_until_trigger(); } @@ -174,7 +177,13 @@ std::chrono::nanoseconds TimersManager::get_head_timeout_unsafe() if (locked_heap.empty()) { return MAX_TIME; } - return locked_heap.front()->time_until_trigger(); + auto time_until_trigger = locked_heap.front()->time_until_trigger(); + // A canceled timer will return a nanoseconds::max duration + if (time_until_trigger == std::chrono::nanoseconds::max()) { + return MAX_TIME; + } else { + return time_until_trigger; + } } void TimersManager::execute_ready_timers_unsafe() diff --git a/rclcpp/src/rclcpp/timer.cpp b/rclcpp/src/rclcpp/timer.cpp index 70414f38c6..c3670f1d98 100644 --- a/rclcpp/src/rclcpp/timer.cpp +++ b/rclcpp/src/rclcpp/timer.cpp @@ -116,8 +116,16 @@ TimerBase::is_ready() std::chrono::nanoseconds TimerBase::time_until_trigger() { + bool is_canceled = false; + rcl_ret_t ret = rcl_timer_is_canceled(timer_handle_.get(), &is_canceled); + if (ret != RCL_RET_OK) { + rclcpp::exceptions::throw_from_rcl_error(ret, "Couldn't get timer cancelled state"); + } + if (is_canceled) { + return std::chrono::nanoseconds::max(); + } int64_t time_until_next_call = 0; - rcl_ret_t ret = rcl_timer_get_time_until_next_call( + ret = rcl_timer_get_time_until_next_call( timer_handle_.get(), &time_until_next_call); if (ret != RCL_RET_OK) { rclcpp::exceptions::throw_from_rcl_error(ret, "Timer could not get time until next call"); From b3dd3a6d709300dd5f86ee8c8680d16be8661c0e Mon Sep 17 00:00:00 2001 From: Mauro Passerino Date: Mon, 15 Mar 2021 15:29:18 -0300 Subject: [PATCH 168/168] time_until_trigger return MAX if canceled Signed-off-by: Mauro Passerino --- rclcpp/src/rclcpp/executors/timers_manager.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/rclcpp/src/rclcpp/executors/timers_manager.cpp b/rclcpp/src/rclcpp/executors/timers_manager.cpp index b3519cae1f..5ce6dd9ed0 100644 --- a/rclcpp/src/rclcpp/executors/timers_manager.cpp +++ b/rclcpp/src/rclcpp/executors/timers_manager.cpp @@ -164,7 +164,7 @@ std::chrono::nanoseconds TimersManager::get_head_timeout_unsafe() if (time_until_trigger == std::chrono::nanoseconds::max()) { return MAX_TIME; } - return head_timer->time_until_trigger(); + return time_until_trigger; } // If the first elements has expired, we can't make other assumptions on the heap @@ -181,9 +181,8 @@ std::chrono::nanoseconds TimersManager::get_head_timeout_unsafe() // A canceled timer will return a nanoseconds::max duration if (time_until_trigger == std::chrono::nanoseconds::max()) { return MAX_TIME; - } else { - return time_until_trigger; } + return time_until_trigger; } void TimersManager::execute_ready_timers_unsafe()