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:
William Woodall 2018-11-29 21:33:01 -08:00 committed by GitHub
parent 36262a5cf5
commit 3af8d2cfed
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
38 changed files with 1077 additions and 196 deletions

View file

@ -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

View file

@ -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);

View file

@ -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_

View file

@ -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)

View file

@ -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);

View file

@ -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

View file

@ -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();
};

View 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_

View file

@ -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:

View file

@ -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

View file

@ -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;
}

View file

@ -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_;

View file

@ -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");
}

View file

@ -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();

View file

@ -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;

View file

@ -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:

View file

@ -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,21 +204,28 @@ 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
*
* \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
* 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
@ -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.
/**

View file

@ -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.

View file

@ -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;
}

View file

@ -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_);
}

View file

@ -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)) {

View file

@ -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);

View file

@ -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");
}

View 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

View file

@ -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

View file

@ -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()
{

View file

@ -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(&notify_guard_condition_, guard_condition_options);
rcl_ret_t ret = rcl_guard_condition_init(
&notify_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();

View file

@ -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_);
}

View file

@ -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()) {

View file

@ -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));

View file

@ -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();

View file

@ -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) {

View file

@ -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(

View file

@ -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);
using rclcpp::contexts::default_context::get_global_default_context;
auto default_context = get_global_default_context();
if (nullptr == context) {
context = default_context;
}
{
std::lock_guard<std::mutex> lock(on_shutdown_mutex_);
for (auto & on_shutdown_callback : on_shutdown_callbacks_) {
on_shutdown_callback();
}
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 *

View file

@ -25,5 +25,5 @@ TEST(TestInit, is_initialized) {
rclcpp::shutdown();
EXPECT_TRUE(rclcpp::is_initialized());
EXPECT_FALSE(rclcpp::is_initialized());
}

View file

@ -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));
}

View file

@ -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;
}

View file

@ -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),