Unverified Commit 3f271538 authored by Noir's avatar Noir Committed by GitHub

Merge pull request #1202

Fix shutdown issue with detached actors
parents 250b0933 aeb67adb
......@@ -97,6 +97,8 @@ is based on [Keep a Changelog](https://keepachangelog.com).
messages.
- `CAF_ADD_TYPE_ID` now works with types that live in namespaces that also exist
as nested namespace in CAF such as `detail` or `io` (#1195).
- Solved a race condition on detached actors that blocked ordinary shutdown of
actor systems in some cases (#1196).
## [0.18.0-rc.1] - 2020-09-09
......
......@@ -118,6 +118,7 @@ caf_add_component(
src/detail/pretty_type_name.cpp
src/detail/print.cpp
src/detail/private_thread.cpp
src/detail/private_thread_pool.cpp
src/detail/ripemd_160.cpp
src/detail/serialized_size.cpp
src/detail/set_thread_name.cpp
......@@ -258,6 +259,7 @@ caf_add_component(
detail.parser.read_string
detail.parser.read_timespan
detail.parser.read_unsigned_integer
detail.private_thread_pool
detail.ringbuffer
detail.ripemd_160
detail.serialized_size
......
......@@ -23,6 +23,7 @@
#include "caf/actor_traits.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/detail/init_fun_factory.hpp"
#include "caf/detail/private_thread_pool.hpp"
#include "caf/detail/spawn_fwd.hpp"
#include "caf/detail/spawnable.hpp"
#include "caf/fwd.hpp"
......@@ -530,21 +531,10 @@ public:
actor_clock& clock() noexcept;
/// Returns the number of detached actors.
size_t detached_actors() {
return detached_.load();
}
size_t detached_actors() const noexcept;
/// @cond PRIVATE
/// Increases running-detached-threads-count by one.
void inc_detached_threads();
/// Decreases running-detached-threads-count by one.
void dec_detached_threads();
/// Blocks the caller until all detached threads are done.
void await_detached_threads();
/// Calls all thread started hooks
/// @warning must be called by thread which is about to start
void thread_started();
......@@ -634,6 +624,10 @@ public:
return tracing_context_;
}
detail::private_thread* acquire_private_thread();
void release_private_thread(detail::private_thread*);
/// @endcond
private:
......@@ -699,15 +693,6 @@ private:
/// Allows fully dynamic spawning of actors.
strong_actor_ptr spawn_serv_;
/// Counts the number of detached actors.
std::atomic<size_t> detached_;
/// Guards `detached`.
mutable std::mutex detached_mtx_;
/// Allows waiting on specific values for `detached`.
mutable std::condition_variable detached_cv_;
/// The system-wide, user-provided configuration.
actor_system_config& cfg_;
......@@ -734,6 +719,9 @@ private:
/// Caches families for optional actor metrics.
actor_metric_families_t actor_metric_families_;
/// Manages threads for detached actors.
detail::private_thread_pool private_threads_;
};
} // namespace caf
......@@ -327,6 +327,8 @@ public:
});
}
using super::fail_state;
/// Sets a user-defined exit reason `err`. This reason
/// is signalized to other actors after `act()` returns.
void fail_state(error err);
......
......@@ -7,41 +7,33 @@
#include <atomic>
#include <condition_variable>
#include <mutex>
#include <thread>
#include "caf/detail/core_export.hpp"
#include "caf/detail/private_thread_pool.hpp"
#include "caf/fwd.hpp"
namespace caf::detail {
class private_thread {
class private_thread : public private_thread_pool::node {
public:
enum worker_state { active, shutdown_requested, await_resume_or_shutdown };
void resume(resumable* ptr);
explicit private_thread(scheduled_actor* self);
bool stop() override;
void run();
static private_thread* launch(actor_system* sys);
bool await_resume();
void resume();
void shutdown();
static void exec(private_thread* this_ptr);
void notify_self_destroyed();
private:
void run(actor_system* sys);
void await_self_destroyed();
static void exec(actor_system* sys, private_thread* this_ptr);
void start();
std::pair<resumable*, bool> await();
private:
std::thread thread_;
std::mutex mtx_;
std::condition_variable cv_;
std::atomic<bool> self_destroyed_;
std::atomic<scheduled_actor*> self_;
std::atomic<worker_state> state_;
actor_system& system_;
resumable* job_ = nullptr;
bool shutdown_ = false;
};
} // namespace caf::detail
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2021 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <atomic>
#include <condition_variable>
#include <forward_list>
#include <mutex>
#include <thread>
#include "caf/fwd.hpp"
namespace caf::detail {
class private_thread_pool {
public:
struct node {
virtual ~node();
node* next = nullptr;
// Called by the private thread pool to stop the node. Regular nodes should
// return true. Returning false signals the thread pool to shut down.
virtual bool stop() = 0;
};
explicit private_thread_pool(actor_system* sys) : sys_(sys), running_(0) {
// nop
}
void start();
void stop();
void run_loop();
private_thread* acquire();
void release(private_thread*);
size_t running() const noexcept;
private:
std::pair<node*, size_t> dequeue();
actor_system* sys_;
std::thread loop_;
mutable std::mutex mtx_;
std::condition_variable cv_;
node* head_ = nullptr;
size_t running_;
};
} // namespace caf::detail
......@@ -99,7 +99,6 @@ private:
auto job = policy_.dequeue(this);
CAF_ASSERT(job != nullptr);
CAF_ASSERT(job->subtype() != resumable::io_actor);
CAF_PUSH_AID_FROM_PTR(dynamic_cast<abstract_actor*>(job));
policy_.before_resume(this, job);
auto res = job->resume(this, max_throughput_);
policy_.after_resume(this, job);
......
......@@ -272,10 +272,10 @@ actor_system::actor_system(actor_system_config& cfg)
groups_(*this),
dummy_execution_unit_(this),
await_actors_before_shutdown_(true),
detached_(0),
cfg_(cfg),
logger_dtor_done_(false),
tracing_context_(cfg.tracing_context) {
tracing_context_(cfg.tracing_context),
private_threads_(this) {
CAF_SET_LOGGER_SYS(this);
for (auto& hook : cfg.thread_hooks_)
hook->init(*this);
......@@ -358,6 +358,7 @@ actor_system::actor_system(actor_system_config& cfg)
config_serv(actor_cast<strong_actor_ptr>(spawn<Flags>(config_serv_impl)));
// Start all modules.
registry_.start();
private_threads_.start();
registry_.put("SpawnServ", spawn_serv());
registry_.put("ConfigServ", config_serv());
for (auto& mod : modules_)
......@@ -392,7 +393,7 @@ actor_system::~actor_system() {
ptr->stop();
}
}
await_detached_threads();
private_threads_.stop();
registry_.stop();
}
// reset logger and wait until dtor was called
......@@ -496,20 +497,8 @@ actor_clock& actor_system::clock() noexcept {
return scheduler().clock();
}
void actor_system::inc_detached_threads() {
++detached_;
}
void actor_system::dec_detached_threads() {
std::unique_lock<std::mutex> guard{detached_mtx_};
if (--detached_ == 0)
detached_cv_.notify_all();
}
void actor_system::await_detached_threads() {
std::unique_lock<std::mutex> guard{detached_mtx_};
while (detached_ != 0)
detached_cv_.wait(guard);
size_t actor_system::detached_actors() const noexcept {
return private_threads_.running();
}
void actor_system::thread_started() {
......@@ -543,4 +532,12 @@ actor_system::dyn_spawn_impl(const std::string& name, message& args,
return std::move(res.first);
}
detail::private_thread* actor_system::acquire_private_thread() {
return private_threads_.acquire();
}
void actor_system::release_private_thread(detail::private_thread* ptr) {
private_threads_.release(ptr);
}
} // namespace caf
......@@ -10,6 +10,7 @@
#include "caf/actor_system.hpp"
#include "caf/detail/default_invoke_result_visitor.hpp"
#include "caf/detail/invoke_result_visitor.hpp"
#include "caf/detail/private_thread.hpp"
#include "caf/detail/set_thread_name.hpp"
#include "caf/detail/sync_request_bouncer.hpp"
#include "caf/invoke_message_result.hpp"
......@@ -84,49 +85,75 @@ const char* blocking_actor::name() const {
return "user.blocking-actor";
}
namespace {
// Runner for passing a blocking actor to a private_thread. We don't actually
// need a reference count here, because the private thread calls
// intrusive_ptr_release_impl exactly once after running this function object.
class blocking_actor_runner : public resumable {
public:
explicit blocking_actor_runner(blocking_actor* self,
detail::private_thread* thread)
: self_(self), thread_(thread) {
intrusive_ptr_add_ref(self->ctrl());
}
resumable::subtype_t subtype() const override {
return resumable::function_object;
}
resumable::resume_result resume(execution_unit* ctx, size_t) override {
CAF_PUSH_AID_FROM_PTR(self_);
self_->context(ctx);
self_->initialize();
error rsn;
#ifdef CAF_ENABLE_EXCEPTIONS
try {
self_->act();
rsn = self_->fail_state();
} catch (...) {
auto ptr = std::current_exception();
rsn = scheduled_actor::default_exception_handler(self_, ptr);
}
try {
self_->on_exit();
} catch (...) {
// simply ignore exception
}
#else
self_->act();
rsn = self_->fail_state();
self_->on_exit();
#endif
self_->cleanup(std::move(rsn), ctx);
intrusive_ptr_release(self_->ctrl());
ctx->system().release_private_thread(thread_);
return resumable::done;
}
void intrusive_ptr_add_ref_impl() override {
// nop
}
void intrusive_ptr_release_impl() override {
delete this;
}
private:
blocking_actor* self_;
detail::private_thread* thread_;
};
} // namespace
void blocking_actor::launch(execution_unit*, bool, bool hide) {
CAF_PUSH_AID_FROM_PTR(this);
CAF_LOG_TRACE(CAF_ARG(hide));
CAF_ASSERT(getf(is_blocking_flag));
if (!hide)
register_at_system();
home_system().inc_detached_threads();
std::thread(
[](strong_actor_ptr ptr) {
// actor lives in its own thread
detail::set_thread_name("caf.actor");
ptr->home_system->thread_started();
auto this_ptr = ptr->get();
CAF_ASSERT(dynamic_cast<blocking_actor*>(this_ptr) != nullptr);
auto self = static_cast<blocking_actor*>(this_ptr);
CAF_SET_LOGGER_SYS(ptr->home_system);
CAF_PUSH_AID_FROM_PTR(self);
self->initialize();
error rsn;
#ifdef CAF_ENABLE_EXCEPTIONS
try {
self->act();
rsn = self->fail_state_;
} catch (...) {
auto eptr = std::current_exception();
rsn = scheduled_actor::default_exception_handler(self, eptr);
}
try {
self->on_exit();
} catch (...) {
// simply ignore exception
}
#else
self->act();
rsn = self->fail_state_;
self->on_exit();
#endif
self->cleanup(std::move(rsn), self->context());
ptr->home_system->thread_terminates();
ptr->home_system->dec_detached_threads();
},
strong_actor_ptr{ctrl()})
.detach();
auto thread = home_system().acquire_private_thread();
thread->resume(new blocking_actor_runner(this, thread));
}
blocking_actor::receive_while_helper
......
......@@ -4,102 +4,75 @@
#include "caf/detail/private_thread.hpp"
#include "caf/actor_system.hpp"
#include "caf/config.hpp"
#include "caf/detail/set_thread_name.hpp"
#include "caf/logger.hpp"
#include "caf/scheduled_actor.hpp"
#include "caf/resumable.hpp"
#include "caf/scoped_execution_unit.hpp"
namespace caf::detail {
private_thread::private_thread(scheduled_actor* self)
: self_destroyed_(false),
self_(self),
state_(active),
system_(self->system()) {
intrusive_ptr_add_ref(self->ctrl());
system_.inc_detached_threads();
}
void private_thread::run() {
auto job = self_.load();
CAF_ASSERT(job != nullptr);
CAF_SET_LOGGER_SYS(&job->system());
CAF_PUSH_AID(job->id());
void private_thread::run(actor_system* sys) {
CAF_LOG_TRACE("");
scoped_execution_unit ctx{&job->system()};
auto max_throughput = std::numeric_limits<size_t>::max();
bool resume_later;
scoped_execution_unit ctx{sys};
auto resume = [&ctx](resumable* job) {
auto res = job->resume(&ctx, std::numeric_limits<size_t>::max());
while (res == resumable::resume_later)
res = job->resume(&ctx, std::numeric_limits<size_t>::max());
return res;
};
for (;;) {
state_ = await_resume_or_shutdown;
do {
resume_later = false;
switch (job->resume(&ctx, max_throughput)) {
case resumable::resume_later:
resume_later = true;
break;
case resumable::done:
intrusive_ptr_release(job->ctrl());
return;
case resumable::awaiting_message:
intrusive_ptr_release(job->ctrl());
break;
case resumable::shutdown_execution_unit:
return;
}
} while (resume_later);
// wait until actor becomes ready again or was destroyed
if (!await_resume())
auto [job, done] = await();
if (job) {
CAF_ASSERT(job->subtype() != resumable::io_actor);
resume(job);
intrusive_ptr_release(job);
}
if (done)
return;
}
}
bool private_thread::await_resume() {
std::unique_lock<std::mutex> guard(mtx_);
while (state_ == await_resume_or_shutdown)
cv_.wait(guard);
return state_ == active;
}
void private_thread::resume() {
std::unique_lock<std::mutex> guard(mtx_);
state_ = active;
cv_.notify_one();
}
void private_thread::shutdown() {
std::unique_lock<std::mutex> guard(mtx_);
state_ = shutdown_requested;
cv_.notify_one();
void private_thread::resume(resumable* ptr) {
std::unique_lock<std::mutex> guard{mtx_};
CAF_ASSERT(job_ == nullptr);
job_ = ptr;
cv_.notify_all();
}
void private_thread::exec(private_thread* this_ptr) {
detail::set_thread_name("caf.actor");
this_ptr->system_.thread_started();
this_ptr->run();
// make sure to not destroy the private thread object before the
// detached actor is destroyed and this object is unreachable
this_ptr->await_self_destroyed();
// signalize destruction of detached thread to registry
this_ptr->system_.thread_terminates();
this_ptr->system_.dec_detached_threads();
// done
delete this_ptr;
bool private_thread::stop() {
{
std::unique_lock<std::mutex> guard{mtx_};
shutdown_ = true;
cv_.notify_all();
}
thread_.join();
return true;
}
void private_thread::notify_self_destroyed() {
std::pair<resumable*, bool> private_thread::await() {
std::unique_lock<std::mutex> guard(mtx_);
self_destroyed_ = true;
cv_.notify_one();
while (job_ == nullptr && !shutdown_)
cv_.wait(guard);
auto ptr = job_;
if (ptr)
job_ = nullptr;
return {ptr, shutdown_};
}
void private_thread::await_self_destroyed() {
std::unique_lock<std::mutex> guard(mtx_);
while (!self_destroyed_)
cv_.wait(guard);
private_thread* private_thread::launch(actor_system* sys) {
auto ptr = std::make_unique<private_thread>();
ptr->thread_ = std::thread{exec, sys, ptr.get()};
return ptr.release();
}
void private_thread::start() {
std::thread{exec, this}.detach();
void private_thread::exec(actor_system* sys, private_thread* this_ptr) {
CAF_SET_LOGGER_SYS(sys);
detail::set_thread_name("caf.thread");
sys->thread_started();
this_ptr->run(sys);
sys->thread_terminates();
}
} // namespace caf::detail
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2021 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/private_thread_pool.hpp"
#include "caf/config.hpp"
#include "caf/detail/private_thread.hpp"
namespace caf::detail {
private_thread_pool::node::~node() {
// nop
}
void private_thread_pool::start() {
loop_ = std::thread{[](private_thread_pool* ptr) { ptr->run_loop(); }, this};
}
void private_thread_pool::stop() {
struct shutdown_helper : node {
bool stop() override {
return false;
}
};
auto ptr = new shutdown_helper;
{
std::unique_lock guard{mtx_};
++running_;
ptr->next = head_;
head_ = ptr;
cv_.notify_all();
}
loop_.join();
}
void private_thread_pool::run_loop() {
bool shutting_down = false;
for (;;) {
auto [ptr, remaining] = dequeue();
CAF_ASSERT(ptr != nullptr);
if (!ptr->stop())
shutting_down = true;
delete ptr;
if (remaining == 0 && shutting_down)
return;
}
}
private_thread* private_thread_pool::acquire() {
{
std::unique_lock guard{mtx_};
++running_;
}
return private_thread::launch(sys_);
}
void private_thread_pool::release(private_thread* ptr) {
std::unique_lock guard{mtx_};
ptr->next = head_;
head_ = ptr;
cv_.notify_all();
}
size_t private_thread_pool::running() const noexcept {
std::unique_lock guard{mtx_};
return running_;
}
std::pair<private_thread_pool::node*, size_t> private_thread_pool::dequeue() {
std::unique_lock guard{mtx_};
while (head_ == nullptr)
cv_.wait(guard);
auto ptr = head_;
head_ = ptr->next;
auto remaining = --running_;
return {ptr, remaining};
}
} // namespace caf::detail
......@@ -129,10 +129,7 @@ scheduled_actor::scheduled_actor(actor_config& cfg)
}
scheduled_actor::~scheduled_actor() {
// signalize to the private thread object that it is
// unreachable and can be destroyed as well
if (private_thread_ != nullptr)
private_thread_->notify_self_destroyed();
// nop
}
// -- overridden functions of abstract_actor -----------------------------------
......@@ -152,17 +149,13 @@ void scheduled_actor::enqueue(mailbox_element_ptr ptr, execution_unit* eu) {
switch (mailbox().push_back(std::move(ptr))) {
case intrusive::inbox_result::unblocked_reader: {
CAF_LOG_ACCEPT_EVENT(true);
// add a reference count to this actor and re-schedule it
intrusive_ptr_add_ref(ctrl());
if (getf(is_detached_flag)) {
CAF_ASSERT(private_thread_ != nullptr);
private_thread_->resume();
} else {
if (eu != nullptr)
eu->exec_later(this);
else
home_system().scheduler().enqueue(this);
}
if (private_thread_)
private_thread_->resume(this);
else if (eu != nullptr)
eu->exec_later(this);
else
home_system().scheduler().enqueue(this);
break;
}
case intrusive::inbox_result::queue_closed: {
......@@ -192,35 +185,31 @@ const char* scheduled_actor::name() const {
return "user.scheduled-actor";
}
void scheduled_actor::launch(execution_unit* eu, bool lazy, bool hide) {
void scheduled_actor::launch(execution_unit* ctx, bool lazy, bool hide) {
CAF_ASSERT(ctx != nullptr);
CAF_PUSH_AID_FROM_PTR(this);
CAF_LOG_TRACE(CAF_ARG(lazy) << CAF_ARG(hide));
CAF_ASSERT(!getf(is_blocking_flag));
if (!hide)
register_at_system();
auto delay_first_scheduling = lazy && mailbox().try_block();
if (getf(is_detached_flag)) {
private_thread_ = new detail::private_thread(this);
private_thread_->start();
return;
private_thread_ = ctx->system().acquire_private_thread();
if (!delay_first_scheduling) {
intrusive_ptr_add_ref(ctrl());
private_thread_->resume(this);
}
} else if (!delay_first_scheduling) {
intrusive_ptr_add_ref(ctrl());
ctx->exec_later(this);
}
CAF_ASSERT(eu != nullptr);
// do not schedule immediately when spawned with `lazy_init`
// mailbox could be set to blocked
if (lazy && mailbox().try_block())
return;
// scheduler has a reference count to the actor as long as
// it is waiting to get scheduled
intrusive_ptr_add_ref(ctrl());
eu->exec_later(this);
}
bool scheduled_actor::cleanup(error&& fail_state, execution_unit* host) {
CAF_LOG_TRACE(CAF_ARG(fail_state));
// Shutdown hosting thread when running detached.
if (getf(is_detached_flag)) {
CAF_ASSERT(private_thread_ != nullptr);
private_thread_->shutdown();
}
if (private_thread_)
home_system().release_private_thread(private_thread_);
// Clear state for open requests.
awaited_responses_.clear();
multiplexed_responses_.clear();
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2021 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#define CAF_SUITE detail.private_thread_pool
#include "caf/detail/private_thread_pool.hpp"
#include "core-test.hpp"
#include "caf/detail/private_thread.hpp"
using namespace caf;
CAF_TEST_FIXTURE_SCOPE(private_thread_pool_tests, test_coordinator_fixture<>)
SCENARIO("private threads count towards detached actors") {
GIVEN("an actor system with a private thread pool") {
detail::private_thread* t1 = nullptr;
detail::private_thread* t2 = nullptr;
WHEN("acquiring new private threads") {
THEN("the detached_actors counter increases") {
CHECK_EQ(sys.detached_actors(), 0u);
t1 = sys.acquire_private_thread();
CHECK_EQ(sys.detached_actors(), 1u);
t2 = sys.acquire_private_thread();
CHECK_EQ(sys.detached_actors(), 2u);
}
}
WHEN("releasing the private threads") {
THEN("the detached_actors counter eventually decreases again") {
auto next_value = [this, old_value{2u}]() mutable {
using namespace std::literals::chrono_literals;
size_t result = 0;
while ((result = sys.detached_actors()) == old_value)
std::this_thread::sleep_for(1ms);
old_value = result;
return result;
};
sys.release_private_thread(t2);
CHECK_EQ(next_value(), 1u);
sys.release_private_thread(t1);
CHECK_EQ(next_value(), 0u);
}
}
}
}
SCENARIO("private threads rerun their resumable when it returns resume_later") {
struct testee : resumable {
std::atomic<size_t> runs = 0;
std::atomic<size_t> refs_added = 0;
std::atomic<size_t> refs_released = 0;
subtype_t subtype() const override {
return resumable::function_object;
}
resume_result resume(execution_unit*, size_t) override {
return ++runs < 2 ? resumable::resume_later : resumable::done;
}
void intrusive_ptr_add_ref_impl() override {
++refs_added;
}
void intrusive_ptr_release_impl() override {
++refs_released;
}
};
GIVEN("a resumable f and a private thread t") {
testee f;
auto t = sys.acquire_private_thread();
WHEN("when resuming f with t") {
t->resume(&f);
THEN("t calls resume until f returns something other than resume_later") {
using namespace std::literals::chrono_literals;
sys.release_private_thread(t);
while (sys.detached_actors() != 0)
std::this_thread::sleep_for(1ms);
CHECK_EQ(f.runs, 2u);
CHECK_EQ(f.refs_added, 0u);
CHECK_EQ(f.refs_released, 1u);
}
}
}
}
CAF_TEST_FIXTURE_SCOPE_END()
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment