refactor init to allow for non-global init (#587)
* refactor init to allow for non-global init Signed-off-by: William Woodall <william@osrfoundation.org> * Update rclcpp/include/rclcpp/utilities.hpp Co-Authored-By: wjwwood <william+github@osrfoundation.org> * Update rclcpp/include/rclcpp/utilities.hpp Co-Authored-By: wjwwood <william+github@osrfoundation.org> * Update rclcpp/include/rclcpp/utilities.hpp Co-Authored-By: wjwwood <william+github@osrfoundation.org> * Update rclcpp/src/rclcpp/utilities.cpp Co-Authored-By: wjwwood <william+github@osrfoundation.org> * refactor state into context objects and fix signal handling Signed-off-by: William Woodall <william@osrfoundation.org> * avoid nullptr access in error messages Signed-off-by: William Woodall <william@osrfoundation.org> * avoid exception in publish after shutdown was called Signed-off-by: William Woodall <william@osrfoundation.org> * fix missing and unused headers Signed-off-by: William Woodall <william@osrfoundation.org> * cpplint Signed-off-by: William Woodall <william@osrfoundation.org> * fixes found during testing Signed-off-by: William Woodall <william@osrfoundation.org> * address bug found in review comment Signed-off-by: William Woodall <william@osrfoundation.org> * fixes and warnings fixed during testing Signed-off-by: William Woodall <william@osrfoundation.org> * addressing review comments Signed-off-by: William Woodall <william@osrfoundation.org> * ensure new ExecutorArgs are used everywhere
This commit is contained in:
parent
36262a5cf5
commit
3af8d2cfed
38 changed files with 1077 additions and 196 deletions
|
@ -40,6 +40,7 @@ set(${PROJECT_NAME}_SRCS
|
|||
src/rclcpp/executors/multi_threaded_executor.cpp
|
||||
src/rclcpp/executors/single_threaded_executor.cpp
|
||||
src/rclcpp/graph_listener.cpp
|
||||
src/rclcpp/init_options.cpp
|
||||
src/rclcpp/intra_process_manager.cpp
|
||||
src/rclcpp/intra_process_manager_impl.cpp
|
||||
src/rclcpp/logger.cpp
|
||||
|
|
|
@ -110,6 +110,7 @@ protected:
|
|||
|
||||
rclcpp::node_interfaces::NodeGraphInterface::WeakPtr node_graph_;
|
||||
std::shared_ptr<rcl_node_t> node_handle_;
|
||||
std::shared_ptr<rclcpp::Context> context_;
|
||||
|
||||
std::shared_ptr<rcl_client_t> client_handle_;
|
||||
};
|
||||
|
@ -171,13 +172,13 @@ public:
|
|||
}
|
||||
|
||||
std::shared_ptr<void>
|
||||
create_response()
|
||||
create_response() override
|
||||
{
|
||||
return std::shared_ptr<void>(new typename ServiceT::Response());
|
||||
}
|
||||
|
||||
std::shared_ptr<rmw_request_id_t>
|
||||
create_request_header()
|
||||
create_request_header() override
|
||||
{
|
||||
// TODO(wjwwood): This should probably use rmw_request_id's allocator.
|
||||
// (since it is a C type)
|
||||
|
@ -187,7 +188,7 @@ public:
|
|||
void
|
||||
handle_response(
|
||||
std::shared_ptr<rmw_request_id_t> request_header,
|
||||
std::shared_ptr<void> response)
|
||||
std::shared_ptr<void> response) override
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(pending_requests_mutex_);
|
||||
auto typed_response = std::static_pointer_cast<typename ServiceT::Response>(response);
|
||||
|
|
|
@ -15,34 +15,208 @@
|
|||
#ifndef RCLCPP__CONTEXT_HPP_
|
||||
#define RCLCPP__CONTEXT_HPP_
|
||||
|
||||
#include <iostream>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <string>
|
||||
#include <typeindex>
|
||||
#include <typeinfo>
|
||||
#include <unordered_map>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
#include "rcl/context.h"
|
||||
#include "rclcpp/init_options.hpp"
|
||||
#include "rclcpp/macros.hpp"
|
||||
#include "rclcpp/visibility_control.hpp"
|
||||
#include "rmw/rmw.h"
|
||||
|
||||
namespace rclcpp
|
||||
{
|
||||
|
||||
class Context
|
||||
/// Thrown when init is called on an already initialized context.
|
||||
class ContextAlreadyInitialized : public std::runtime_error
|
||||
{
|
||||
public:
|
||||
ContextAlreadyInitialized()
|
||||
: std::runtime_error("context is already initialized") {}
|
||||
};
|
||||
|
||||
/// Context which encapsulates shared state between nodes and other similar entities.
|
||||
/**
|
||||
* A context also represents the lifecycle between init and shutdown of rclcpp.
|
||||
* It is often used in conjunction with rclcpp::init, or rclcpp::init_local,
|
||||
* and rclcpp::shutdown.
|
||||
*/
|
||||
class Context : public std::enable_shared_from_this<Context>
|
||||
{
|
||||
public:
|
||||
RCLCPP_SMART_PTR_DEFINITIONS(Context)
|
||||
|
||||
/// Default constructor, after which the Context is still not "initialized".
|
||||
/**
|
||||
* Every context which is constructed is added to a global vector of contexts,
|
||||
* which is used by the signal handler to conditionally shutdown each context
|
||||
* on SIGINT.
|
||||
* See the shutdown_on_sigint option in the InitOptions class.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
Context();
|
||||
|
||||
RCLCPP_PUBLIC
|
||||
virtual
|
||||
~Context();
|
||||
|
||||
/// Initialize the context, and the underlying elements like the rcl context.
|
||||
/**
|
||||
* This method must be called before passing this context to things like the
|
||||
* constructor of Node.
|
||||
* It must also be called before trying to shutdown the context.
|
||||
*
|
||||
* Note that this function does not setup any signal handlers, so if you want
|
||||
* it to be shutdown by the signal handler, then you need to either install
|
||||
* them manually with rclcpp::install_signal_handers() or use rclcpp::init().
|
||||
* In addition to installing the signal handlers, the shutdown_on_sigint
|
||||
* of the InitOptions needs to be `true` for this context to be shutdown by
|
||||
* the signal handler, otherwise it will be passed over.
|
||||
*
|
||||
* After calling this method, shutdown() can be called to invalidate the
|
||||
* context for derived entities, e.g. nodes, guard conditions, etc.
|
||||
* However, the underlying rcl context is not finalized until this Context's
|
||||
* destructor is called or this function is called again.
|
||||
* Allowing this class to go out of scope and get destructed or calling this
|
||||
* function a second time while derived entities are still using the context
|
||||
* is undefined behavior and should be avoided.
|
||||
* It's a good idea to not reuse context objects and instead create a new one
|
||||
* each time you need to shutdown and init again.
|
||||
* This allows derived entities to hold on to shard pointers to the first
|
||||
* context object until they are done.
|
||||
*
|
||||
* This function is thread-safe.
|
||||
*
|
||||
* \param[in] argc number of arguments
|
||||
* \param[in] argv argument array which may contain arguments intended for ROS
|
||||
* \param[in] init_options initialization options for rclcpp and underlying layers
|
||||
* \throw ContextAlreadyInitialized if called if init is called more than once
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
virtual
|
||||
void
|
||||
init(
|
||||
int argc,
|
||||
char const * const argv[],
|
||||
const rclcpp::InitOptions & init_options = rclcpp::InitOptions());
|
||||
|
||||
/// Return true if the context is valid, otherwise false.
|
||||
/**
|
||||
* The context is valid if it has been initialized but not shutdown.
|
||||
*
|
||||
* This function is thread-safe.
|
||||
* This function is lock free so long as pointers and uint64_t atomics are
|
||||
* lock free.
|
||||
*
|
||||
* \return true if valid, otherwise false
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
is_valid() const;
|
||||
|
||||
/// Return the init options used during init.
|
||||
RCLCPP_PUBLIC
|
||||
const rclcpp::InitOptions &
|
||||
get_init_options() const;
|
||||
|
||||
/// Return a copy of the init options used during init.
|
||||
RCLCPP_PUBLIC
|
||||
rclcpp::InitOptions
|
||||
get_init_options();
|
||||
|
||||
/// Return the shutdown reason, or empty string if not shutdown.
|
||||
/**
|
||||
* This function is thread-safe.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
std::string
|
||||
shutdown_reason();
|
||||
|
||||
/// Shutdown the context, making it uninitialized and therefore invalid for derived entities.
|
||||
/**
|
||||
* Several things happen when the context is shutdown, in this order:
|
||||
*
|
||||
* - acquires a lock to prevent race conditions with init, on_shutdown, etc.
|
||||
* - if the context is not initialized, return false
|
||||
* - rcl_shutdown() is called on the internal rcl_context_t instance
|
||||
* - the shutdown reason is set
|
||||
* - each on_shutdown callback is called, in the order that they were added
|
||||
* - if notify_all is true, rclcpp::notify_all is called to unblock some ROS functions
|
||||
*
|
||||
* The underlying rcl context is not finalized by this function.
|
||||
*
|
||||
* This function is thread-safe.
|
||||
*
|
||||
* \param[in] reason the description of why shutdown happened
|
||||
* \param[in] notify_all if true, then rclcpp::notify_all will be called
|
||||
* \return true if shutdown was successful, false if context was already shutdown
|
||||
* \throw various exceptions derived from RCLErrorBase, if rcl_shutdown fails
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
virtual
|
||||
bool
|
||||
shutdown(const std::string & reason, bool notify_all = true);
|
||||
|
||||
using OnShutdownCallback = std::function<void ()>;
|
||||
|
||||
/// Add a on_shutdown callback to be called when shutdown is called for this context.
|
||||
/**
|
||||
* These callbacks will be called in the order they are added as the second
|
||||
* to last step in shutdown().
|
||||
*
|
||||
* These callbacks may be run in the signal handler as the signal handler
|
||||
* may call shutdown() on this context.
|
||||
* Also, shutdown() may be called from the destructor of this function.
|
||||
* Therefore, it is not safe to throw exceptions from these callbacks.
|
||||
* Instead, log errors or use some other mechanism to indicate an error has
|
||||
* occurred.
|
||||
*
|
||||
* On shutdown callbacks may be registered before init and after shutdown,
|
||||
* and persist on repeated init's.
|
||||
*
|
||||
* \param[in] callback the on shutdown callback to be registered
|
||||
* \return the callback passed, for convenience when storing a passed lambda
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
virtual
|
||||
OnShutdownCallback
|
||||
on_shutdown(OnShutdownCallback callback);
|
||||
|
||||
/// Return the shutdown callbacks as const.
|
||||
/**
|
||||
* Using the returned reference is not thread-safe with calls that modify
|
||||
* the list of "on shutdown" callbacks, i.e. on_shutdown().
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
const std::vector<OnShutdownCallback> &
|
||||
get_on_shutdown_callbacks() const;
|
||||
|
||||
/// Return the shutdown callbacks.
|
||||
/**
|
||||
* Using the returned reference is not thread-safe with calls that modify
|
||||
* the list of "on shutdown" callbacks, i.e. on_shutdown().
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
std::vector<OnShutdownCallback> &
|
||||
get_on_shutdown_callbacks();
|
||||
|
||||
/// Return the internal rcl context.
|
||||
RCLCPP_PUBLIC
|
||||
std::shared_ptr<rcl_context_t>
|
||||
get_rcl_context();
|
||||
|
||||
/// Return a singleton instance for the SubContext type, constructing one if necessary.
|
||||
template<typename SubContext, typename ... Args>
|
||||
std::shared_ptr<SubContext>
|
||||
get_sub_context(Args && ... args)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex_);
|
||||
std::lock_guard<std::recursive_mutex> lock(sub_contexts_mutex_);
|
||||
|
||||
std::type_index type_i(typeid(SubContext));
|
||||
std::shared_ptr<SubContext> sub_context;
|
||||
|
@ -62,13 +236,41 @@ public:
|
|||
return sub_context;
|
||||
}
|
||||
|
||||
protected:
|
||||
// Called by constructor and destructor to clean up by finalizing the
|
||||
// shutdown rcl context and preparing for a new init cycle.
|
||||
RCLCPP_PUBLIC
|
||||
virtual
|
||||
void
|
||||
clean_up();
|
||||
|
||||
private:
|
||||
RCLCPP_DISABLE_COPY(Context)
|
||||
|
||||
// This mutex is recursive so that the destructor can ensure atomicity
|
||||
// between is_initialized and shutdown.
|
||||
std::recursive_mutex init_mutex_;
|
||||
std::shared_ptr<rcl_context_t> rcl_context_;
|
||||
rclcpp::InitOptions init_options_;
|
||||
std::string shutdown_reason_;
|
||||
|
||||
std::unordered_map<std::type_index, std::shared_ptr<void>> sub_contexts_;
|
||||
std::mutex mutex_;
|
||||
// This mutex is recursive so that the constructor of a sub context may
|
||||
// attempt to acquire another sub context.
|
||||
std::recursive_mutex sub_contexts_mutex_;
|
||||
|
||||
std::vector<OnShutdownCallback> on_shutdown_callbacks_;
|
||||
std::mutex on_shutdown_callbacks_mutex_;
|
||||
};
|
||||
|
||||
/// Return a copy of the list of context shared pointers.
|
||||
/**
|
||||
* This function is thread-safe.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
std::vector<Context::SharedPtr>
|
||||
get_contexts();
|
||||
|
||||
} // namespace rclcpp
|
||||
|
||||
#endif // RCLCPP__CONTEXT_HPP_
|
||||
|
|
|
@ -28,9 +28,10 @@
|
|||
#include "rcl/guard_condition.h"
|
||||
#include "rcl/wait.h"
|
||||
|
||||
#include "rclcpp/node_interfaces/node_base_interface.hpp"
|
||||
#include "rclcpp/contexts/default_context.hpp"
|
||||
#include "rclcpp/memory_strategies.hpp"
|
||||
#include "rclcpp/memory_strategy.hpp"
|
||||
#include "rclcpp/node_interfaces/node_base_interface.hpp"
|
||||
#include "rclcpp/utilities.hpp"
|
||||
#include "rclcpp/visibility_control.hpp"
|
||||
|
||||
|
@ -65,16 +66,20 @@ to_string(const FutureReturnCode & future_return_code);
|
|||
*/
|
||||
struct ExecutorArgs
|
||||
{
|
||||
ExecutorArgs()
|
||||
: memory_strategy(memory_strategies::create_default_strategy()),
|
||||
context(rclcpp::contexts::default_context::get_global_default_context()),
|
||||
max_conditions(0)
|
||||
{}
|
||||
|
||||
memory_strategy::MemoryStrategy::SharedPtr memory_strategy;
|
||||
size_t max_conditions = 0;
|
||||
std::shared_ptr<rclcpp::Context> context;
|
||||
size_t max_conditions;
|
||||
};
|
||||
|
||||
static inline ExecutorArgs create_default_executor_arguments()
|
||||
{
|
||||
ExecutorArgs args;
|
||||
args.memory_strategy = memory_strategies::create_default_strategy();
|
||||
args.max_conditions = 0;
|
||||
return args;
|
||||
return ExecutorArgs();
|
||||
}
|
||||
|
||||
/// Coordinate the order and timing of available communication tasks.
|
||||
|
@ -95,7 +100,7 @@ public:
|
|||
/// Default constructor.
|
||||
// \param[in] ms The memory strategy to be used with this executor.
|
||||
RCLCPP_PUBLIC
|
||||
explicit Executor(const ExecutorArgs & args = create_default_executor_arguments());
|
||||
explicit Executor(const ExecutorArgs & args = ExecutorArgs());
|
||||
|
||||
/// Default destructor.
|
||||
RCLCPP_PUBLIC
|
||||
|
@ -237,7 +242,7 @@ public:
|
|||
}
|
||||
std::chrono::nanoseconds timeout_left = timeout_ns;
|
||||
|
||||
while (rclcpp::ok()) {
|
||||
while (rclcpp::ok(this->context_)) {
|
||||
// Do one item of work.
|
||||
spin_once(timeout_left);
|
||||
// Check if the future is set, return SUCCESS if it is.
|
||||
|
@ -353,6 +358,9 @@ protected:
|
|||
/// The memory strategy: an interface for handling user-defined memory allocation strategies.
|
||||
memory_strategy::MemoryStrategy::SharedPtr memory_strategy_;
|
||||
|
||||
/// The context associated with this executor.
|
||||
std::shared_ptr<rclcpp::Context> context_;
|
||||
|
||||
private:
|
||||
RCLCPP_DISABLE_COPY(Executor)
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ public:
|
|||
*/
|
||||
RCLCPP_PUBLIC
|
||||
MultiThreadedExecutor(
|
||||
const executor::ExecutorArgs & args = rclcpp::executor::create_default_executor_arguments(),
|
||||
const executor::ExecutorArgs & args = executor::ExecutorArgs(),
|
||||
size_t number_of_threads = 0,
|
||||
bool yield_before_execute = false);
|
||||
|
||||
|
|
|
@ -45,7 +45,7 @@ public:
|
|||
/// Default constructor. See the default constructor for Executor.
|
||||
RCLCPP_PUBLIC
|
||||
SingleThreadedExecutor(
|
||||
const executor::ExecutorArgs & args = rclcpp::executor::create_default_executor_arguments());
|
||||
const executor::ExecutorArgs & args = executor::ExecutorArgs());
|
||||
|
||||
/// Default destrcutor.
|
||||
RCLCPP_PUBLIC
|
||||
|
|
|
@ -23,6 +23,7 @@
|
|||
|
||||
#include "rcl/guard_condition.h"
|
||||
#include "rcl/wait.h"
|
||||
#include "rclcpp/context.hpp"
|
||||
#include "rclcpp/macros.hpp"
|
||||
#include "rclcpp/node_interfaces/node_graph_interface.hpp"
|
||||
#include "rclcpp/visibility_control.hpp"
|
||||
|
@ -62,7 +63,7 @@ class GraphListener : public std::enable_shared_from_this<GraphListener>
|
|||
{
|
||||
public:
|
||||
RCLCPP_PUBLIC
|
||||
GraphListener();
|
||||
explicit GraphListener(std::shared_ptr<rclcpp::Context> parent_context);
|
||||
|
||||
RCLCPP_PUBLIC
|
||||
virtual ~GraphListener();
|
||||
|
@ -164,6 +165,7 @@ private:
|
|||
std::vector<rclcpp::node_interfaces::NodeGraphInterface *> node_graph_interfaces_;
|
||||
|
||||
rcl_guard_condition_t interrupt_guard_condition_ = rcl_get_zero_initialized_guard_condition();
|
||||
std::shared_ptr<rcl_context_t> interrupt_guard_condition_context_;
|
||||
rcl_guard_condition_t * shutdown_guard_condition_;
|
||||
rcl_wait_set_t wait_set_ = rcl_get_zero_initialized_wait_set();
|
||||
};
|
||||
|
|
69
rclcpp/include/rclcpp/init_options.hpp
Normal file
69
rclcpp/include/rclcpp/init_options.hpp
Normal file
|
@ -0,0 +1,69 @@
|
|||
// Copyright 2018 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__INIT_OPTIONS_HPP_
|
||||
#define RCLCPP__INIT_OPTIONS_HPP_
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include "rcl/init_options.h"
|
||||
#include "rclcpp/visibility_control.hpp"
|
||||
|
||||
namespace rclcpp
|
||||
{
|
||||
|
||||
/// Encapsulation of options for initializing rclcpp.
|
||||
class InitOptions
|
||||
{
|
||||
public:
|
||||
/// If true, the context will be shutdown on SIGINT by the signal handler (if it was installed).
|
||||
bool shutdown_on_sigint = true;
|
||||
|
||||
/// Constructor which allows you to specify the allocator used within the init options.
|
||||
RCLCPP_PUBLIC
|
||||
explicit InitOptions(rcl_allocator_t allocator = rcl_get_default_allocator());
|
||||
|
||||
/// Constructor which is initialized by an existing init_options.
|
||||
RCLCPP_PUBLIC
|
||||
explicit InitOptions(const rcl_init_options_t & init_options);
|
||||
|
||||
/// Copy constructor.
|
||||
RCLCPP_PUBLIC
|
||||
InitOptions(const InitOptions & other);
|
||||
|
||||
/// Assignment operator.
|
||||
RCLCPP_PUBLIC
|
||||
InitOptions &
|
||||
operator=(const InitOptions & other);
|
||||
|
||||
RCLCPP_PUBLIC
|
||||
virtual
|
||||
~InitOptions();
|
||||
|
||||
/// Return the rcl init options.
|
||||
RCLCPP_PUBLIC
|
||||
const rcl_init_options_t *
|
||||
get_rcl_init_options() const;
|
||||
|
||||
protected:
|
||||
void
|
||||
finalize_init_options();
|
||||
|
||||
private:
|
||||
std::unique_ptr<rcl_init_options_t> init_options_;
|
||||
};
|
||||
|
||||
} // namespace rclcpp
|
||||
|
||||
#endif // RCLCPP__INIT_OPTIONS_HPP_
|
|
@ -20,6 +20,8 @@
|
|||
|
||||
#include "rclcpp/visibility_control.hpp"
|
||||
|
||||
#include "rcl/node.h"
|
||||
|
||||
/**
|
||||
* \def RCLCPP_LOGGING_ENABLED
|
||||
* When this define evaluates to true (default), logger factory functions will
|
||||
|
@ -60,6 +62,18 @@ RCLCPP_PUBLIC
|
|||
Logger
|
||||
get_logger(const std::string & name);
|
||||
|
||||
/// Return a named logger using an rcl_node_t.
|
||||
/**
|
||||
* This is a convenience function that does error checking and returns the node
|
||||
* logger name, or "rclcpp" if it is unable to get the node name.
|
||||
*
|
||||
* \param[in] node the rcl node from which to get the logger name
|
||||
* \return a logger based on the node name, or "rclcpp" if there's an error
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
Logger
|
||||
get_node_logger(const rcl_node_t * node);
|
||||
|
||||
class Logger
|
||||
{
|
||||
private:
|
||||
|
|
|
@ -428,6 +428,11 @@ public:
|
|||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr
|
||||
get_node_graph_interface();
|
||||
|
||||
/// Return the Node's internal NodeLoggingInterface implementation.
|
||||
RCLCPP_PUBLIC
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr
|
||||
get_node_logging_interface();
|
||||
|
||||
/// Return the Node's internal NodeTimersInterface implementation.
|
||||
RCLCPP_PUBLIC
|
||||
rclcpp::node_interfaces::NodeTimersInterface::SharedPtr
|
||||
|
|
|
@ -160,7 +160,8 @@ Node::create_wall_timer(
|
|||
{
|
||||
auto timer = rclcpp::WallTimer<CallbackT>::make_shared(
|
||||
std::chrono::duration_cast<std::chrono::nanoseconds>(period),
|
||||
std::move(callback));
|
||||
std::move(callback),
|
||||
this->node_base_->get_context());
|
||||
node_timers_->add_timer(timer, group);
|
||||
return timer;
|
||||
}
|
||||
|
|
|
@ -41,7 +41,8 @@ public:
|
|||
rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_base,
|
||||
rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics,
|
||||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph,
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services);
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services,
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr node_logging);
|
||||
|
||||
RCLCPP_PUBLIC
|
||||
virtual
|
||||
|
@ -60,6 +61,7 @@ private:
|
|||
rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics_;
|
||||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph_;
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services_;
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr node_logging_;
|
||||
|
||||
rclcpp::Clock::SharedPtr ros_clock_;
|
||||
rclcpp::TimeSource time_source_;
|
||||
|
|
|
@ -205,6 +205,16 @@ public:
|
|||
ipm.publisher_id = intra_process_publisher_id_;
|
||||
ipm.message_sequence = message_seq;
|
||||
auto status = rcl_publish(&intra_process_publisher_handle_, &ipm);
|
||||
if (RCL_RET_PUBLISHER_INVALID == status) {
|
||||
rcl_reset_error(); // next call will reset error message if not context
|
||||
if (rcl_publisher_is_valid_except_context(&intra_process_publisher_handle_)) {
|
||||
rcl_context_t * context = rcl_publisher_get_context(&intra_process_publisher_handle_);
|
||||
if (nullptr != context && !rcl_context_is_valid(context)) {
|
||||
// publisher is invalid due to context being shutdown
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (RCL_RET_OK != status) {
|
||||
rclcpp::exceptions::throw_from_rcl_error(status, "failed to publish intra process message");
|
||||
}
|
||||
|
@ -305,6 +315,16 @@ protected:
|
|||
do_inter_process_publish(const MessageT * msg)
|
||||
{
|
||||
auto status = rcl_publish(&publisher_handle_, msg);
|
||||
if (RCL_RET_PUBLISHER_INVALID == status) {
|
||||
rcl_reset_error(); // next call will reset error message if not context
|
||||
if (rcl_publisher_is_valid_except_context(&publisher_handle_)) {
|
||||
rcl_context_t * context = rcl_publisher_get_context(&publisher_handle_);
|
||||
if (nullptr != context && !rcl_context_is_valid(context)) {
|
||||
// publisher is invalid due to context being shutdown
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (RCL_RET_OK != status) {
|
||||
rclcpp::exceptions::throw_from_rcl_error(status, "failed to publish message");
|
||||
}
|
||||
|
|
|
@ -119,7 +119,7 @@ public:
|
|||
if (handle) {
|
||||
if (rcl_service_fini(service, handle.get()) != RCL_RET_OK) {
|
||||
RCLCPP_ERROR(
|
||||
rclcpp::get_logger(rcl_node_get_logger_name(handle.get())).get_child("rclcpp"),
|
||||
rclcpp::get_node_logger(handle.get()).get_child("rclcpp"),
|
||||
"Error in destruction of rcl service handle: %s",
|
||||
rcl_get_error_string().str);
|
||||
rcl_reset_error();
|
||||
|
|
|
@ -47,10 +47,11 @@ public:
|
|||
|
||||
RCLCPP_PUBLIC
|
||||
void attachNode(
|
||||
const rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_base_interface,
|
||||
const rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics_interface,
|
||||
const rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph_interface,
|
||||
const rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services_interface);
|
||||
rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_base_interface,
|
||||
rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics_interface,
|
||||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph_interface,
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services_interface,
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr node_logging_interface);
|
||||
|
||||
RCLCPP_PUBLIC
|
||||
void detachNode();
|
||||
|
@ -74,6 +75,10 @@ private:
|
|||
rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics_;
|
||||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph_;
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services_;
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr node_logging_;
|
||||
|
||||
// Store (and update on node attach) logger for logging.
|
||||
Logger logger_;
|
||||
|
||||
// The subscription for the clock callback
|
||||
using MessageT = rosgraph_msgs::msg::Clock;
|
||||
|
|
|
@ -24,6 +24,7 @@
|
|||
#include <utility>
|
||||
|
||||
#include "rclcpp/clock.hpp"
|
||||
#include "rclcpp/context.hpp"
|
||||
#include "rclcpp/function_traits.hpp"
|
||||
#include "rclcpp/macros.hpp"
|
||||
#include "rclcpp/rate.hpp"
|
||||
|
@ -45,7 +46,10 @@ public:
|
|||
RCLCPP_SMART_PTR_DEFINITIONS_NOT_COPYABLE(TimerBase)
|
||||
|
||||
RCLCPP_PUBLIC
|
||||
explicit TimerBase(Clock::SharedPtr clock, std::chrono::nanoseconds period);
|
||||
explicit TimerBase(
|
||||
Clock::SharedPtr clock,
|
||||
std::chrono::nanoseconds period,
|
||||
rclcpp::Context::SharedPtr context);
|
||||
|
||||
RCLCPP_PUBLIC
|
||||
~TimerBase();
|
||||
|
@ -114,9 +118,10 @@ public:
|
|||
* \param[in] callback User-specified callback function.
|
||||
*/
|
||||
explicit GenericTimer(
|
||||
Clock::SharedPtr clock, std::chrono::nanoseconds period, FunctorT && callback
|
||||
Clock::SharedPtr clock, std::chrono::nanoseconds period, FunctorT && callback,
|
||||
rclcpp::Context::SharedPtr context
|
||||
)
|
||||
: TimerBase(clock, period), callback_(std::forward<FunctorT>(callback))
|
||||
: TimerBase(clock, period, context), callback_(std::forward<FunctorT>(callback))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -128,7 +133,7 @@ public:
|
|||
}
|
||||
|
||||
void
|
||||
execute_callback()
|
||||
execute_callback() override
|
||||
{
|
||||
rcl_ret_t ret = rcl_timer_call(timer_handle_.get());
|
||||
if (ret == RCL_RET_TIMER_CANCELED) {
|
||||
|
@ -165,8 +170,8 @@ public:
|
|||
callback_(*this);
|
||||
}
|
||||
|
||||
virtual bool
|
||||
is_steady()
|
||||
bool
|
||||
is_steady() override
|
||||
{
|
||||
return clock_->get_clock_type() == RCL_STEADY_TIME;
|
||||
}
|
||||
|
@ -189,9 +194,12 @@ class WallTimer : public GenericTimer<FunctorT>
|
|||
public:
|
||||
RCLCPP_SMART_PTR_DEFINITIONS(WallTimer)
|
||||
|
||||
explicit WallTimer(std::chrono::nanoseconds period, FunctorT && callback)
|
||||
WallTimer(
|
||||
std::chrono::nanoseconds period,
|
||||
FunctorT && callback,
|
||||
rclcpp::Context::SharedPtr context)
|
||||
: GenericTimer<FunctorT>(
|
||||
std::make_shared<Clock>(RCL_STEADY_TIME), period, std::move(callback))
|
||||
std::make_shared<Clock>(RCL_STEADY_TIME), period, std::move(callback), context)
|
||||
{}
|
||||
|
||||
protected:
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
#include <limits>
|
||||
#include <vector>
|
||||
|
||||
#include "rclcpp/context.hpp"
|
||||
#include "rclcpp/init_options.hpp"
|
||||
#include "rclcpp/visibility_control.hpp"
|
||||
|
||||
#include "rcl/guard_condition.h"
|
||||
|
@ -48,28 +50,75 @@ namespace rclcpp
|
|||
{
|
||||
/// Initialize communications via the rmw implementation and set up a global signal handler.
|
||||
/**
|
||||
* \param[in] argc Number of arguments.
|
||||
* \param[in] argv Argument vector. Will eventually be used for passing options to rclcpp.
|
||||
* Initializes the global context which is accessible via the function
|
||||
* rclcpp::contexts::default_context::get_global_default_context().
|
||||
* Also, installs the global signal handlers with the function
|
||||
* rclcpp::install_signal_handlers().
|
||||
*
|
||||
* \sa rclcpp::Context::init() for more details on arguments and possible exceptions
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
void
|
||||
init(int argc, char const * const argv[]);
|
||||
init(int argc, char const * const argv[], const InitOptions & init_options = InitOptions());
|
||||
|
||||
/// Install the global signal handler for rclcpp.
|
||||
/**
|
||||
* This function should only need to be run one time per process.
|
||||
* It is implicitly run by rclcpp::init(), and therefore this function does not
|
||||
* need to be run manually if rclcpp::init() has already been run.
|
||||
*
|
||||
* The signal handler will shutdown all initialized context.
|
||||
* It will also interrupt any blocking functions in ROS allowing them react to
|
||||
* any changes in the state of the system (like shutdown).
|
||||
*
|
||||
* This function is thread-safe.
|
||||
*
|
||||
* \return true if signal handler was installed by this function, false if already installed.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
install_signal_handlers();
|
||||
|
||||
/// Return true if the signal handlers are installed, otherwise false.
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
signal_handlers_installed();
|
||||
|
||||
/// Uninstall the global signal handler for rclcpp.
|
||||
/**
|
||||
* This function does not necessarily need to be called, but can be used to
|
||||
* undo what rclcpp::install_signal_handlers() or rclcpp::init() do with
|
||||
* respect to signal handling.
|
||||
* If you choose to use it, this function only needs to be run one time.
|
||||
* It is implicitly run by rclcpp::shutdown(), and therefore this function does
|
||||
* not need to be run manually if rclcpp::shutdown() has already been run.
|
||||
*
|
||||
* This function is thread-safe.
|
||||
*
|
||||
* \return true if signal handler was uninstalled by this function, false if was not installed.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
uninstall_signal_handlers();
|
||||
|
||||
/// Initialize communications via the rmw implementation and set up a global signal handler.
|
||||
/**
|
||||
* Additionally removes ROS-specific arguments from the argument vector.
|
||||
* \param[in] argc Number of arguments.
|
||||
* \param[in] argv Argument vector.
|
||||
*
|
||||
* \sa rclcpp::Context::init() for more details on arguments and possible exceptions
|
||||
* \returns Members of the argument vector that are not ROS arguments.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
std::vector<std::string>
|
||||
init_and_remove_ros_arguments(int argc, char const * const argv[]);
|
||||
init_and_remove_ros_arguments(
|
||||
int argc,
|
||||
char const * const argv[],
|
||||
const InitOptions & init_options = InitOptions());
|
||||
|
||||
/// Remove ROS-specific arguments from argument vector.
|
||||
/**
|
||||
* Some arguments may not have been intended as ROS arguments.
|
||||
* This function populates a the aruments in a vector.
|
||||
* This function populates the arguments in a vector.
|
||||
* Since the first argument is always assumed to be a process name, the vector
|
||||
* will always contain the process name.
|
||||
*
|
||||
|
@ -82,27 +131,70 @@ std::vector<std::string>
|
|||
remove_ros_arguments(int argc, char const * const argv[]);
|
||||
|
||||
/// Check rclcpp's status.
|
||||
/** \return True if SIGINT hasn't fired yet, false otherwise. */
|
||||
/**
|
||||
* This may return false for a context which has been shutdown, or for a
|
||||
* context that was shutdown due to SIGINT being received by the rclcpp signal
|
||||
* handler.
|
||||
*
|
||||
* If nullptr is given for the context, then the global context is used, i.e.
|
||||
* the context initialized by rclcpp::init().
|
||||
*
|
||||
* \param[in] context Check for shutdown of this Context.
|
||||
* \return true if shutdown has been called, false otherwise
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
ok();
|
||||
ok(rclcpp::Context::SharedPtr context = nullptr);
|
||||
|
||||
/// Returns true if init() has already been called.
|
||||
/** \return True if init() has been called, false otherwise. */
|
||||
/// Return true if init() has already been called for the given context.
|
||||
/**
|
||||
* If nullptr is given for the context, then the global context is used, i.e.
|
||||
* the context initialized by rclcpp::init().
|
||||
*
|
||||
* Deprecated, as it is no longer different from rcl_ok().
|
||||
*
|
||||
* \param[in] context Check for initialization of this Context.
|
||||
* \return true if the context is initialized, and false otherwise
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
is_initialized();
|
||||
is_initialized(rclcpp::Context::SharedPtr context = nullptr);
|
||||
|
||||
/// Notify the signal handler and rmw that rclcpp is shutting down.
|
||||
/// Shutdown rclcpp context, invalidating it for derived entities.
|
||||
/**
|
||||
* If nullptr is given for the context, then the global context is used, i.e.
|
||||
* the context initialized by rclcpp::init().
|
||||
*
|
||||
* If the global context is used, then the signal handlers are also uninstalled.
|
||||
*
|
||||
* This will also cause the "on_shutdown" callbacks to be called.
|
||||
*
|
||||
* \sa rclcpp::Context::shutdown()
|
||||
* \param[in] context to be shutdown
|
||||
* \return true if shutdown was successful, false if context was already shutdown
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
shutdown(
|
||||
rclcpp::Context::SharedPtr context = nullptr,
|
||||
const std::string & reason = "user called rclcpp::shutdown()");
|
||||
|
||||
/// Register a function to be called when shutdown is called on the context.
|
||||
/**
|
||||
* If nullptr is given for the context, then the global context is used, i.e.
|
||||
* the context initialized by rclcpp::init().
|
||||
*
|
||||
* Note that these callbacks should be non-blocking and not throw exceptions,
|
||||
* as they may be called from signal handlers and from the destructor of the
|
||||
* context.
|
||||
*
|
||||
* \sa rclcpp::Context::on_shutdown()
|
||||
* \param[in] callback to be called when the given context is shutdown
|
||||
* \param[in] context with which to associate the context
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
void
|
||||
shutdown();
|
||||
|
||||
/// Register a function to be called when shutdown is called.
|
||||
/** Calling the callbacks is the last thing shutdown() does. */
|
||||
RCLCPP_PUBLIC
|
||||
void
|
||||
on_shutdown(std::function<void(void)> callback);
|
||||
on_shutdown(std::function<void()> callback, rclcpp::Context::SharedPtr context = nullptr);
|
||||
|
||||
/// Get a handle to the rmw guard condition that manages the signal handler.
|
||||
/**
|
||||
|
@ -112,22 +204,29 @@ on_shutdown(std::function<void(void)> callback);
|
|||
* that the same guard condition is not reused across wait sets (e.g., when
|
||||
* using multiple executors in the same process). Will throw an exception if
|
||||
* initialization of the guard condition fails.
|
||||
* \param wait_set Pointer to the rcl_wait_set_t that will be using the
|
||||
* resulting guard condition.
|
||||
*
|
||||
* \param[in] wait_set Pointer to the rcl_wait_set_t that will be using the
|
||||
* resulting guard condition.
|
||||
* \param[in] context The context associated with the guard condition.
|
||||
* \return Pointer to the guard condition.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
rcl_guard_condition_t *
|
||||
get_sigint_guard_condition(rcl_wait_set_t * wait_set);
|
||||
get_sigint_guard_condition(rcl_wait_set_t * wait_set, rclcpp::Context::SharedPtr context);
|
||||
|
||||
/// Release the previously allocated guard condition that manages the signal handler.
|
||||
/**
|
||||
* If you previously called get_sigint_guard_condition() for a given wait set
|
||||
* to get a sigint guard condition, then you should call release_sigint_guard_condition()
|
||||
* when you're done, to free that condition. Will throw an exception if
|
||||
* get_sigint_guard_condition() wasn't previously called for the given wait set.
|
||||
* \param wait_set Pointer to the rcl_wait_set_t that was using the
|
||||
* resulting guard condition.
|
||||
* to get a sigint guard condition, then you should call
|
||||
* release_sigint_guard_condition() when you're done, to free that condition.
|
||||
* Will throw an exception if get_sigint_guard_condition() wasn't previously
|
||||
* called for the given wait set.
|
||||
*
|
||||
* If nullptr is given for the context, then the global context is used, i.e.
|
||||
* the context initialized by rclcpp::init().
|
||||
*
|
||||
* \param[in] wait_set Pointer to the rcl_wait_set_t that was using the
|
||||
* resulting guard condition.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
void
|
||||
|
@ -135,12 +234,26 @@ release_sigint_guard_condition(rcl_wait_set_t * wait_set);
|
|||
|
||||
/// Use the global condition variable to block for the specified amount of time.
|
||||
/**
|
||||
* This function can be interrupted early if the associated context becomes
|
||||
* invalid due to rclcpp::shutdown() or the signal handler.
|
||||
*
|
||||
* If nullptr is given for the context, then the global context is used, i.e.
|
||||
* the context initialized by rclcpp::init().
|
||||
*
|
||||
* \param[in] nanoseconds A std::chrono::duration representing how long to sleep for.
|
||||
* \param[in] context which may interrupt this sleep
|
||||
* \return True if the condition variable did not timeout.
|
||||
*/
|
||||
RCLCPP_PUBLIC
|
||||
bool
|
||||
sleep_for(const std::chrono::nanoseconds & nanoseconds);
|
||||
sleep_for(
|
||||
const std::chrono::nanoseconds & nanoseconds,
|
||||
rclcpp::Context::SharedPtr context = nullptr);
|
||||
|
||||
/// Notify all blocking calls so they can check for changes in rclcpp::ok().
|
||||
RCLCPP_PUBLIC
|
||||
void
|
||||
notify_all();
|
||||
|
||||
/// Safely check if addition will overflow.
|
||||
/**
|
||||
|
|
|
@ -37,17 +37,20 @@ ClientBase::ClientBase(
|
|||
rclcpp::node_interfaces::NodeBaseInterface * node_base,
|
||||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph)
|
||||
: node_graph_(node_graph),
|
||||
node_handle_(node_base->get_shared_rcl_node_handle())
|
||||
node_handle_(node_base->get_shared_rcl_node_handle()),
|
||||
context_(node_base->get_context())
|
||||
{
|
||||
std::weak_ptr<rcl_node_t> weak_node_handle(node_handle_);
|
||||
client_handle_ = std::shared_ptr<rcl_client_t>(
|
||||
new rcl_client_t, [weak_node_handle](rcl_client_t * client)
|
||||
rcl_client_t * new_rcl_client = new rcl_client_t;
|
||||
*new_rcl_client = rcl_get_zero_initialized_client();
|
||||
client_handle_.reset(
|
||||
new_rcl_client, [weak_node_handle](rcl_client_t * client)
|
||||
{
|
||||
auto handle = weak_node_handle.lock();
|
||||
if (handle) {
|
||||
if (rcl_client_fini(client, handle.get()) != RCL_RET_OK) {
|
||||
RCLCPP_ERROR(
|
||||
rclcpp::get_logger(rcl_node_get_logger_name(handle.get())).get_child("rclcpp"),
|
||||
rclcpp::get_node_logger(handle.get()).get_child("rclcpp"),
|
||||
"Error in destruction of rcl client handle: %s", rcl_get_error_string().str);
|
||||
rcl_reset_error();
|
||||
}
|
||||
|
@ -59,7 +62,6 @@ ClientBase::ClientBase(
|
|||
}
|
||||
delete client;
|
||||
});
|
||||
*client_handle_.get() = rcl_get_zero_initialized_client();
|
||||
}
|
||||
|
||||
ClientBase::~ClientBase()
|
||||
|
@ -94,6 +96,13 @@ ClientBase::service_is_ready() const
|
|||
this->get_rcl_node_handle(),
|
||||
this->get_client_handle().get(),
|
||||
&is_ready);
|
||||
if (RCL_RET_NODE_INVALID == ret) {
|
||||
const rcl_node_t * node_handle = this->get_rcl_node_handle();
|
||||
if (node_handle && !rcl_context_is_valid(node_handle->context)) {
|
||||
// context is shutdown, do a soft failure
|
||||
return false;
|
||||
}
|
||||
}
|
||||
if (ret != RCL_RET_OK) {
|
||||
throw_from_rcl_error(ret, "rcl_service_server_is_available failed");
|
||||
}
|
||||
|
@ -128,7 +137,7 @@ ClientBase::wait_for_service_nanoseconds(std::chrono::nanoseconds timeout)
|
|||
}
|
||||
// continue forever if timeout is negative, otherwise continue until out of time_to_wait
|
||||
do {
|
||||
if (!rclcpp::ok()) {
|
||||
if (!rclcpp::ok(this->context_)) {
|
||||
return false;
|
||||
}
|
||||
// Limit each wait to 100ms to workaround an issue specific to the Connext RMW implementation.
|
||||
|
|
|
@ -14,6 +14,203 @@
|
|||
|
||||
#include "rclcpp/context.hpp"
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "rcl/init.h"
|
||||
#include "rclcpp/exceptions.hpp"
|
||||
#include "rclcpp/logging.hpp"
|
||||
|
||||
/// Mutex to protect initialized contexts.
|
||||
static std::mutex g_contexts_mutex;
|
||||
/// Weak list of context to be shutdown by the signal handler.
|
||||
static std::vector<std::weak_ptr<rclcpp::Context>> g_contexts;
|
||||
|
||||
using rclcpp::Context;
|
||||
|
||||
Context::Context() {}
|
||||
Context::Context()
|
||||
: rcl_context_(nullptr), shutdown_reason_("") {}
|
||||
|
||||
Context::~Context()
|
||||
{
|
||||
// acquire the init lock to prevent race conditions with init and shutdown
|
||||
// this will not prevent errors, but will maybe make them easier to reproduce
|
||||
std::lock_guard<std::recursive_mutex> lock(init_mutex_);
|
||||
try {
|
||||
this->shutdown("context destructor was called while still not shutdown");
|
||||
// at this point it is shutdown and cannot reinit
|
||||
// clean_up will finalize the rcl context
|
||||
this->clean_up();
|
||||
} catch (const std::exception & exc) {
|
||||
RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "unhandled exception in ~Context(): %s", exc.what());
|
||||
} catch (...) {
|
||||
RCLCPP_ERROR(rclcpp::get_logger("rclcpp"), "unhandled exception in ~Context()");
|
||||
}
|
||||
}
|
||||
|
||||
RCLCPP_LOCAL
|
||||
void
|
||||
__delete_context(rcl_context_t * context)
|
||||
{
|
||||
if (context) {
|
||||
if (rcl_context_is_valid(context)) {
|
||||
RCLCPP_ERROR(
|
||||
rclcpp::get_logger("rclcpp"), "rcl context unexpectedly not shutdown during cleanup");
|
||||
} else {
|
||||
// if context pointer is not null and is shutdown, then it's ready for fini
|
||||
rcl_ret_t ret = rcl_context_fini(context);
|
||||
if (RCL_RET_OK != ret) {
|
||||
RCLCPP_ERROR(
|
||||
rclcpp::get_logger("rclcpp"),
|
||||
"failed to finalize context: %s", rcl_get_error_string().str);
|
||||
rcl_reset_error();
|
||||
}
|
||||
}
|
||||
delete context;
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
Context::init(
|
||||
int argc,
|
||||
char const * const argv[],
|
||||
const rclcpp::InitOptions & init_options)
|
||||
{
|
||||
std::lock_guard<std::recursive_mutex> init_lock(init_mutex_);
|
||||
if (this->is_valid()) {
|
||||
throw rclcpp::ContextAlreadyInitialized();
|
||||
}
|
||||
this->clean_up();
|
||||
rcl_context_.reset(new rcl_context_t, __delete_context);
|
||||
*rcl_context_.get() = rcl_get_zero_initialized_context();
|
||||
rcl_ret_t ret = rcl_init(argc, argv, init_options.get_rcl_init_options(), rcl_context_.get());
|
||||
if (RCL_RET_OK != ret) {
|
||||
rcl_context_.reset();
|
||||
rclcpp::exceptions::throw_from_rcl_error(ret, "failed to initialize rcl");
|
||||
}
|
||||
init_options_ = init_options;
|
||||
|
||||
std::lock_guard<std::mutex> lock(g_contexts_mutex);
|
||||
g_contexts.push_back(this->shared_from_this());
|
||||
}
|
||||
|
||||
bool
|
||||
Context::is_valid() const
|
||||
{
|
||||
// Take a local copy of the shared pointer to avoid it getting nulled under our feet.
|
||||
auto local_rcl_context = rcl_context_;
|
||||
if (!local_rcl_context) {
|
||||
return false;
|
||||
}
|
||||
return rcl_context_is_valid(local_rcl_context.get());
|
||||
}
|
||||
|
||||
const rclcpp::InitOptions &
|
||||
Context::get_init_options() const
|
||||
{
|
||||
return init_options_;
|
||||
}
|
||||
|
||||
rclcpp::InitOptions
|
||||
Context::get_init_options()
|
||||
{
|
||||
return init_options_;
|
||||
}
|
||||
|
||||
std::string
|
||||
Context::shutdown_reason()
|
||||
{
|
||||
std::lock_guard<std::recursive_mutex> lock(init_mutex_);
|
||||
return shutdown_reason_;
|
||||
}
|
||||
|
||||
bool
|
||||
Context::shutdown(const std::string & reason, bool notify_all)
|
||||
{
|
||||
// prevent races
|
||||
std::lock_guard<std::recursive_mutex> init_lock(init_mutex_);
|
||||
// ensure validity
|
||||
if (!this->is_valid()) {
|
||||
// if it is not valid, then it cannot be shutdown
|
||||
return false;
|
||||
}
|
||||
// rcl shutdown
|
||||
rcl_ret_t ret = rcl_shutdown(rcl_context_.get());
|
||||
if (RCL_RET_OK != ret) {
|
||||
rclcpp::exceptions::throw_from_rcl_error(ret);
|
||||
}
|
||||
// set shutdown reason
|
||||
shutdown_reason_ = reason;
|
||||
// call each shutdown callback
|
||||
for (const auto & callback : on_shutdown_callbacks_) {
|
||||
callback();
|
||||
}
|
||||
// notify all blocking calls, if asked
|
||||
if (notify_all) {
|
||||
rclcpp::notify_all();
|
||||
}
|
||||
// remove self from the global contexts
|
||||
std::lock_guard<std::mutex> context_lock(g_contexts_mutex);
|
||||
for (auto it = g_contexts.begin(); it != g_contexts.end(); ) {
|
||||
auto shared_context = it->lock();
|
||||
if (shared_context.get() == this) {
|
||||
it = g_contexts.erase(it);
|
||||
break;
|
||||
} else {
|
||||
++it;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
rclcpp::Context::OnShutdownCallback
|
||||
Context::on_shutdown(OnShutdownCallback callback)
|
||||
{
|
||||
on_shutdown_callbacks_.push_back(callback);
|
||||
return callback;
|
||||
}
|
||||
|
||||
const std::vector<rclcpp::Context::OnShutdownCallback> &
|
||||
Context::get_on_shutdown_callbacks() const
|
||||
{
|
||||
return on_shutdown_callbacks_;
|
||||
}
|
||||
|
||||
std::vector<rclcpp::Context::OnShutdownCallback> &
|
||||
Context::get_on_shutdown_callbacks()
|
||||
{
|
||||
return on_shutdown_callbacks_;
|
||||
}
|
||||
|
||||
std::shared_ptr<rcl_context_t>
|
||||
Context::get_rcl_context()
|
||||
{
|
||||
return rcl_context_;
|
||||
}
|
||||
|
||||
void
|
||||
Context::clean_up()
|
||||
{
|
||||
shutdown_reason_ = "";
|
||||
rcl_context_.reset();
|
||||
}
|
||||
|
||||
std::vector<Context::SharedPtr>
|
||||
rclcpp::get_contexts()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(g_contexts_mutex);
|
||||
std::vector<Context::SharedPtr> shared_contexts;
|
||||
for (auto it = g_contexts.begin(); it != g_contexts.end(); /* noop */) {
|
||||
auto context_ptr = it->lock();
|
||||
if (!context_ptr) {
|
||||
// remove invalid weak context pointers
|
||||
it = g_contexts.erase(it);
|
||||
} else {
|
||||
++it;
|
||||
shared_contexts.push_back(context_ptr);
|
||||
}
|
||||
}
|
||||
return shared_contexts;
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@
|
|||
|
||||
#include "rcutils/logging_macros.h"
|
||||
|
||||
using rclcpp::exceptions::throw_from_rcl_error;
|
||||
using rclcpp::executor::AnyExecutable;
|
||||
using rclcpp::executor::Executor;
|
||||
using rclcpp::executor::ExecutorArgs;
|
||||
|
@ -40,24 +41,26 @@ Executor::Executor(const ExecutorArgs & args)
|
|||
memory_strategy_(args.memory_strategy)
|
||||
{
|
||||
rcl_guard_condition_options_t guard_condition_options = rcl_guard_condition_get_default_options();
|
||||
if (rcl_guard_condition_init(
|
||||
&interrupt_guard_condition_, guard_condition_options) != RCL_RET_OK)
|
||||
{
|
||||
throw std::runtime_error(
|
||||
std::string("Failed to create interrupt guard condition in Executor constructor: ") +
|
||||
rcl_get_error_string().str);
|
||||
rcl_ret_t ret = rcl_guard_condition_init(
|
||||
&interrupt_guard_condition_, args.context->get_rcl_context().get(), guard_condition_options);
|
||||
if (RCL_RET_OK != ret) {
|
||||
throw_from_rcl_error(ret, "Failed to create interrupt guard condition in Executor constructor");
|
||||
}
|
||||
|
||||
// The number of guard conditions is always at least 2: 1 for the ctrl-c guard cond,
|
||||
// and one for the executor's guard cond (interrupt_guard_condition_)
|
||||
|
||||
// Put the global ctrl-c guard condition in
|
||||
memory_strategy_->add_guard_condition(rclcpp::get_sigint_guard_condition(&wait_set_));
|
||||
memory_strategy_->add_guard_condition(
|
||||
rclcpp::get_sigint_guard_condition(&wait_set_, args.context));
|
||||
|
||||
// Put the executor's guard condition in
|
||||
memory_strategy_->add_guard_condition(&interrupt_guard_condition_);
|
||||
rcl_allocator_t allocator = memory_strategy_->get_allocator();
|
||||
|
||||
// Store the context for later use.
|
||||
context_ = args.context;
|
||||
|
||||
if (rcl_wait_set_init(
|
||||
&wait_set_, 0, 2, 0, 0, 0, allocator) != RCL_RET_OK)
|
||||
{
|
||||
|
@ -103,7 +106,7 @@ Executor::~Executor()
|
|||
}
|
||||
// Remove and release the sigint guard condition
|
||||
memory_strategy_->remove_guard_condition(
|
||||
rclcpp::get_sigint_guard_condition(&wait_set_));
|
||||
rclcpp::get_sigint_guard_condition(&wait_set_, context_));
|
||||
rclcpp::release_sigint_guard_condition(&wait_set_);
|
||||
}
|
||||
|
||||
|
|
|
@ -70,11 +70,11 @@ MultiThreadedExecutor::get_number_of_threads()
|
|||
void
|
||||
MultiThreadedExecutor::run(size_t)
|
||||
{
|
||||
while (rclcpp::ok() && spinning.load()) {
|
||||
while (rclcpp::ok(this->context_) && spinning.load()) {
|
||||
executor::AnyExecutable any_exec;
|
||||
{
|
||||
std::lock_guard<std::mutex> wait_lock(wait_mutex_);
|
||||
if (!rclcpp::ok() || !spinning.load()) {
|
||||
if (!rclcpp::ok(this->context_) || !spinning.load()) {
|
||||
return;
|
||||
}
|
||||
if (!get_next_executable(any_exec)) {
|
||||
|
|
|
@ -30,7 +30,7 @@ SingleThreadedExecutor::spin()
|
|||
throw std::runtime_error("spin() called while already spinning");
|
||||
}
|
||||
RCLCPP_SCOPE_EXIT(this->spinning.store(false); );
|
||||
while (rclcpp::ok() && spinning.load()) {
|
||||
while (rclcpp::ok(this->context_) && spinning.load()) {
|
||||
rclcpp::executor::AnyExecutable any_executable;
|
||||
if (get_next_executable(any_executable)) {
|
||||
execute_any_executable(any_executable);
|
||||
|
|
|
@ -35,17 +35,26 @@ namespace rclcpp
|
|||
namespace graph_listener
|
||||
{
|
||||
|
||||
GraphListener::GraphListener()
|
||||
: is_started_(false), is_shutdown_(false), shutdown_guard_condition_(nullptr)
|
||||
GraphListener::GraphListener(std::shared_ptr<rclcpp::Context> parent_context)
|
||||
: is_started_(false),
|
||||
is_shutdown_(false),
|
||||
interrupt_guard_condition_context_(nullptr),
|
||||
shutdown_guard_condition_(nullptr)
|
||||
{
|
||||
// TODO(wjwwood): make a guard condition class in rclcpp so this can be tracked
|
||||
// automatically with the rcl guard condition
|
||||
// hold on to this context to prevent it from going out of scope while this
|
||||
// guard condition is using it.
|
||||
interrupt_guard_condition_context_ = parent_context->get_rcl_context();
|
||||
rcl_ret_t ret = rcl_guard_condition_init(
|
||||
&interrupt_guard_condition_,
|
||||
interrupt_guard_condition_context_.get(),
|
||||
rcl_guard_condition_get_default_options());
|
||||
if (RCL_RET_OK != ret) {
|
||||
throw_from_rcl_error(ret, "failed to create interrupt guard condition");
|
||||
}
|
||||
|
||||
shutdown_guard_condition_ = rclcpp::get_sigint_guard_condition(&wait_set_);
|
||||
shutdown_guard_condition_ = rclcpp::get_sigint_guard_condition(&wait_set_, parent_context);
|
||||
}
|
||||
|
||||
GraphListener::~GraphListener()
|
||||
|
@ -337,6 +346,7 @@ GraphListener::shutdown()
|
|||
listener_thread_.join();
|
||||
}
|
||||
rcl_ret_t ret = rcl_guard_condition_fini(&interrupt_guard_condition_);
|
||||
interrupt_guard_condition_context_.reset(); // release context guard condition was using
|
||||
if (RCL_RET_OK != ret) {
|
||||
throw_from_rcl_error(ret, "failed to finalize interrupt guard condition");
|
||||
}
|
||||
|
|
86
rclcpp/src/rclcpp/init_options.cpp
Normal file
86
rclcpp/src/rclcpp/init_options.cpp
Normal file
|
@ -0,0 +1,86 @@
|
|||
// Copyright 2018 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/init_options.hpp"
|
||||
|
||||
#include "rclcpp/exceptions.hpp"
|
||||
#include "rclcpp/logging.hpp"
|
||||
|
||||
namespace rclcpp
|
||||
{
|
||||
|
||||
InitOptions::InitOptions(rcl_allocator_t allocator)
|
||||
: init_options_(new rcl_init_options_t)
|
||||
{
|
||||
*init_options_ = rcl_get_zero_initialized_init_options();
|
||||
rcl_ret_t ret = rcl_init_options_init(init_options_.get(), allocator);
|
||||
if (RCL_RET_OK != ret) {
|
||||
rclcpp::exceptions::throw_from_rcl_error(ret, "failed to initialized rcl init options");
|
||||
}
|
||||
}
|
||||
|
||||
InitOptions::InitOptions(const rcl_init_options_t & init_options)
|
||||
: init_options_(new rcl_init_options_t)
|
||||
{
|
||||
*init_options_ = rcl_get_zero_initialized_init_options();
|
||||
rcl_ret_t ret = rcl_init_options_copy(&init_options, init_options_.get());
|
||||
if (RCL_RET_OK != ret) {
|
||||
rclcpp::exceptions::throw_from_rcl_error(ret, "failed to copy rcl init options");
|
||||
}
|
||||
}
|
||||
|
||||
InitOptions::InitOptions(const InitOptions & other)
|
||||
: InitOptions(*other.get_rcl_init_options())
|
||||
{}
|
||||
|
||||
InitOptions &
|
||||
InitOptions::operator=(const InitOptions & other)
|
||||
{
|
||||
if (this != &other) {
|
||||
this->finalize_init_options();
|
||||
rcl_ret_t ret = rcl_init_options_copy(other.get_rcl_init_options(), init_options_.get());
|
||||
if (RCL_RET_OK != ret) {
|
||||
rclcpp::exceptions::throw_from_rcl_error(ret, "failed to copy rcl init options");
|
||||
}
|
||||
}
|
||||
return *this;
|
||||
}
|
||||
|
||||
InitOptions::~InitOptions()
|
||||
{
|
||||
this->finalize_init_options();
|
||||
}
|
||||
|
||||
void
|
||||
InitOptions::finalize_init_options()
|
||||
{
|
||||
if (init_options_) {
|
||||
rcl_ret_t ret = rcl_init_options_fini(init_options_.get());
|
||||
if (RCL_RET_OK != ret) {
|
||||
RCLCPP_ERROR(
|
||||
rclcpp::get_logger("rclcpp"),
|
||||
"failed to finalize rcl init options: %s", rcl_get_error_string().str);
|
||||
rcl_reset_error();
|
||||
}
|
||||
*init_options_ = rcl_get_zero_initialized_init_options();
|
||||
}
|
||||
}
|
||||
|
||||
const rcl_init_options_t *
|
||||
InitOptions::get_rcl_init_options() const
|
||||
{
|
||||
return init_options_.get();
|
||||
}
|
||||
|
||||
} // namespace rclcpp
|
|
@ -16,10 +16,12 @@
|
|||
|
||||
#include "rclcpp/logger.hpp"
|
||||
|
||||
#include "rclcpp/logging.hpp"
|
||||
|
||||
namespace rclcpp
|
||||
{
|
||||
|
||||
rclcpp::Logger
|
||||
Logger
|
||||
get_logger(const std::string & name)
|
||||
{
|
||||
#if RCLCPP_LOGGING_ENABLED
|
||||
|
@ -30,4 +32,16 @@ get_logger(const std::string & name)
|
|||
#endif
|
||||
}
|
||||
|
||||
Logger
|
||||
get_node_logger(const rcl_node_t * node)
|
||||
{
|
||||
const char * logger_name = rcl_node_get_logger_name(node);
|
||||
if (nullptr == logger_name) {
|
||||
auto logger = rclcpp::get_logger("rclcpp");
|
||||
RCLCPP_ERROR(logger, "failed to get logger name from node at address %p", node);
|
||||
return logger;
|
||||
}
|
||||
return rclcpp::get_logger(logger_name);
|
||||
}
|
||||
|
||||
} // namespace rclcpp
|
||||
|
|
|
@ -79,7 +79,8 @@ Node::Node(
|
|||
node_base_,
|
||||
node_topics_,
|
||||
node_graph_,
|
||||
node_services_
|
||||
node_services_,
|
||||
node_logging_
|
||||
)),
|
||||
node_waitables_(new rclcpp::node_interfaces::NodeWaitables(node_base_.get())),
|
||||
use_intra_process_comms_(use_intra_process_comms)
|
||||
|
@ -255,6 +256,12 @@ Node::get_node_graph_interface()
|
|||
return node_graph_;
|
||||
}
|
||||
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr
|
||||
Node::get_node_logging_interface()
|
||||
{
|
||||
return node_logging_;
|
||||
}
|
||||
|
||||
rclcpp::node_interfaces::NodeTimersInterface::SharedPtr
|
||||
Node::get_node_timers_interface()
|
||||
{
|
||||
|
|
|
@ -22,8 +22,8 @@
|
|||
#include "rcl/arguments.h"
|
||||
#include "rclcpp/exceptions.hpp"
|
||||
#include "rcutils/logging_macros.h"
|
||||
#include "rmw/validate_node_name.h"
|
||||
#include "rmw/validate_namespace.h"
|
||||
#include "rmw/validate_node_name.h"
|
||||
|
||||
using rclcpp::exceptions::throw_from_rcl_error;
|
||||
|
||||
|
@ -43,7 +43,8 @@ NodeBase::NodeBase(
|
|||
{
|
||||
// Setup the guard condition that is notified when changes occur in the graph.
|
||||
rcl_guard_condition_options_t guard_condition_options = rcl_guard_condition_get_default_options();
|
||||
rcl_ret_t ret = rcl_guard_condition_init(¬ify_guard_condition_, guard_condition_options);
|
||||
rcl_ret_t ret = rcl_guard_condition_init(
|
||||
¬ify_guard_condition_, context->get_rcl_context().get(), guard_condition_options);
|
||||
if (ret != RCL_RET_OK) {
|
||||
throw_from_rcl_error(ret, "failed to create interrupt guard condition");
|
||||
}
|
||||
|
@ -113,7 +114,10 @@ NodeBase::NodeBase(
|
|||
// TODO(wjwwood): pass the Allocator to the options
|
||||
options.domain_id = domain_id;
|
||||
|
||||
ret = rcl_node_init(rcl_node.get(), node_name.c_str(), namespace_.c_str(), &options);
|
||||
ret = rcl_node_init(
|
||||
rcl_node.get(),
|
||||
node_name.c_str(), namespace_.c_str(),
|
||||
context->get_rcl_context().get(), &options);
|
||||
if (ret != RCL_RET_OK) {
|
||||
// Finalize the interrupt guard condition.
|
||||
finalize_notify_guard_condition();
|
||||
|
|
|
@ -23,18 +23,21 @@ NodeClock::NodeClock(
|
|||
rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_base,
|
||||
rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics,
|
||||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph,
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services)
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services,
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr node_logging)
|
||||
: node_base_(node_base),
|
||||
node_topics_(node_topics),
|
||||
node_graph_(node_graph),
|
||||
node_services_(node_services),
|
||||
node_logging_(node_logging),
|
||||
ros_clock_(std::make_shared<rclcpp::Clock>(RCL_ROS_TIME))
|
||||
{
|
||||
time_source_.attachNode(
|
||||
node_base_,
|
||||
node_topics_,
|
||||
node_graph_,
|
||||
node_services_);
|
||||
node_services_,
|
||||
node_logging_);
|
||||
time_source_.attachClock(ros_clock_);
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,9 @@ using rclcpp::graph_listener::GraphListener;
|
|||
|
||||
NodeGraph::NodeGraph(rclcpp::node_interfaces::NodeBaseInterface * node_base)
|
||||
: node_base_(node_base),
|
||||
graph_listener_(node_base->get_context()->get_sub_context<GraphListener>()),
|
||||
graph_listener_(
|
||||
node_base->get_context()->get_sub_context<GraphListener>(node_base->get_context())
|
||||
),
|
||||
should_add_to_graph_listener_(true),
|
||||
graph_users_count_(0)
|
||||
{}
|
||||
|
@ -334,8 +336,8 @@ NodeGraph::wait_for_graph_change(
|
|||
throw EventNotRegisteredError();
|
||||
}
|
||||
}
|
||||
auto pred = [&event]() {
|
||||
return event->check() || !rclcpp::ok();
|
||||
auto pred = [&event, context = node_base_->get_context()]() {
|
||||
return event->check() || !rclcpp::ok(context);
|
||||
};
|
||||
std::unique_lock<std::mutex> graph_lock(graph_mutex_);
|
||||
if (!pred()) {
|
||||
|
|
|
@ -64,7 +64,7 @@ NodeParameters::NodeParameters(
|
|||
}
|
||||
const rcl_node_options_t * options = rcl_node_get_options(node);
|
||||
if (nullptr == options) {
|
||||
throw std::runtime_error("Need valid node options NodeParameters");
|
||||
throw std::runtime_error("Need valid node options in NodeParameters");
|
||||
}
|
||||
|
||||
// Get paths to yaml files containing initial parameter values
|
||||
|
@ -93,7 +93,8 @@ NodeParameters::NodeParameters(
|
|||
|
||||
// global before local so that local overwrites global
|
||||
if (options->use_global_arguments) {
|
||||
get_yaml_paths(rcl_get_global_arguments());
|
||||
auto context_ptr = node_base->get_context()->get_rcl_context();
|
||||
get_yaml_paths(&(context_ptr->global_arguments));
|
||||
}
|
||||
get_yaml_paths(&(options->arguments));
|
||||
|
||||
|
|
|
@ -41,7 +41,7 @@ SubscriptionBase::SubscriptionBase(
|
|||
{
|
||||
if (rcl_subscription_fini(rcl_subs, node_handle.get()) != RCL_RET_OK) {
|
||||
RCLCPP_ERROR(
|
||||
rclcpp::get_logger(rcl_node_get_logger_name(node_handle.get())).get_child("rclcpp"),
|
||||
rclcpp::get_node_logger(node_handle.get()).get_child("rclcpp"),
|
||||
"Error in destruction of rcl subscription handle: %s",
|
||||
rcl_get_error_string().str);
|
||||
rcl_reset_error();
|
||||
|
|
|
@ -21,10 +21,9 @@
|
|||
|
||||
#include "rcl/time.h"
|
||||
|
||||
#include "rcutils/logging_macros.h"
|
||||
|
||||
#include "rclcpp/clock.hpp"
|
||||
#include "rclcpp/exceptions.hpp"
|
||||
#include "rclcpp/logging.hpp"
|
||||
#include "rclcpp/node.hpp"
|
||||
#include "rclcpp/time.hpp"
|
||||
#include "rclcpp/time_source.hpp"
|
||||
|
@ -34,14 +33,16 @@ namespace rclcpp
|
|||
{
|
||||
|
||||
TimeSource::TimeSource(std::shared_ptr<rclcpp::Node> node)
|
||||
: clock_subscription_(nullptr),
|
||||
: logger_(rclcpp::get_logger("rclcpp")),
|
||||
clock_subscription_(nullptr),
|
||||
ros_time_active_(false)
|
||||
{
|
||||
this->attachNode(node);
|
||||
}
|
||||
|
||||
TimeSource::TimeSource()
|
||||
: ros_time_active_(false)
|
||||
: logger_(rclcpp::get_logger("rclcpp")),
|
||||
ros_time_active_(false)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -51,21 +52,26 @@ void TimeSource::attachNode(rclcpp::Node::SharedPtr node)
|
|||
node->get_node_base_interface(),
|
||||
node->get_node_topics_interface(),
|
||||
node->get_node_graph_interface(),
|
||||
node->get_node_services_interface());
|
||||
node->get_node_services_interface(),
|
||||
node->get_node_logging_interface());
|
||||
}
|
||||
|
||||
void TimeSource::attachNode(
|
||||
const rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_base_interface,
|
||||
const rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics_interface,
|
||||
const rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph_interface,
|
||||
const rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services_interface)
|
||||
rclcpp::node_interfaces::NodeBaseInterface::SharedPtr node_base_interface,
|
||||
rclcpp::node_interfaces::NodeTopicsInterface::SharedPtr node_topics_interface,
|
||||
rclcpp::node_interfaces::NodeGraphInterface::SharedPtr node_graph_interface,
|
||||
rclcpp::node_interfaces::NodeServicesInterface::SharedPtr node_services_interface,
|
||||
rclcpp::node_interfaces::NodeLoggingInterface::SharedPtr node_logging_interface)
|
||||
{
|
||||
node_base_ = node_base_interface;
|
||||
node_topics_ = node_topics_interface;
|
||||
node_graph_ = node_graph_interface;
|
||||
node_services_ = node_services_interface;
|
||||
node_logging_ = node_logging_interface;
|
||||
// TODO(tfoote): Update QOS
|
||||
|
||||
logger_ = node_logging_->get_logger();
|
||||
|
||||
parameter_client_ = std::make_shared<rclcpp::AsyncParametersClient>(
|
||||
node_base_,
|
||||
node_topics_,
|
||||
|
@ -111,7 +117,7 @@ void TimeSource::detachClock(std::shared_ptr<rclcpp::Clock> clock)
|
|||
if (result != associated_clocks_.end()) {
|
||||
associated_clocks_.erase(result);
|
||||
} else {
|
||||
RCUTILS_LOG_ERROR("Failed to remove clock");
|
||||
RCLCPP_ERROR(logger_, "failed to remove clock");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -200,7 +206,7 @@ void TimeSource::on_parameter_event(const rcl_interfaces::msg::ParameterEvent::S
|
|||
rclcpp::ParameterEventsFilter::EventType::CHANGED});
|
||||
for (auto & it : filter.get_events()) {
|
||||
if (it.second->value.type != ParameterType::PARAMETER_BOOL) {
|
||||
RCUTILS_LOG_ERROR("use_sim_time parameter set to something besides a bool");
|
||||
RCLCPP_ERROR(logger_, "use_sim_time parameter set to something besides a bool");
|
||||
continue;
|
||||
}
|
||||
if (it.second->value.bool_value) {
|
||||
|
|
|
@ -18,13 +18,22 @@
|
|||
#include <string>
|
||||
#include <memory>
|
||||
|
||||
#include "rclcpp/contexts/default_context.hpp"
|
||||
#include "rcutils/logging_macros.h"
|
||||
|
||||
using rclcpp::TimerBase;
|
||||
|
||||
TimerBase::TimerBase(rclcpp::Clock::SharedPtr clock, std::chrono::nanoseconds period)
|
||||
TimerBase::TimerBase(
|
||||
rclcpp::Clock::SharedPtr clock,
|
||||
std::chrono::nanoseconds period,
|
||||
rclcpp::Context::SharedPtr context)
|
||||
: clock_(clock), timer_handle_(nullptr)
|
||||
{
|
||||
clock_ = clock;
|
||||
if (nullptr == context) {
|
||||
context = rclcpp::contexts::default_context::get_global_default_context();
|
||||
}
|
||||
|
||||
auto rcl_context = context->get_rcl_context();
|
||||
|
||||
timer_handle_ = std::shared_ptr<rcl_timer_t>(
|
||||
new rcl_timer_t, [ = ](rcl_timer_t * timer) mutable
|
||||
|
@ -36,15 +45,16 @@ TimerBase::TimerBase(rclcpp::Clock::SharedPtr clock, std::chrono::nanoseconds pe
|
|||
rcl_reset_error();
|
||||
}
|
||||
delete timer;
|
||||
// Captured shared pointer by copy, reset to make sure timer is finalized before clock
|
||||
// Captured shared pointers by copy, reset to make sure timer is finalized before clock
|
||||
clock.reset();
|
||||
rcl_context.reset();
|
||||
});
|
||||
|
||||
*timer_handle_.get() = rcl_get_zero_initialized_timer();
|
||||
|
||||
rcl_clock_t * clock_handle = clock_->get_clock_handle();
|
||||
if (rcl_timer_init(
|
||||
timer_handle_.get(), clock_handle, period.count(), nullptr,
|
||||
timer_handle_.get(), clock_handle, rcl_context.get(), period.count(), nullptr,
|
||||
rcl_get_default_allocator()) != RCL_RET_OK)
|
||||
{
|
||||
RCUTILS_LOG_ERROR_NAMED(
|
||||
|
|
|
@ -24,7 +24,10 @@
|
|||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "rclcpp/contexts/default_context.hpp"
|
||||
#include "rclcpp/logging.hpp"
|
||||
#include "rclcpp/exceptions.hpp"
|
||||
#include "rclcpp/scope_exit.hpp"
|
||||
|
||||
#include "rcl/error_handling.h"
|
||||
#include "rcl/rcl.h"
|
||||
|
@ -39,15 +42,18 @@
|
|||
#define HAS_SIGACTION
|
||||
#endif
|
||||
|
||||
/// Represent the status of the global interrupt signal.
|
||||
static volatile sig_atomic_t g_signal_status = 0;
|
||||
/// Guard conditions for interrupting the rmw implementation when the global interrupt signal fired.
|
||||
static std::map<rcl_wait_set_t *, rcl_guard_condition_t> g_sigint_guard_cond_handles;
|
||||
/// Mutex to protect installation and uninstallation of signal handlers.
|
||||
static std::mutex g_signal_handlers_mutex;
|
||||
/// Atomic bool to control setup of signal handlers.
|
||||
static std::atomic<bool> g_signal_handlers_installed(false);
|
||||
|
||||
/// Mutex to protect g_sigint_guard_cond_handles
|
||||
static std::mutex g_sigint_guard_cond_handles_mutex;
|
||||
/// Guard conditions for interrupting the rmw implementation when the global interrupt signal fired.
|
||||
static std::map<rcl_wait_set_t *, rcl_guard_condition_t> g_sigint_guard_cond_handles;
|
||||
|
||||
/// Condition variable for timed sleep (see sleep_for).
|
||||
static std::condition_variable g_interrupt_condition_variable;
|
||||
static std::atomic<bool> g_is_interrupted(false);
|
||||
/// Mutex for protecting the global condition variable.
|
||||
static std::mutex g_interrupt_mutex;
|
||||
|
||||
|
@ -109,25 +115,6 @@ set_signal_handler(int signal_value, signal_handler_t signal_handler)
|
|||
#endif
|
||||
}
|
||||
|
||||
void
|
||||
trigger_interrupt_guard_condition(int signal_value)
|
||||
{
|
||||
g_signal_status = signal_value;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(g_sigint_guard_cond_handles_mutex);
|
||||
for (auto & kv : g_sigint_guard_cond_handles) {
|
||||
rcl_ret_t status = rcl_trigger_guard_condition(&(kv.second));
|
||||
if (status != RCL_RET_OK) {
|
||||
RCUTILS_LOG_ERROR_NAMED(
|
||||
"rclcpp",
|
||||
"failed to trigger guard condition: %s", rcl_get_error_string().str);
|
||||
}
|
||||
}
|
||||
}
|
||||
g_is_interrupted.store(true);
|
||||
g_interrupt_condition_variable.notify_all();
|
||||
}
|
||||
|
||||
void
|
||||
#ifdef HAS_SIGACTION
|
||||
signal_handler(int signal_value, siginfo_t * siginfo, void * context)
|
||||
|
@ -135,8 +122,7 @@ signal_handler(int signal_value, siginfo_t * siginfo, void * context)
|
|||
signal_handler(int signal_value)
|
||||
#endif
|
||||
{
|
||||
// TODO(wjwwood): remove? move to console logging at some point?
|
||||
printf("signal_handler(%d)\n", signal_value);
|
||||
RCLCPP_INFO(rclcpp::get_logger("rclcpp"), "signal_handler(signal_value=%d)", signal_value);
|
||||
|
||||
#ifdef HAS_SIGACTION
|
||||
if (old_action.sa_flags & SA_SIGINFO) {
|
||||
|
@ -158,18 +144,31 @@ signal_handler(int signal_value)
|
|||
}
|
||||
#endif
|
||||
|
||||
trigger_interrupt_guard_condition(signal_value);
|
||||
for (auto context_ptr : rclcpp::get_contexts()) {
|
||||
if (context_ptr->get_init_options().shutdown_on_sigint) {
|
||||
// do not notify all, instead do that once after all are shutdown
|
||||
context_ptr->shutdown("signal handler", false /* notify_all */);
|
||||
}
|
||||
}
|
||||
rclcpp::notify_all();
|
||||
}
|
||||
|
||||
void
|
||||
rclcpp::init(int argc, char const * const argv[])
|
||||
rclcpp::init(int argc, char const * const argv[], const rclcpp::InitOptions & init_options)
|
||||
{
|
||||
g_is_interrupted.store(false);
|
||||
if (rcl_init(argc, argv, rcl_get_default_allocator()) != RCL_RET_OK) {
|
||||
std::string msg = "failed to initialize rmw implementation: ";
|
||||
msg += rcl_get_error_string().str;
|
||||
rcl_reset_error();
|
||||
throw std::runtime_error(msg);
|
||||
using rclcpp::contexts::default_context::get_global_default_context;
|
||||
get_global_default_context()->init(argc, argv, init_options);
|
||||
// Install the signal handlers.
|
||||
rclcpp::install_signal_handlers();
|
||||
}
|
||||
|
||||
bool
|
||||
rclcpp::install_signal_handlers()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(g_signal_handlers_mutex);
|
||||
bool already_installed = g_signal_handlers_installed.exchange(true);
|
||||
if (already_installed) {
|
||||
return false;
|
||||
}
|
||||
#ifdef HAS_SIGACTION
|
||||
struct sigaction action;
|
||||
|
@ -178,25 +177,43 @@ rclcpp::init(int argc, char const * const argv[])
|
|||
action.sa_sigaction = ::signal_handler;
|
||||
action.sa_flags = SA_SIGINFO;
|
||||
::old_action = set_sigaction(SIGINT, action);
|
||||
// Register an on_shutdown hook to restore the old action.
|
||||
rclcpp::on_shutdown(
|
||||
[]() {
|
||||
set_sigaction(SIGINT, ::old_action);
|
||||
});
|
||||
#else
|
||||
::old_signal_handler = set_signal_handler(SIGINT, ::signal_handler);
|
||||
// Register an on_shutdown hook to restore the old signal handler.
|
||||
rclcpp::on_shutdown(
|
||||
[]() {
|
||||
set_signal_handler(SIGINT, ::old_signal_handler);
|
||||
});
|
||||
#endif
|
||||
RCLCPP_DEBUG(rclcpp::get_logger("rclcpp"), "signal handler installed");
|
||||
return true;
|
||||
}
|
||||
|
||||
bool
|
||||
rclcpp::signal_handlers_installed()
|
||||
{
|
||||
return g_signal_handlers_installed.load();
|
||||
}
|
||||
|
||||
bool
|
||||
rclcpp::uninstall_signal_handlers()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(g_signal_handlers_mutex);
|
||||
bool installed = g_signal_handlers_installed.exchange(false);
|
||||
if (!installed) {
|
||||
return false;
|
||||
}
|
||||
#ifdef HAS_SIGACTION
|
||||
set_sigaction(SIGINT, ::old_action);
|
||||
#else
|
||||
set_signal_handler(SIGINT, ::old_signal_handler);
|
||||
#endif
|
||||
RCLCPP_DEBUG(rclcpp::get_logger("rclcpp"), "signal handler uninstalled");
|
||||
return true;
|
||||
}
|
||||
|
||||
std::vector<std::string>
|
||||
rclcpp::init_and_remove_ros_arguments(int argc, char const * const argv[])
|
||||
rclcpp::init_and_remove_ros_arguments(
|
||||
int argc,
|
||||
char const * const argv[],
|
||||
const rclcpp::InitOptions & init_options)
|
||||
{
|
||||
rclcpp::init(argc, argv);
|
||||
rclcpp::init(argc, argv, init_options);
|
||||
return rclcpp::remove_ros_arguments(argc, argv);
|
||||
}
|
||||
|
||||
|
@ -210,7 +227,7 @@ rclcpp::remove_ros_arguments(int argc, char const * const argv[])
|
|||
|
||||
ret = rcl_parse_arguments(argc, argv, alloc, &parsed_args);
|
||||
if (RCL_RET_OK != ret) {
|
||||
exceptions::throw_from_rcl_error(ret, "failed to parse arguments");
|
||||
rclcpp::exceptions::throw_from_rcl_error(ret, "failed to parse arguments");
|
||||
}
|
||||
|
||||
int nonros_argc = 0;
|
||||
|
@ -225,7 +242,7 @@ rclcpp::remove_ros_arguments(int argc, char const * const argv[])
|
|||
|
||||
if (RCL_RET_OK != ret) {
|
||||
// Not using throw_from_rcl_error, because we may need to append deallocation failures.
|
||||
exceptions::RCLErrorBase base_exc(ret, rcl_get_error_state());
|
||||
rclcpp::exceptions::RCLErrorBase base_exc(ret, rcl_get_error_state());
|
||||
rcl_reset_error();
|
||||
if (NULL != nonros_argv) {
|
||||
alloc.deallocate(nonros_argv, alloc.state);
|
||||
|
@ -236,7 +253,7 @@ rclcpp::remove_ros_arguments(int argc, char const * const argv[])
|
|||
rcl_get_error_string().str;
|
||||
rcl_reset_error();
|
||||
}
|
||||
throw exceptions::RCLError(base_exc, "");
|
||||
throw rclcpp::exceptions::RCLError(base_exc, "");
|
||||
}
|
||||
|
||||
std::vector<std::string> return_arguments;
|
||||
|
@ -252,63 +269,75 @@ rclcpp::remove_ros_arguments(int argc, char const * const argv[])
|
|||
|
||||
ret = rcl_arguments_fini(&parsed_args);
|
||||
if (RCL_RET_OK != ret) {
|
||||
exceptions::throw_from_rcl_error(ret, "failed to cleanup parsed arguments, leaking memory");
|
||||
rclcpp::exceptions::throw_from_rcl_error(
|
||||
ret, "failed to cleanup parsed arguments, leaking memory");
|
||||
}
|
||||
|
||||
return return_arguments;
|
||||
}
|
||||
|
||||
bool
|
||||
rclcpp::ok()
|
||||
rclcpp::ok(rclcpp::Context::SharedPtr context)
|
||||
{
|
||||
return ::g_signal_status == 0;
|
||||
using rclcpp::contexts::default_context::get_global_default_context;
|
||||
if (nullptr == context) {
|
||||
context = get_global_default_context();
|
||||
}
|
||||
return context->is_valid();
|
||||
}
|
||||
|
||||
bool
|
||||
rclcpp::is_initialized()
|
||||
rclcpp::is_initialized(rclcpp::Context::SharedPtr context)
|
||||
{
|
||||
return rcl_ok();
|
||||
return rclcpp::ok(context);
|
||||
}
|
||||
|
||||
static std::mutex on_shutdown_mutex_;
|
||||
static std::vector<std::function<void(void)>> on_shutdown_callbacks_;
|
||||
|
||||
void
|
||||
rclcpp::shutdown()
|
||||
bool
|
||||
rclcpp::shutdown(rclcpp::Context::SharedPtr context, const std::string & reason)
|
||||
{
|
||||
trigger_interrupt_guard_condition(SIGINT);
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(on_shutdown_mutex_);
|
||||
for (auto & on_shutdown_callback : on_shutdown_callbacks_) {
|
||||
on_shutdown_callback();
|
||||
}
|
||||
using rclcpp::contexts::default_context::get_global_default_context;
|
||||
auto default_context = get_global_default_context();
|
||||
if (nullptr == context) {
|
||||
context = default_context;
|
||||
}
|
||||
|
||||
bool ret = context->shutdown(reason);
|
||||
|
||||
// Uninstall the signal handlers if this is the default context's shutdown.
|
||||
if (context == default_context) {
|
||||
uninstall_signal_handlers();
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
void
|
||||
rclcpp::on_shutdown(std::function<void(void)> callback)
|
||||
rclcpp::on_shutdown(std::function<void()> callback, rclcpp::Context::SharedPtr context)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(on_shutdown_mutex_);
|
||||
on_shutdown_callbacks_.push_back(callback);
|
||||
using rclcpp::contexts::default_context::get_global_default_context;
|
||||
if (nullptr == context) {
|
||||
context = get_global_default_context();
|
||||
}
|
||||
context->on_shutdown(callback);
|
||||
}
|
||||
|
||||
rcl_guard_condition_t *
|
||||
rclcpp::get_sigint_guard_condition(rcl_wait_set_t * wait_set)
|
||||
rclcpp::get_sigint_guard_condition(rcl_wait_set_t * wait_set, rclcpp::Context::SharedPtr context)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(g_sigint_guard_cond_handles_mutex);
|
||||
auto kv = g_sigint_guard_cond_handles.find(wait_set);
|
||||
if (kv != g_sigint_guard_cond_handles.end()) {
|
||||
return &kv->second;
|
||||
} else {
|
||||
rcl_guard_condition_t handle =
|
||||
rcl_get_zero_initialized_guard_condition();
|
||||
using rclcpp::contexts::default_context::get_global_default_context;
|
||||
if (nullptr == context) {
|
||||
context = get_global_default_context();
|
||||
}
|
||||
rcl_guard_condition_t handle = rcl_get_zero_initialized_guard_condition();
|
||||
rcl_guard_condition_options_t options = rcl_guard_condition_get_default_options();
|
||||
if (rcl_guard_condition_init(&handle, options) != RCL_RET_OK) {
|
||||
// *INDENT-OFF* (prevent uncrustify from making unnecessary indents here)
|
||||
throw std::runtime_error(std::string(
|
||||
"Couldn't initialize guard condition: ") + rcl_get_error_string().str);
|
||||
// *INDENT-ON*
|
||||
auto ret = rcl_guard_condition_init(&handle, context->get_rcl_context().get(), options);
|
||||
if (RCL_RET_OK != ret) {
|
||||
rclcpp::exceptions::throw_from_rcl_error(ret, "Couldn't initialize guard condition: ");
|
||||
}
|
||||
g_sigint_guard_cond_handles[wait_set] = handle;
|
||||
return &g_sigint_guard_cond_handles[wait_set];
|
||||
|
@ -338,20 +367,38 @@ rclcpp::release_sigint_guard_condition(rcl_wait_set_t * wait_set)
|
|||
}
|
||||
|
||||
bool
|
||||
rclcpp::sleep_for(const std::chrono::nanoseconds & nanoseconds)
|
||||
rclcpp::sleep_for(const std::chrono::nanoseconds & nanoseconds, rclcpp::Context::SharedPtr context)
|
||||
{
|
||||
std::chrono::nanoseconds time_left = nanoseconds;
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(::g_interrupt_mutex);
|
||||
auto start = std::chrono::steady_clock::now();
|
||||
// this will release the lock while waiting
|
||||
::g_interrupt_condition_variable.wait_for(lock, nanoseconds);
|
||||
time_left -= std::chrono::steady_clock::now() - start;
|
||||
}
|
||||
if (time_left > std::chrono::nanoseconds::zero() && !g_is_interrupted) {
|
||||
if (time_left > std::chrono::nanoseconds::zero() && ok(context)) {
|
||||
return sleep_for(time_left);
|
||||
}
|
||||
// Return true if the timeout elapsed successfully, otherwise false.
|
||||
return !g_is_interrupted;
|
||||
return ok(context);
|
||||
}
|
||||
|
||||
void
|
||||
rclcpp::notify_all()
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(g_sigint_guard_cond_handles_mutex);
|
||||
for (auto & kv : g_sigint_guard_cond_handles) {
|
||||
rcl_ret_t status = rcl_trigger_guard_condition(&(kv.second));
|
||||
if (status != RCL_RET_OK) {
|
||||
RCUTILS_LOG_ERROR_NAMED(
|
||||
"rclcpp",
|
||||
"failed to trigger guard condition: %s", rcl_get_error_string().str);
|
||||
}
|
||||
}
|
||||
}
|
||||
g_interrupt_condition_variable.notify_all();
|
||||
}
|
||||
|
||||
const char *
|
||||
|
|
|
@ -25,5 +25,5 @@ TEST(TestInit, is_initialized) {
|
|||
|
||||
rclcpp::shutdown();
|
||||
|
||||
EXPECT_TRUE(rclcpp::is_initialized());
|
||||
EXPECT_FALSE(rclcpp::is_initialized());
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
#include <string>
|
||||
#include <memory>
|
||||
|
||||
#include "rclcpp/contexts/default_context.hpp"
|
||||
#include "rclcpp/exceptions.hpp"
|
||||
#include "rclcpp/utilities.hpp"
|
||||
|
||||
|
@ -49,5 +50,34 @@ TEST(TestUtilities, init_with_args) {
|
|||
ASSERT_EQ(1u, other_args.size());
|
||||
ASSERT_EQ(std::string{"process_name"}, other_args[0]);
|
||||
|
||||
EXPECT_TRUE(rclcpp::is_initialized());
|
||||
EXPECT_TRUE(rclcpp::ok());
|
||||
rclcpp::shutdown();
|
||||
}
|
||||
|
||||
TEST(TestUtilities, multi_init) {
|
||||
auto context1 = std::make_shared<rclcpp::contexts::default_context::DefaultContext>();
|
||||
auto context2 = std::make_shared<rclcpp::contexts::default_context::DefaultContext>();
|
||||
|
||||
EXPECT_FALSE(rclcpp::ok(context1));
|
||||
EXPECT_FALSE(rclcpp::ok(context2));
|
||||
|
||||
context1->init(0, nullptr);
|
||||
|
||||
EXPECT_TRUE(rclcpp::ok(context1));
|
||||
EXPECT_FALSE(rclcpp::ok(context2));
|
||||
|
||||
context2->init(0, nullptr);
|
||||
|
||||
EXPECT_TRUE(rclcpp::ok(context1));
|
||||
EXPECT_TRUE(rclcpp::ok(context2));
|
||||
|
||||
rclcpp::shutdown(context1);
|
||||
|
||||
EXPECT_FALSE(rclcpp::ok(context1));
|
||||
EXPECT_TRUE(rclcpp::ok(context2));
|
||||
|
||||
rclcpp::shutdown(context2);
|
||||
|
||||
EXPECT_FALSE(rclcpp::ok(context1));
|
||||
EXPECT_FALSE(rclcpp::ok(context2));
|
||||
}
|
||||
|
|
|
@ -141,7 +141,7 @@ LifecycleNode::create_wall_timer(
|
|||
{
|
||||
auto timer = rclcpp::WallTimer<CallbackT>::make_shared(
|
||||
std::chrono::duration_cast<std::chrono::nanoseconds>(period),
|
||||
std::move(callback));
|
||||
std::move(callback), this->node_base_->get_context());
|
||||
node_timers_->add_timer(timer, group);
|
||||
return timer;
|
||||
}
|
||||
|
|
|
@ -86,7 +86,8 @@ LifecycleNode::LifecycleNode(
|
|||
node_base_,
|
||||
node_topics_,
|
||||
node_graph_,
|
||||
node_services_
|
||||
node_services_,
|
||||
node_logging_
|
||||
)),
|
||||
node_waitables_(new rclcpp::node_interfaces::NodeWaitables(node_base_.get())),
|
||||
use_intra_process_comms_(use_intra_process_comms),
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue