diff --git a/requirements_frozen.txt b/requirements_frozen.txt index 161cfca7074..e7f06dc22ec 100644 --- a/requirements_frozen.txt +++ b/requirements_frozen.txt @@ -1,47 +1,44 @@ -attrs==21.4.0 +Deprecated==1.2.14 +PyGithub==2.1.1 +PyJWT==2.8.0 +PyNaCl==1.5.0 autorepr==0.3.0 -bashlex==0.16 +bashlex==0.18 boto==2.49.0 -botocore==1.4.59 -certifi==2021.5.30 -cffi==1.14.6 -charset-normalizer==2.0.6 -click==8.0.1 -codecheck==1.1.3 +botocore==1.31.70 +certifi==2023.7.22 +cffi==1.16.0 +charset-normalizer==3.3.1 +click==8.1.7 +codecheck==1.3.1 compiledb==0.10.1 compiler-identification==1.0.3 -Deprecated==1.2.13 -distro==1.6.0 -downloadutil==1.0.2 -idna==3.2 -importlib-metadata==4.8.3 -iniconfig==1.1.1 -mypy==0.910 -mypy-extensions==0.4.3 -overrides==6.1.0 -packaging==21.0 -pluggy==1.0.0 -psutil==5.8.0 -py==1.11.0 -pycodestyle==2.7.0 -pycparser==2.20 -PyGithub==1.55 -PyJWT==2.1.0 -PyNaCl==1.4.0 -pyparsing==2.4.7 -pytest==6.2.5 -requests==2.26.0 -ruamel.yaml==0.17.16 -ruamel.yaml.clib==0.2.6 -semantic-version==2.8.5 +cryptography==41.0.5 +distro==1.8.0 +downloadutil==1.0.4 +exceptiongroup==1.1.3 +idna==3.4 +iniconfig==2.0.0 +jmespath==1.0.1 +mypy-extensions==1.0.0 +mypy==1.6.1 +overrides==7.4.0 +packaging==23.2 +pluggy==1.3.0 +psutil==5.9.6 +pycodestyle==2.11.1 +pycparser==2.21 +pytest==7.4.3 +python-dateutil==2.8.2 +requests==2.31.0 +ruamel.yaml.clib==0.2.8 +ruamel.yaml==0.18.2 +semantic-version==2.10.0 shutilwhich==1.1.0 six==1.16.0 -sys-detection==1.2.0 -toml==0.10.2 -typed-ast==1.5.1 -typing-extensions==3.10.0.2 -typing-utils==0.1.0 -urllib3==1.26.7 -wrapt==1.12.1 +sys-detection==1.3.0 +tomli==2.0.1 +typing_extensions==4.8.0 +urllib3==1.26.18 +wrapt==1.15.0 yugabyte-pycommon==1.9.15 -zipp==3.6.0 diff --git a/src/yb/docdb/deadline_info.cc b/src/yb/docdb/deadline_info.cc index 4be993d5e40..3000f890a32 100644 --- a/src/yb/docdb/deadline_info.cc +++ b/src/yb/docdb/deadline_info.cc @@ -45,8 +45,9 @@ bool DeadlineInfo::CheckAndSetDeadlinePassed() { } std::string DeadlineInfo::ToString() const { + auto now = CoarseMonoClock::now(); return Format("{ now: $0 deadline: $1 counter: $2 }", - CoarseMonoClock::now(), deadline_, counter_); + now, ToStringRelativeToNow(deadline_, now), counter_); } void SimulateTimeoutIfTesting(CoarseTimePoint* deadline) { diff --git a/src/yb/master/async_rpc_tasks.cc b/src/yb/master/async_rpc_tasks.cc index 2a17e0db193..5ed9406eacc 100644 --- a/src/yb/master/async_rpc_tasks.cc +++ b/src/yb/master/async_rpc_tasks.cc @@ -388,17 +388,18 @@ bool RetryingTSRpcTask::RescheduleWithBackoffDelay() { LOG_WITH_PREFIX(WARNING) << "Unable to mark this task as MonitoredTaskState::kScheduling"; return false; } - auto task_id = master_->messenger()->ScheduleOnReactor( + auto task_id_result = master_->messenger()->ScheduleOnReactor( std::bind(&RetryingTSRpcTask::RunDelayedTask, shared_from(this), _1), - MonoDelta::FromMilliseconds(delay_millis), SOURCE_LOCATION(), master_->messenger()); - VLOG_WITH_PREFIX_AND_FUNC(4) << "Task id: " << task_id; - reactor_task_id_.store(task_id, std::memory_order_release); - - if (task_id == rpc::kInvalidTaskId) { - AbortTask(STATUS(Aborted, "Messenger closing")); + MonoDelta::FromMilliseconds(delay_millis), SOURCE_LOCATION()); + if (!task_id_result.ok()) { + AbortTask(task_id_result.status()); UnregisterAsyncTask(); return false; } + auto task_id = *task_id_result; + + VLOG_WITH_PREFIX_AND_FUNC(4) << "Task id: " << task_id; + reactor_task_id_.store(task_id, std::memory_order_release); return TransitionToWaitingState(MonitoredTaskState::kScheduling); } diff --git a/src/yb/rpc/binary_call_parser.h b/src/yb/rpc/binary_call_parser.h index e9bacb0f4f8..e4f64b3b498 100644 --- a/src/yb/rpc/binary_call_parser.h +++ b/src/yb/rpc/binary_call_parser.h @@ -20,6 +20,7 @@ #include "yb/rpc/rpc_fwd.h" #include "yb/rpc/call_data.h" +#include "yb/rpc/reactor_thread_role.h" namespace yb { namespace rpc { @@ -47,7 +48,7 @@ class BinaryCallParser { // (or any of its ancestors) exceeds soft memory limit. Result Parse(const rpc::ConnectionPtr& connection, const IoVecs& data, ReadBufferFull read_buffer_full, - const MemTrackerPtr* tracker_for_throttle); + const MemTrackerPtr* tracker_for_throttle) ON_REACTOR_THREAD; private: MemTrackerPtr buffer_tracker_; diff --git a/src/yb/rpc/compressed_stream.cc b/src/yb/rpc/compressed_stream.cc index 8da691df3ef..eef4879f970 100644 --- a/src/yb/rpc/compressed_stream.cc +++ b/src/yb/rpc/compressed_stream.cc @@ -26,6 +26,7 @@ #include "yb/rpc/circular_read_buffer.h" #include "yb/rpc/outbound_data.h" #include "yb/rpc/refined_stream.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/logging.h" #include "yb/util/result.h" @@ -53,7 +54,8 @@ class Compressor { // Compress specified vector of input buffers into single output buffer. virtual Status Compress( - const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) = 0; + const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) + ON_REACTOR_THREAD = 0; // Decompress specified input slice to specified output buffer. virtual Result Decompress(StreamReadBuffer* inp, StreamReadBuffer* out) = 0; @@ -177,7 +179,8 @@ class ZlibCompressor : public Compressor { } Status Compress( - const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) override { + const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) + ON_REACTOR_THREAD override { RefCntBuffer output(deflateBound(&deflate_stream_, TotalLen(input))); deflate_stream_.avail_out = static_cast(output.size()); deflate_stream_.next_out = output.udata(); @@ -398,7 +401,8 @@ class SnappyCompressor : public Compressor { } Status Compress( - const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) override { + const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) + ON_REACTOR_THREAD override { RangeSource source(input.begin(), input.end()); auto input_size = source.Available(); bool stop = false; @@ -685,7 +689,8 @@ class LZ4Compressor : public Compressor { } Status Compress( - const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) override { + const SmallRefCntBuffers& input, RefinedStream* stream, OutboundDataPtr data) + ON_REACTOR_THREAD override { // Increment iterator in loop body to be able to check whether it is last iteration or not. VLOG_WITH_FUNC(4) << "input: " << CollectionToString(input, [](const auto& buf) { return buf.size(); @@ -772,7 +777,7 @@ class CompressedRefiner : public StreamRefiner { stream_ = stream; } - Status ProcessHeader() override { + Status ProcessHeader() ON_REACTOR_THREAD override { constexpr int kHeaderLen = 3; auto data = stream_->ReadBuffer().AppendedVecs(); @@ -797,13 +802,13 @@ class CompressedRefiner : public StreamRefiner { return stream_->Established(RefinedStreamState::kDisabled); } - Status Send(OutboundDataPtr data) override { + Status Send(OutboundDataPtr data) ON_REACTOR_THREAD override { boost::container::small_vector input; data->Serialize(&input); return compressor_->Compress(input, stream_, std::move(data)); } - Status Handshake() override { + Status Handshake() ON_REACTOR_THREAD override { if (stream_->local_side() == LocalSide::kClient) { compressor_ = CreateOutboundCompressor(stream_->buffer_tracker()); if (!compressor_) { diff --git a/src/yb/rpc/connection.cc b/src/yb/rpc/connection.cc index 4678260ebc2..7f88e2c3337 100644 --- a/src/yb/rpc/connection.cc +++ b/src/yb/rpc/connection.cc @@ -47,9 +47,12 @@ #include "yb/rpc/rpc_controller.h" #include "yb/rpc/rpc_introspection.pb.h" #include "yb/rpc/rpc_metrics.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/debug-util.h" #include "yb/util/enums.h" +#include "yb/util/flag_tags.h" +#include "yb/util/flags.h" #include "yb/util/format.h" #include "yb/util/logging.h" #include "yb/util/metrics.h" @@ -58,6 +61,7 @@ #include "yb/util/string_util.h" #include "yb/util/trace.h" #include "yb/util/tsan_util.h" +#include "yb/util/unique_lock.h" using namespace std::literals; using namespace std::placeholders; @@ -73,12 +77,53 @@ METRIC_DEFINE_histogram_with_percentiles( yb::MetricUnit::kMicroseconds, "Microseconds spent to queue and write the response to the wire", 60000000LU, 2); +DEFINE_test_flag(double, simulated_sent_stuck_call_probability, 0.0, + "Probability of a simulated stuck call in SENT state. We set the state to SENT without " + "actually sending the call."); + +DEFINE_test_flag(double, simulated_failure_to_send_call_probability, 0.0, + "Probability of a simulated failure to send a call's data, which should result in connection " + "being closed."); + + namespace yb { namespace rpc { -/// -/// Connection -/// +namespace { + +template +void ActiveCallExpired( + ContainerIndex& index, + typename ContainerIndex::iterator iter, + Reactor* reactor, + Stream* stream) ON_REACTOR_THREAD { + auto call = iter->call; + if (!call) { + LOG(DFATAL) << __func__ << ": call is null in " << iter->ToString(); + return; + } + auto handle = iter->handle; + auto erase = false; + if (!call->IsFinished()) { + call->SetTimedOut(); + if (handle != kUnknownCallHandle) { + erase = stream->Cancelled(handle); + } + } + if (erase) { + index.erase(iter); + call->SetActiveCallState(ActiveCallState::kErasedOnExpiration); + } else { + index.modify(iter, [](auto& active_call) { + active_call.call.reset(); + active_call.expires_at = CoarseTimePoint::max(); + }); + call->SetActiveCallState(ActiveCallState::kResetOnExpiration); + } +} + +} // anonymous namespace + Connection::Connection(Reactor* reactor, std::unique_ptr stream, Direction direction, @@ -87,18 +132,18 @@ Connection::Connection(Reactor* reactor, : reactor_(reactor), stream_(std::move(stream)), direction_(direction), - last_activity_time_(CoarseMonoClock::Now()), - rpc_metrics_(rpc_metrics), - context_(std::move(context)) { - const auto metric_entity = reactor->messenger()->metric_entity(); + rpc_metrics_(*rpc_metrics), + context_(std::move(context)), + last_activity_time_(CoarseMonoClock::Now()) { + const auto metric_entity = reactor->messenger().metric_entity(); handler_latency_outbound_transfer_ = metric_entity ? METRIC_handler_latency_outbound_transfer.Instantiate(metric_entity) : nullptr; - IncrementCounter(rpc_metrics_->connections_created); - IncrementGauge(rpc_metrics_->connections_alive); + IncrementCounter(rpc_metrics_.connections_created); + IncrementGauge(rpc_metrics_.connections_alive); } Connection::~Connection() { - DecrementGauge(rpc_metrics_->connections_alive); + DecrementGauge(rpc_metrics_.connections_alive); } void UpdateIdleReason(const char* message, bool* result, std::string* reason) { @@ -140,12 +185,23 @@ void Connection::Shutdown(const Status& provided_status) { provided_status.ok() ? STATUS_FORMAT(RuntimeError, "Connection shutdown called with OK status") : provided_status; + { std::vector outbound_data_being_processed; { - std::lock_guard lock(outbound_data_queue_lock_); + std::lock_guard lock(outbound_data_queue_mtx_); + + // Perform this compare-and-set when holding outbound_data_queue_mtx_ so that + // ShutdownStatus() would retrieve the correct status. + if (shutdown_initiated_.exchange(true, std::memory_order_release)) { + LOG_WITH_PREFIX(WARNING) + << "Connection shutdown invoked multiple times. Previously with status " + << shutdown_status_ << " and now with status " << provided_status + << ", completed=" << shutdown_completed() << ". Skipping repeated shutdown."; + return; + } - outbound_data_being_processed.swap(outbound_data_to_process_); + outbound_data_being_processed = std::move(outbound_data_to_process_); shutdown_status_ = status; } @@ -163,8 +219,11 @@ void Connection::Shutdown(const Status& provided_status) { // Clear any calls which have been sent and were awaiting a response. active_calls_during_shutdown_.store(active_calls_.size(), std::memory_order_release); for (auto& v : active_calls_) { - if (v.call && !v.call->IsFinished()) { - v.call->SetFailed(status); + if (v.call) { + if (!v.call->IsFinished()) { + v.call->SetFailed(status); + } + v.call->SetActiveCallState(ActiveCallState::kErasedOnConnectionShutdown); } } active_calls_.clear(); @@ -173,21 +232,18 @@ void Connection::Shutdown(const Status& provided_status) { // TODO(bogdan): re-enable once we decide how to control verbose logs better... // LOG_WITH_PREFIX(INFO) << "Connection::Shutdown completed, status: " << status; + shutdown_completed_.store(true, std::memory_order_release); } -void Connection::OutboundQueued() { +Status Connection::OutboundQueued() { DCHECK(reactor_->IsCurrentThread()); - + RETURN_NOT_OK(ShutdownStatus()); auto status = stream_->TryWrite(); if (!status.ok()) { VLOG_WITH_PREFIX(1) << "Write failed: " << status; - if (!queued_destroy_connection_.exchange(true, std::memory_order_acq_rel)) { - auto scheduled = reactor_->ScheduleReactorTask(MakeFunctorReactorTask( - std::bind(&Reactor::DestroyConnection, reactor_, this, status), shared_from_this(), - SOURCE_LOCATION())); - LOG_IF_WITH_PREFIX(WARNING, !scheduled) << "Failed to schedule destroy"; - } + ScheduleDestroyConnection(status); } + return status; } void Connection::HandleTimeout(ev::timer& watcher, int revents) { // NOLINT @@ -205,10 +261,12 @@ void Connection::HandleTimeout(ev::timer& watcher, int revents) { // NOLINT CoarseTimePoint deadline = CoarseTimePoint::max(); if (!stream_->IsConnected()) { const MonoDelta timeout = FLAGS_rpc_connection_timeout_ms * 1ms; - deadline = last_activity_time_ + timeout; - DVLOG_WITH_PREFIX(5) << Format("now: $0, deadline: $1, timeout: $2", now, deadline, timeout); + auto current_last_activity_time = last_activity_time(); + deadline = current_last_activity_time + timeout; + DVLOG_WITH_PREFIX(5) << Format( + "now: $0, deadline: $1, timeout: $2", now, ToStringRelativeToNow(deadline, now), timeout); if (now > deadline) { - auto passed = reactor_->cur_time() - last_activity_time_; + auto passed = reactor_->cur_time() - current_last_activity_time; reactor_->DestroyConnection( this, STATUS_EC_FORMAT(NetworkError, NetworkError(NetworkErrorCode::kConnectFailed), @@ -232,24 +290,7 @@ void Connection::HandleTimeout(ev::timer& watcher, int revents) { // NOLINT void Connection::CleanupExpirationQueue(CoarseTimePoint now) { auto& index = active_calls_.get(); while (!index.empty() && index.begin()->expires_at <= now) { - auto& top = *index.begin(); - auto call = top.call; - auto handle = top.handle; - auto erase = false; - if (!call->IsFinished()) { - call->SetTimedOut(); - if (handle != std::numeric_limits::max()) { - erase = stream_->Cancelled(handle); - } - } - if (erase) { - index.erase(index.begin()); - } else { - index.modify(index.begin(), [](auto& active_call) { - active_call.call.reset(); - active_call.expires_at = CoarseTimePoint::max(); - }); - } + ActiveCallExpired(index, index.begin(), reactor_, stream_.get()); } } @@ -257,7 +298,23 @@ void Connection::QueueOutboundCall(const OutboundCallPtr& call) { DCHECK(call); DCHECK_EQ(direction_, Direction::CLIENT); - auto handle = DoQueueOutboundData(call, true); + size_t handle; + + const bool simulate_stuck_sent_call = + RandomActWithProbability(FLAGS_TEST_simulated_sent_stuck_call_probability); + if (simulate_stuck_sent_call) { + handle = kUnknownCallHandle; + call->SetQueued(); + auto _ [[maybe_unused]] = call->SetSent(); // NOLINT + LOG_WITH_PREFIX(WARNING) << "Simulating a call stuck in SENT state: " << call->DebugString(); + } else { + auto queue_result = DoQueueOutboundData(call, /* batch= */ true); + if (!queue_result.ok()) { + // The connection has been shut down by this point, and the callback has been called. + return; + } + handle = *queue_result; + } // Set up the timeout timer. MonoDelta timeout = call->controller()->timeout(); @@ -265,38 +322,66 @@ void Connection::QueueOutboundCall(const OutboundCallPtr& call) { if (timeout.Initialized()) { auto now = CoarseMonoClock::Now(); expires_at = now + timeout.ToSteadyDuration(); + call->SetExpiration(expires_at); auto reschedule = active_calls_.empty() || active_calls_.get().begin()->expires_at > expires_at; CleanupExpirationQueue(now); if (reschedule && (stream_->IsConnected() || - expires_at < last_activity_time_ + FLAGS_rpc_connection_timeout_ms * 1ms)) { + expires_at < last_activity_time() + FLAGS_rpc_connection_timeout_ms * 1ms)) { timer_.Start(timeout.ToSteadyDuration()); } } else { // Call never expires. expires_at = CoarseTimePoint::max(); } - active_calls_.insert(ActiveCall { - .id = call->call_id(), - .call = call, - .expires_at = expires_at, - .handle = handle, - }); + if (!PREDICT_FALSE(simulate_stuck_sent_call)) { + call->SetActiveCallState(ActiveCallState::kAdded); + active_calls_.insert(ActiveCall { + .id = call->call_id(), + .call = call, + .expires_at = expires_at, + .handle = handle, + }); + + call->SetQueued(); + } +} - call->SetQueued(); +void Connection::FailCallAndDestroyConnection( + const OutboundDataPtr& outbound_data, const Status& status) { + outbound_data->Transferred(status, shared_from_this()); + ScheduleDestroyConnection(status); } -size_t Connection::DoQueueOutboundData(OutboundDataPtr outbound_data, bool batch) { +void Connection::ScheduleDestroyConnection(const Status& status) { + if (!queued_destroy_connection_.exchange(true, std::memory_order_acq_rel)) { + // Even though we are already on the reactor thread, try to schedule a task so that it + // would run later than all other already scheduled tasks, to preserve historical + // behavior. + auto scheduling_status = reactor_->ScheduleReactorTask(MakeFunctorReactorTask( + std::bind(&Reactor::DestroyConnection, reactor_, this, status), shared_from_this(), + SOURCE_LOCATION())); + if (!scheduling_status.ok()) { + LOG_WITH_PREFIX(WARNING) << "Failed to schedule DestroyConnection: " << scheduling_status + << "on reactor, destroying connection immediately"; + reactor_->DestroyConnection(this, status); + } + } +} + +Result Connection::DoQueueOutboundData(OutboundDataPtr outbound_data, bool batch) { DCHECK(reactor_->IsCurrentThread()); DVLOG_WITH_PREFIX(4) << "Connection::DoQueueOutboundData: " << AsString(outbound_data); + RSTATUS_DCHECK_NOTNULL(outbound_data); - if (!shutdown_status_.ok()) { + Status shutdown_status = ShutdownStatus(); + if (!shutdown_status.ok()) { YB_LOG_EVERY_N_SECS(INFO, 5) << "Connection::DoQueueOutboundData data: " - << AsString(outbound_data) << " shutdown_status_: " - << shutdown_status_; - outbound_data->Transferred(shutdown_status_, shared_from_this()); + << AsString(outbound_data) << " shutdown_status: " + << shutdown_status; + outbound_data->Transferred(shutdown_status, shared_from_this()); calls_queued_after_shutdown_.fetch_add(1, std::memory_order_acq_rel); - return std::numeric_limits::max(); + return shutdown_status;; } // If the connection is torn down, then the QueueOutbound() call that @@ -308,22 +393,38 @@ size_t Connection::DoQueueOutboundData(OutboundDataPtr outbound_data, bool batch // check if we are now over the limit. Status s = context_->ReportPendingWriteBytes(stream_->GetPendingWriteBytes()); if (!s.ok()) { - Shutdown(s); - return std::numeric_limits::max(); + FailCallAndDestroyConnection(outbound_data, s); + return s; + } + + if (PREDICT_FALSE( + RandomActWithProbability(FLAGS_TEST_simulated_failure_to_send_call_probability))) { + s = STATUS_FORMAT( + NetworkError, "Simulated failure to send outbound data for $0", outbound_data); + LOG(WARNING) << "Simulated network failure: " << s.ToString(); + FailCallAndDestroyConnection(outbound_data, s); + return s; } - auto result = stream_->Send(std::move(outbound_data)); + + auto result = stream_->Send(outbound_data); if (!result.ok()) { - Shutdown(result.status()); - return std::numeric_limits::max(); + FailCallAndDestroyConnection(outbound_data, result.status()); + return result.status(); } + s = context_->ReportPendingWriteBytes(stream_->GetPendingWriteBytes()); if (!s.ok()) { - Shutdown(s); - return std::numeric_limits::max(); + FailCallAndDestroyConnection(outbound_data, s); + return s; } if (!batch) { - OutboundQueued(); + s = OutboundQueued(); + if (!s.ok()) { + outbound_data->Transferred(s, shared_from_this()); + // The connection shutdown has already been triggered by OutboundQueued. + return s; + } } return *result; @@ -367,8 +468,9 @@ Status Connection::HandleCallResponse(CallData* call_data) { } auto call = awaiting->call; active_calls_.erase(awaiting); - - if (PREDICT_FALSE(!call)) { + if (PREDICT_TRUE(call)) { + call->SetActiveCallState(ActiveCallState::kErasedOnResponse); + } else { // The call already failed due to a timeout. VLOG_WITH_PREFIX(1) << "Got response to call id " << resp.call_id() << " after client already timed out"; @@ -397,8 +499,9 @@ void Connection::QueueDumpConnectionState(int32_t call_id, const void* call_ptr) auto task = MakeFunctorReactorTask( std::bind(&Connection::DumpConnectionState, this, call_id, call_ptr), shared_from_this(), SOURCE_LOCATION()); - auto scheduled = reactor_->ScheduleReactorTask(task); - LOG_IF_WITH_PREFIX(DFATAL, !scheduled) << "Failed to schedule call to dump connection state."; + auto scheduling_status = reactor_->ScheduleReactorTask(task); + LOG_IF_WITH_PREFIX(DFATAL, !scheduling_status.ok()) + << "Failed to schedule call to dump connection state: " << scheduling_status; } void Connection::DumpConnectionState(int32_t call_id, const void* call_ptr) const { @@ -406,12 +509,13 @@ void Connection::DumpConnectionState(int32_t call_id, const void* call_ptr) cons ? CoarseTimePoint() : active_calls_.get().begin()->expires_at; auto found_call_id = active_calls_.find(call_id) != active_calls_.end(); + auto now = CoarseMonoClock::Now(); LOG_WITH_PREFIX(INFO) << Format( "LastActivityTime: $0, " "ActiveCalls stats: { " "during shutdown: $1, " "current size: $2, " - "earliest expiry: $3, " + "earliest expiry: $3 " "}, " "OutboundCall: { " "ptr: $4, " @@ -424,15 +528,15 @@ void Connection::DumpConnectionState(int32_t call_id, const void* call_ptr) cons "calls: $9, " "responses: $10 " "}", - /* $0 */ last_activity_time(), + /* $0 */ ToStringRelativeToNow(last_activity_time(), now), /* $1 */ active_calls_during_shutdown_.load(std::memory_order_acquire), /* $2 */ active_calls_.size(), - /* $3 */ earliest_expiry, + /* $3 */ ToStringRelativeToNow(earliest_expiry, now), /* $4 */ call_ptr, /* $5 */ call_id, /* $6 */ found_call_id, /* $7 */ shutdown_status_, - /* $8 */ shutdown_time_, + /* $8 */ ToStringRelativeToNow(shutdown_time_, now), /* $9 */ calls_queued_after_shutdown_.load(std::memory_order_acquire), /* $10 */ responses_queued_after_shutdown_.load(std::memory_order_acquire)); } @@ -472,30 +576,39 @@ void Connection::QueueOutboundDataBatch(const OutboundDataBatch& batch) { DCHECK(reactor_->IsCurrentThread()); for (const auto& call : batch) { - DoQueueOutboundData(call, /* batch */ true); + // If one of these calls fails and shuts down the connection, all calls after that will fail + // as well. + auto ignored_status = DoQueueOutboundData(call, /* batch */ true); } - OutboundQueued(); + auto ignored_status = OutboundQueued(); } -void Connection::QueueOutboundData(OutboundDataPtr outbound_data) { +Status Connection::QueueOutboundData(OutboundDataPtr outbound_data) { + CHECK_NOTNULL(outbound_data); if (reactor_->IsCurrentThread()) { - DoQueueOutboundData(std::move(outbound_data), /* batch */ false); - return; + ReactorThreadRoleGuard guard; + auto result = DoQueueOutboundData(std::move(outbound_data), /* batch */ false); + if (result.ok()) { + return Status::OK(); + } + return result.status(); } bool was_empty; + std::shared_ptr process_response_queue_task; { - std::unique_lock lock(outbound_data_queue_lock_); + UniqueLock outbound_data_queue_lock(outbound_data_queue_mtx_); if (!shutdown_status_.ok()) { auto task = MakeFunctorReactorTaskWithAbort( std::bind(&OutboundData::Transferred, outbound_data, _2, /* conn */ nullptr), SOURCE_LOCATION()); - lock.unlock(); + outbound_data_queue_lock.unlock(); responses_queued_after_shutdown_.fetch_add(1, std::memory_order_acq_rel); - auto scheduled = reactor_->ScheduleReactorTask(task, true /* schedule_even_closing */); - LOG_IF_WITH_PREFIX(DFATAL, !scheduled) << "Failed to schedule OutboundData::Transferred"; - return; + auto scheduling_status = reactor_->ScheduleReactorTask(task, true /* even_if_not_running */); + LOG_IF_WITH_PREFIX(DFATAL, !scheduling_status.ok()) + << "Failed to schedule OutboundData::Transferred: " << scheduling_status; + return scheduling_status; } was_empty = outbound_data_to_process_.empty(); outbound_data_to_process_.push_back(std::move(outbound_data)); @@ -504,30 +617,31 @@ void Connection::QueueOutboundData(OutboundDataPtr outbound_data) { MakeFunctorReactorTask(std::bind(&Connection::ProcessResponseQueue, this), shared_from_this(), SOURCE_LOCATION()); } + process_response_queue_task = process_response_queue_task_; } if (was_empty) { - // TODO: what happens if the reactor is shutting down? Currently Abort is ignored. - auto scheduled = reactor_->ScheduleReactorTask(process_response_queue_task_); - LOG_IF_WITH_PREFIX(WARNING, !scheduled) - << "Failed to schedule Connection::ProcessResponseQueue"; + return reactor_->ScheduleReactorTask(process_response_queue_task); } + return Status::OK(); } void Connection::ProcessResponseQueue() { DCHECK(reactor_->IsCurrentThread()); { - std::lock_guard lock(outbound_data_queue_lock_); - outbound_data_to_process_.swap(outbound_data_being_processed_); + std::lock_guard lock(outbound_data_queue_mtx_); + outbound_data_being_processed_ = std::move(outbound_data_to_process_); } if (!outbound_data_being_processed_.empty()) { for (auto& call : outbound_data_being_processed_) { - DoQueueOutboundData(std::move(call), /* batch */ true); + // If one of these calls fails and shuts down the connection, all calls after that will fail + // as well. + auto ignored_status = DoQueueOutboundData(std::move(call), /* batch= */ true); } outbound_data_being_processed_.clear(); - OutboundQueued(); + auto ignored_status = OutboundQueued(); } } @@ -546,9 +660,7 @@ Status Connection::Start(ev::loop_ref* loop) { } auto self = shared_from_this(); - context_->AssignConnection(self); - - return Status::OK(); + return context_->AssignConnection(self); } void Connection::Connected() { @@ -575,6 +687,25 @@ void Connection::Close() { stream_->Close(); } +Status Connection::ShutdownStatus() const { + if (!shutdown_initiated_.load(std::memory_order_acquire)) { + return Status::OK(); + } + std::lock_guard lock(outbound_data_queue_mtx_); + return shutdown_status_; +} + +void Connection::ForceCallExpiration(const OutboundCallPtr& call) { + auto call_id = call->call_id(); + auto it = active_calls_.find(call_id); + if (it != active_calls_.end()) { + // This will call SetTimedOut in addition to updating active_calls_. + ActiveCallExpired(active_calls_, it, reactor_, stream_.get()); + } else if (!call->IsFinished()) { + call->SetTimedOut(); + } +} + void Connection::UpdateLastActivity() { last_activity_time_ = reactor_->cur_time(); VLOG_WITH_PREFIX(4) << "Updated last_activity_time_=" << AsString(last_activity_time_); diff --git a/src/yb/rpc/connection.h b/src/yb/rpc/connection.h index 002f2f73bf9..5dec5874d97 100644 --- a/src/yb/rpc/connection.h +++ b/src/yb/rpc/connection.h @@ -61,6 +61,7 @@ #include "yb/rpc/rpc_fwd.h" #include "yb/rpc/stream.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/metrics_fwd.h" #include "yb/util/enums.h" @@ -117,7 +118,7 @@ class Connection final : public StreamContext, public std::enable_shared_from_th ~Connection(); CoarseTimePoint last_activity_time() const { - return last_activity_time_; + return last_activity_time_.load(std::memory_order_acquire); } void UpdateLastActivity() override; @@ -125,21 +126,22 @@ class Connection final : public StreamContext, public std::enable_shared_from_th // Returns true if we are not in the process of receiving or sending a // message, and we have no outstanding calls. // When reason_not_idle is specified it contains reason why this connection is not idle. - bool Idle(std::string* reason_not_idle = nullptr) const; + bool Idle(std::string* reason_not_idle = nullptr) const ON_REACTOR_THREAD; // A human-readable reason why the connection is not idle. Empty string if connection is idle. - std::string ReasonNotIdle() const; + std::string ReasonNotIdle() const ON_REACTOR_THREAD; // Fail any calls which are currently queued or awaiting response. // Prohibits any future calls (they will be failed immediately with this // same Status). - void Shutdown(const Status& status); + void Shutdown(const Status& status) ON_REACTOR_THREAD EXCLUDES(outbound_data_queue_mtx_); // Queue a new call to be made. If the queueing fails, the call will be // marked failed. // Takes ownership of the 'call' object regardless of whether it succeeds or fails. // This may be called from a non-reactor thread. - void QueueOutboundCall(const OutboundCallPtr& call); + void QueueOutboundCall(const OutboundCallPtr& call) ON_REACTOR_THREAD + EXCLUDES(outbound_data_queue_mtx_); // The address of the remote end of the connection. const Endpoint& remote() const; @@ -149,7 +151,7 @@ class Connection final : public StreamContext, public std::enable_shared_from_th // The address of the local end of the connection. const Endpoint& local() const; - void HandleTimeout(ev::timer& watcher, int revents); // NOLINT + void HandleTimeout(ev::timer& watcher, int revents) ON_REACTOR_THREAD; // NOLINT // Safe to be called from other threads. std::string ToString() const; @@ -166,26 +168,30 @@ class Connection final : public StreamContext, public std::enable_shared_from_th // This is usually called by the IPC worker thread when the response is set, but in some // circumstances may also be called by the reactor thread (e.g. if the service has shut down). // In addition to this, its also called for processing events generated by the server. - void QueueOutboundData(OutboundDataPtr outbound_data); + // + // In case is called outside of the reactor thread, it might return an error if it fails to submit + // a task to the reactor thread, e.g. when the reactor is shutting down. + Status QueueOutboundData(OutboundDataPtr outbound_data) EXCLUDES(outbound_data_queue_mtx_); - void QueueOutboundDataBatch(const OutboundDataBatch& batch); + void QueueOutboundDataBatch(const OutboundDataBatch& batch) ON_REACTOR_THREAD + EXCLUDES(outbound_data_queue_mtx_); Reactor* reactor() const { return reactor_; } - Status DumpPB(const DumpRunningRpcsRequestPB& req, - RpcConnectionPB* resp); + Status DumpPB(const DumpRunningRpcsRequestPB& req, RpcConnectionPB* resp) ON_REACTOR_THREAD; - // Do appropriate actions after adding outbound call. - void OutboundQueued(); + // Do appropriate actions after adding outbound call. If the connection is shutting down, + // returns the connection's shutdown status. + Status OutboundQueued() ON_REACTOR_THREAD EXCLUDES(outbound_data_queue_mtx_); // An incoming packet has completed on the client side. This parses the // call response, looks up the CallAwaitingResponse, and calls the // client callback. - Status HandleCallResponse(CallData* call_data); + Status HandleCallResponse(CallData* call_data) ON_REACTOR_THREAD; ConnectionContext& context() { return *context_; } - Status Start(ev::loop_ref* loop); + Status Start(ev::loop_ref* loop) ON_REACTOR_THREAD; // Try to parse already received data. void ParseReceived(); @@ -193,16 +199,41 @@ class Connection final : public StreamContext, public std::enable_shared_from_th void Close(); RpcMetrics& rpc_metrics() { - return *rpc_metrics_; + return rpc_metrics_; + } + + // Returns the connection's shutdown status, or OK if shutdown has not happened yet. + Status ShutdownStatus() const EXCLUDES(outbound_data_queue_mtx_); + + bool shutdown_initiated() const { + return shutdown_initiated_.load(std::memory_order_acquire); } + bool shutdown_completed() const { + return shutdown_completed_.load(std::memory_order_acquire); + } + + // Used in Reactor-based stuck outbound call monitoring mechanism. + void ForceCallExpiration(const OutboundCallPtr& call) ON_REACTOR_THREAD; + private: - Status DoWrite(); + // Marks the given call as failed and schedules destruction of the connection. + void FailCallAndDestroyConnection(const OutboundDataPtr& outbound_data, + const Status& status) ON_REACTOR_THREAD; - // Does actual outbound data queueing. Invoked in appropriate reactor thread. - size_t DoQueueOutboundData(OutboundDataPtr call, bool batch); + void ScheduleDestroyConnection(const Status& status) ON_REACTOR_THREAD; - void ProcessResponseQueue(); + // Does actual outbound data queuing. + // + // If the `batch` argument is false, calls the OutboundQueued function at the end. See + // QueueOutboundDataBatch for how this is used. + // + // Returns the handle corresponding to the queued call, or std::numeric_limits::max() in + // case the handle is unknown, or an error in case the connection is shutting down. + Result DoQueueOutboundData(OutboundDataPtr call, bool batch) ON_REACTOR_THREAD + EXCLUDES(outbound_data_queue_mtx_); + + void ProcessResponseQueue() ON_REACTOR_THREAD EXCLUDES(outbound_data_queue_mtx_); // Stream context implementation void UpdateLastRead() override; @@ -210,17 +241,22 @@ class Connection final : public StreamContext, public std::enable_shared_from_th void UpdateLastWrite() override; void Transferred(const OutboundDataPtr& data, const Status& status) override; - void Destroy(const Status& status) override; - Result ProcessReceived(ReadBufferFull read_buffer_full) override; + void Destroy(const Status& status) ON_REACTOR_THREAD override; + Result ProcessReceived(ReadBufferFull read_buffer_full) ON_REACTOR_THREAD override; void Connected() override; StreamReadBuffer& ReadBuffer() override; - void CleanupExpirationQueue(CoarseTimePoint now); + void CleanupExpirationQueue(CoarseTimePoint now) ON_REACTOR_THREAD; // call_ptr is used only for logging to correlate with OutboundCall trace. - void DumpConnectionState(int32_t call_id, const void* call_ptr) const; + void DumpConnectionState(int32_t call_id, const void* call_ptr) const ON_REACTOR_THREAD + EXCLUDES(outbound_data_queue_mtx_); std::string LogPrefix() const; + // ---------------------------------------------------------------------------------------------- + // Fields set in the constructor + // ---------------------------------------------------------------------------------------------- + // The reactor thread that created this connection. Reactor* const reactor_; @@ -229,12 +265,12 @@ class Connection final : public StreamContext, public std::enable_shared_from_th // whether we are client or server Direction direction_; - // The last time we read or wrote from the socket. - CoarseTimePoint last_activity_time_; + // RPC related metrics. + RpcMetrics& rpc_metrics_; - // Starts as Status::OK, gets set to a shutdown status upon Shutdown(). Guarded by - // outbound_data_queue_lock_. - Status shutdown_status_; + // Connection is responsible for sending and receiving bytes. + // Context is responsible for what to do with them. + std::unique_ptr context_; // We instantiate and store this metric instance at the level of connection, but not at the level // of the class emitting metrics (OutboundTransfer) as recommended in metrics.h. This is on @@ -244,12 +280,24 @@ class Connection final : public StreamContext, public std::enable_shared_from_th // at connection level. scoped_refptr handler_latency_outbound_transfer_; + // ---------------------------------------------------------------------------------------------- + // Fields that are only accessed on the reactor thread and do not require other synchronization + // ---------------------------------------------------------------------------------------------- + // Information about active call. struct ActiveCall { int32_t id; // Call id. OutboundCallPtr call; // Call object, null if call has expired. CoarseTimePoint expires_at; // Expiration time, kMax when call has expired. - size_t handle; // Call handle in outbound stream. + CallHandle handle = 0; // Call handle in outbound stream. + + std::string ToString(std::optional now = std::nullopt) const { + return YB_STRUCT_TO_STRING( + id, + (call, AsString(pointer_cast(call.get()))), + (expires_at, yb::ToStringRelativeToNow(expires_at, now)), + handle); + } }; class ExpirationTag; @@ -264,35 +312,45 @@ class Connection final : public StreamContext, public std::enable_shared_from_th boost::multi_index::tag, boost::multi_index::member >>>; - ActiveCalls active_calls_; + ActiveCalls active_calls_ GUARDED_BY_REACTOR_THREAD; + + EvTimerHolder timer_ GUARDED_BY_REACTOR_THREAD; - EvTimerHolder timer_; + // ---------------------------------------------------------------------------------------------- + // Fields protected by outbound_data_queue_mtx_ + // ---------------------------------------------------------------------------------------------- - simple_spinlock outbound_data_queue_lock_; + mutable simple_spinlock outbound_data_queue_mtx_; // Responses we are going to process. - std::vector outbound_data_to_process_; + std::vector outbound_data_to_process_ GUARDED_BY(outbound_data_queue_mtx_); + + // Starts as Status::OK, gets set to a shutdown status upon Shutdown(). + Status shutdown_status_ GUARDED_BY(outbound_data_queue_mtx_); // Responses that are currently being processed. // It could be in function variable, but declared as member for optimization. - std::vector outbound_data_being_processed_; + std::vector outbound_data_being_processed_ GUARDED_BY_REACTOR_THREAD; std::atomic shutdown_time_{CoarseTimePoint::min()}; - std::shared_ptr process_response_queue_task_; + std::shared_ptr process_response_queue_task_ GUARDED_BY(outbound_data_queue_mtx_); - // RPC related metrics. - RpcMetrics* rpc_metrics_; - - // Connection is responsible for sending and receiving bytes. - // Context is responsible for what to do with them. - std::unique_ptr context_; + // ---------------------------------------------------------------------------------------------- + // Atomic fields + // ---------------------------------------------------------------------------------------------- std::atomic responded_call_count_{0}; std::atomic active_calls_during_shutdown_{0}; std::atomic calls_queued_after_shutdown_{0}; std::atomic responses_queued_after_shutdown_{0}; + // The last time we read or wrote from the socket. + std::atomic last_activity_time_{CoarseTimePoint::min()}; + std::atomic queued_destroy_connection_{false}; + + std::atomic shutdown_initiated_{false}; + std::atomic shutdown_completed_{false}; }; } // namespace rpc diff --git a/src/yb/rpc/connection_context.h b/src/yb/rpc/connection_context.h index 7c01272eb56..50b5895575b 100644 --- a/src/yb/rpc/connection_context.h +++ b/src/yb/rpc/connection_context.h @@ -18,6 +18,7 @@ #include "yb/rpc/rpc_fwd.h" #include "yb/rpc/rpc_introspection.pb.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/net/socket.h" #include "yb/util/strongly_typed_bool.h" @@ -55,19 +56,19 @@ class ConnectionContext { // Checks whether this connection context is idle. // If reason is supplied, then human-readable description of why the context is not idle is // appended to it. - virtual bool Idle(std::string* reason_not_idle = nullptr) = 0; + virtual bool Idle(std::string* reason_not_idle = nullptr) ON_REACTOR_THREAD = 0; // Listen for when context becomes idle. virtual void ListenIdle(IdleListener listener) = 0; // Shutdown this context. - virtual void Shutdown(const Status& status) = 0; + virtual void Shutdown(const Status& status) ON_REACTOR_THREAD = 0; - virtual void QueueResponse(const ConnectionPtr& connection, InboundCallPtr call) = 0; + virtual Status QueueResponse(const ConnectionPtr& connection, InboundCallPtr call) = 0; virtual void SetEventLoop(ev::loop_ref* loop) {} - virtual void AssignConnection(const ConnectionPtr& connection) {} + virtual Status AssignConnection(const ConnectionPtr& connection) { return Status::OK(); } virtual void Connected(const ConnectionPtr& connection) = 0; diff --git a/src/yb/rpc/inbound_call.cc b/src/yb/rpc/inbound_call.cc index 4d5eb321cbb..995bf8c9896 100644 --- a/src/yb/rpc/inbound_call.cc +++ b/src/yb/rpc/inbound_call.cc @@ -208,7 +208,12 @@ void InboundCall::QueueResponse(bool is_success) { LogTrace(); bool expected = false; if (responded_.compare_exchange_strong(expected, true, std::memory_order_acq_rel)) { - connection()->context().QueueResponse(connection(), shared_from(this)); + auto queuing_status = + connection()->context().QueueResponse(connection(), shared_from(this)); + // Do not DFATAL here because it is a normal situation during reactor shutdown. The client + // should detect and handle the error. + LOG_IF_WITH_PREFIX(WARNING, !queuing_status.ok()) + << "Could not queue response to an inbound call: " << queuing_status; } else { LOG_WITH_PREFIX(DFATAL) << "Response already queued"; } diff --git a/src/yb/rpc/local_call.cc b/src/yb/rpc/local_call.cc index 029b8825a5e..580fe0b2cdf 100644 --- a/src/yb/rpc/local_call.cc +++ b/src/yb/rpc/local_call.cc @@ -31,7 +31,7 @@ namespace rpc { using std::shared_ptr; LocalOutboundCall::LocalOutboundCall( - const RemoteMethod* remote_method, + const RemoteMethod& remote_method, const shared_ptr& outbound_call_metrics, AnyMessagePtr response_storage, RpcController* controller, std::shared_ptr rpc_metrics, ResponseCallback callback, diff --git a/src/yb/rpc/local_call.h b/src/yb/rpc/local_call.h index 801433549fa..69cda42a782 100644 --- a/src/yb/rpc/local_call.h +++ b/src/yb/rpc/local_call.h @@ -30,7 +30,7 @@ class LocalYBInboundCall; // A short-circuited outbound call. class LocalOutboundCall : public OutboundCall { public: - LocalOutboundCall(const RemoteMethod* remote_method, + LocalOutboundCall(const RemoteMethod& remote_method, const std::shared_ptr& outbound_call_metrics, AnyMessagePtr response_storage, RpcController* controller, std::shared_ptr rpc_metrics, ResponseCallback callback, @@ -46,6 +46,8 @@ class LocalOutboundCall : public OutboundCall { return req_; } + bool is_local() const override { return true; } + protected: void Serialize(boost::container::small_vector_base* output) override; diff --git a/src/yb/rpc/messenger.cc b/src/yb/rpc/messenger.cc index 44bde21ee8d..9448e1d93d5 100644 --- a/src/yb/rpc/messenger.cc +++ b/src/yb/rpc/messenger.cc @@ -149,7 +149,11 @@ Result> MessengerBuilder::Build() { UseDefaultConnectionContextFactory(); } std::unique_ptr messenger(new Messenger(*this)); - RETURN_NOT_OK(messenger->Init()); + auto messenger_init_status = messenger->Init(); + if (!messenger_init_status.ok()) { + LOG(WARNING) << "Messenger initialization error: " << messenger_init_status.ToString(); + } + RETURN_NOT_OK(messenger_init_status); return messenger; } @@ -208,7 +212,7 @@ void Messenger::Shutdown() { } for (auto* reactor : reactors) { - reactor->Shutdown(); + reactor->StartShutdown(); } scheduler_.Shutdown(); @@ -294,8 +298,9 @@ void Messenger::BreakConnectivity(const IpAddress& address, bool incoming, bool if (inserted_from || inserted_to) { latch.emplace(reactors_.size()); for (const auto& reactor : reactors_) { - auto scheduled = reactor->ScheduleReactorTask(MakeFunctorReactorTask( + auto scheduling_status = reactor->ScheduleReactorTask(MakeFunctorReactorTask( [&latch, address, incoming, outgoing](Reactor* reactor) { + ReactorThreadRoleGuard guard; if (incoming) { reactor->DropIncomingWithRemoteAddress(address); } @@ -305,8 +310,9 @@ void Messenger::BreakConnectivity(const IpAddress& address, bool incoming, bool latch->CountDown(); }, SOURCE_LOCATION())); - if (!scheduled) { - LOG(INFO) << "Failed to schedule drop connection with: " << address.to_string(); + if (!scheduling_status.ok()) { + LOG(DFATAL) << "Failed to schedule drop connection with: " + << address.to_string() << ": " << scheduling_status; latch->CountDown(); } } @@ -345,7 +351,10 @@ void Messenger::RestoreConnectivity(const IpAddress& address, bool incoming, boo } } -bool Messenger::TEST_ShouldArtificiallyRejectIncomingCallsFrom(const IpAddress &remote) { +bool Messenger::TEST_ShouldArtificiallyRejectIncomingCallsFrom(const IpAddress &remote) + // Disabling thread safety analysis because annotations are not working for percpu_rwlock in + // this branch. + NO_THREAD_SAFETY_ANALYSIS { if (has_broken_connectivity_.load(std::memory_order_acquire)) { shared_lock guard(lock_.get_lock()); return broken_connectivity_from_.count(remote) != 0; @@ -353,7 +362,9 @@ bool Messenger::TEST_ShouldArtificiallyRejectIncomingCallsFrom(const IpAddress & return false; } -bool Messenger::TEST_ShouldArtificiallyRejectOutgoingCallsTo(const IpAddress &remote) { +bool Messenger::TEST_ShouldArtificiallyRejectOutgoingCallsTo(const IpAddress &remote) + // Ditto. + NO_THREAD_SAFETY_ANALYSIS { if (has_broken_connectivity_.load(std::memory_order_acquire)) { shared_lock guard(lock_.get_lock()); return broken_connectivity_to_.count(remote) != 0; @@ -458,10 +469,11 @@ void Messenger::QueueOutboundCall(OutboundCallPtr call) { if (TEST_ShouldArtificiallyRejectOutgoingCallsTo(remote.address())) { VLOG(1) << "TEST: Rejected connection to " << remote; - auto scheduled = reactor->ScheduleReactorTask(std::make_shared( - call, SOURCE_LOCATION())); - if (!scheduled) { - call->Transferred(STATUS(Aborted, "Reactor is closing"), nullptr /* conn */); + auto scheduling_status = + reactor->ScheduleReactorTask(std::make_shared( + call, SOURCE_LOCATION())); + if (!scheduling_status.ok()) { + call->Transferred(scheduling_status, nullptr /* conn */); } return; } @@ -618,20 +630,39 @@ Status Messenger::DumpRunningRpcs(const DumpRunningRpcsRequestPB& req, Status Messenger::QueueEventOnAllReactors( ServerEventListPtr server_event, const SourceLocation& source_location) { shared_lock guard(lock_.get_lock()); + Status overall_status; for (const auto& reactor : reactors_) { - reactor->QueueEventOnAllConnections(server_event, source_location); + auto queuing_status = reactor->QueueEventOnAllConnections(server_event, source_location); + if (!queuing_status.ok()) { + LOG(DFATAL) << "Failed to queue a server event on all connections of a reactor: " + << queuing_status; + if (overall_status.ok()) { + // Use the first error status. + overall_status = std::move(queuing_status); + } + } } - return Status::OK(); + return overall_status; } Status Messenger::QueueEventOnFilteredConnections( ServerEventListPtr server_event, const SourceLocation& source_location, ConnectionFilter connection_filter) { shared_lock guard(lock_.get_lock()); + Status overall_status; for (const auto& reactor : reactors_) { - reactor->QueueEventOnFilteredConnections(server_event, source_location, connection_filter); + auto queuing_status = + reactor->QueueEventOnFilteredConnections(server_event, source_location, connection_filter); + if (!queuing_status.ok()) { + LOG(DFATAL) << "Failed to queue a server event on filtered connections of a reactor: " + << queuing_status; + if (overall_status.ok()) { + // Use the first error status. + overall_status = std::move(queuing_status); + } + } } - return Status::OK(); + return overall_status; } void Messenger::RemoveScheduledTask(ScheduledTaskId id) { @@ -658,44 +689,46 @@ void Messenger::AbortOnReactor(ScheduledTaskId task_id) { } } -ScheduledTaskId Messenger::ScheduleOnReactor( - StatusFunctor func, MonoDelta when, const SourceLocation& source_location, Messenger* msgr) { +Result Messenger::ScheduleOnReactor( + StatusFunctor func, MonoDelta when, const SourceLocation& source_location) { + if (closing_.load(std::memory_order_acquire)) { + return STATUS(Aborted, "Cannot schedule task, messenger is closing"); + } DCHECK(!reactors_.empty()); // If we're already running on a reactor thread, reuse it. - Reactor* chosen = nullptr; + Reactor* chosen_reactor = nullptr; for (const auto& r : reactors_) { if (r->IsCurrentThread()) { - chosen = r.get(); + chosen_reactor = r.get(); break; } } - if (chosen == nullptr) { + if (chosen_reactor == nullptr) { // Not running on a reactor thread, pick one at random. - chosen = reactors_[rand() % reactors_.size()].get(); + chosen_reactor = reactors_[rand() % reactors_.size()].get(); } - ScheduledTaskId task_id = 0; - if (msgr != nullptr) { - task_id = next_task_id_.fetch_add(1); - } + auto task_id = next_task_id_.fetch_add(1); auto task = std::make_shared( - std::move(func), when, task_id, source_location, msgr); - if (msgr != nullptr) { - std::lock_guard guard(mutex_scheduled_tasks_); + std::move(func), when, task_id, source_location, this); + { + std::lock_guard guard(mutex_scheduled_tasks_); scheduled_tasks_.emplace(task_id, task); } - if (chosen->ScheduleReactorTask(task)) { + auto scheduling_status = chosen_reactor->ScheduleReactorTask(task); + + if (scheduling_status.ok()) { return task_id; } { - std::lock_guard guard(mutex_scheduled_tasks_); + std::lock_guard guard(mutex_scheduled_tasks_); scheduled_tasks_.erase(task_id); } - return kInvalidTaskId; + return scheduling_status; } scoped_refptr Messenger::metric_entity() const { diff --git a/src/yb/rpc/messenger.h b/src/yb/rpc/messenger.h index 1244c03644e..259a587954d 100644 --- a/src/yb/rpc/messenger.h +++ b/src/yb/rpc/messenger.h @@ -245,10 +245,8 @@ class Messenger : public ProxyContext { // // The status argument conveys whether 'func' was run correctly (i.e. after the elapsed time) or // not. - MUST_USE_RESULT ScheduledTaskId ScheduleOnReactor( - StatusFunctor func, MonoDelta when, - const SourceLocation& source_location, - rpc::Messenger* msgr); + Result ScheduleOnReactor( + StatusFunctor func, MonoDelta when, const SourceLocation& source_location); std::string name() const { return name_; @@ -308,6 +306,10 @@ class Messenger : public ProxyContext { Status TEST_GetReactorMetrics(size_t reactor_idx, ReactorMetrics* metrics); + ScheduledTaskId TEST_next_task_id() const { + return next_task_id_.load(std::memory_order_acquire); + } + private: friend class DelayedTask; @@ -336,7 +338,7 @@ class Messenger : public ProxyContext { // Protects closing_, acceptor_pools_. mutable percpu_rwlock lock_; - bool closing_ = false; + std::atomic_bool closing_ = false; // RPC services that handle inbound requests. mutable RWOperationCounter rpc_services_counter_; @@ -349,7 +351,7 @@ class Messenger : public ProxyContext { const scoped_refptr outgoing_queue_time_; // Acceptor which is listening on behalf of this messenger. - std::unique_ptr acceptor_; + std::unique_ptr acceptor_ GUARDED_BY(lock_); IpAddress outbound_address_v4_; IpAddress outbound_address_v6_; @@ -359,14 +361,15 @@ class Messenger : public ProxyContext { std::mutex mutex_scheduled_tasks_; - std::unordered_map> scheduled_tasks_; + std::unordered_map> scheduled_tasks_ + GUARDED_BY(mutex_scheduled_tasks_); // Flag that we have at least on address with artificially broken connectivity. std::atomic has_broken_connectivity_ = {false}; // Set of addresses with artificially broken connectivity. - std::unordered_set broken_connectivity_from_; - std::unordered_set broken_connectivity_to_; + std::unordered_set broken_connectivity_from_ GUARDED_BY(lock_); + std::unordered_set broken_connectivity_to_ GUARDED_BY(lock_); IoThreadPool io_thread_pool_; Scheduler scheduler_; diff --git a/src/yb/rpc/outbound_call.cc b/src/yb/rpc/outbound_call.cc index 0b297d24f8e..16e7540542f 100644 --- a/src/yb/rpc/outbound_call.cc +++ b/src/yb/rpc/outbound_call.cc @@ -61,6 +61,7 @@ #include "yb/util/scope_exit.h" #include "yb/util/status_format.h" #include "yb/util/thread_restrictions.h" +#include "yb/util/tostring.h" #include "yb/util/trace.h" #include "yb/util/tsan_util.h" @@ -87,6 +88,10 @@ TAG_FLAG(rpc_callback_max_cycles, advanced); TAG_FLAG(rpc_callback_max_cycles, runtime); DECLARE_bool(rpc_dump_all_traces); +DEFINE_test_flag(double, outbound_call_skip_callback_probability, 0.0, + "Test flag for skipping an OutboundCall callback, to simulate a bug with a stuck " + "OutboundCall."); + namespace yb { namespace rpc { @@ -118,6 +123,39 @@ int32_t NextCallId() { const std::string kEmptyString; +bool FinishedState(RpcCallState state) { + switch (state) { + case READY: + case ON_OUTBOUND_QUEUE: + case SENT: + return false; + case TIMED_OUT: + case FINISHED_ERROR: + case FINISHED_SUCCESS: + return true; + } + LOG(FATAL) << "Unknown call state: " << state; + return false; +} + +bool ValidStateTransition(RpcCallState old_state, RpcCallState new_state) { + switch (new_state) { + case ON_OUTBOUND_QUEUE: + return old_state == READY; + case SENT: + return old_state == ON_OUTBOUND_QUEUE; + case TIMED_OUT: + return old_state == SENT || old_state == ON_OUTBOUND_QUEUE; + case FINISHED_SUCCESS: + return old_state == SENT; + case FINISHED_ERROR: + return old_state == SENT || old_state == ON_OUTBOUND_QUEUE || old_state == READY; + default: + // No sanity checks for others. + return true; + } +} + } // namespace void InvokeCallbackTask::Run() { @@ -141,11 +179,37 @@ void InvokeCallbackTask::Done(const Status& status) { call_ = nullptr; } +/// +/// CompletedCallQueue +/// + +void CompletedCallQueue::AddCompletedCall(int32_t call_id) { + if (!stopping_.load(std::memory_order_acquire)) { + completed_calls_.Push(new CompletedCallEntry(call_id)); + } +} + +std::optional CompletedCallQueue::Pop() { + auto entry = std::unique_ptr(completed_calls_.Pop()); + if (!entry) { + return std::nullopt; + } + auto call_id = entry->call_id; + return call_id; +} + +void CompletedCallQueue::Shutdown() { + // Using sequential consistency because we don't want queue draining operations to be reordered + // before setting stopping_ to true, which could have happened with memory_order_release. + stopping_ = true; + completed_calls_.Drain(); +} + /// /// OutboundCall /// -OutboundCall::OutboundCall(const RemoteMethod* remote_method, +OutboundCall::OutboundCall(const RemoteMethod& remote_method, const std::shared_ptr& outbound_call_metrics, std::shared_ptr method_metrics, AnyMessagePtr response_storage, @@ -165,7 +229,7 @@ OutboundCall::OutboundCall(const RemoteMethod* remote_method, outbound_call_metrics_(outbound_call_metrics), rpc_metrics_(std::move(rpc_metrics)), method_metrics_(std::move(method_metrics)) { - TRACE_TO_WITH_TIME(trace_, start_, "$0.", remote_method_->ToString()); + TRACE_TO_WITH_TIME(trace_, start_, "$0.", remote_method_.ToString()); DVLOG(4) << "OutboundCall " << this << " constructed with state_: " << StateName(state_) << " and RPC timeout: " @@ -176,7 +240,16 @@ OutboundCall::OutboundCall(const RemoteMethod* remote_method, } OutboundCall::~OutboundCall() { - DCHECK(IsFinished()); + { + auto current_state = state(); + bool was_callback_invoked = callback_invoked(); + auto invalid_state_transition = invalid_state_transition_.load(std::memory_order_acquire); + LOG_IF_WITH_PREFIX(DFATAL, + !FinishedState(current_state) || + !was_callback_invoked || + invalid_state_transition) + << "Outbound call is in a bad state when destroyed: " << DebugString(); + } DVLOG(4) << "OutboundCall " << this << " destroyed with state_: " << StateName(state_); if (PREDICT_FALSE(FLAGS_rpc_dump_all_traces)) { @@ -189,7 +262,7 @@ OutboundCall::~OutboundCall() { } void OutboundCall::NotifyTransferred(const Status& status, const ConnectionPtr& conn) { - sent_time_ = CoarseMonoClock::Now(); + sent_time_.store(CoarseMonoClock::Now(), std::memory_order_release); if (IsFinished()) { LOG_IF_WITH_PREFIX(DFATAL, !IsTimedOut()) << "Transferred call is in wrong state: " << state_.load(std::memory_order_acquire) @@ -199,8 +272,10 @@ void OutboundCall::NotifyTransferred(const Status& status, const ConnectionPtr& LOG_WITH_PREFIX_AND_FUNC(WARNING) << this << " - Unexpected - Connection is null with ok status"; } - connection_ = conn; - SetSent(); + if (SetSent()) { + std::lock_guard lock(sent_on_connection_mutex_); + sent_on_connection_ = conn; + } } else { VLOG_WITH_PREFIX(1) << "Connection torn down: " << status; SetFailed(status); @@ -220,7 +295,7 @@ Status OutboundCall::SetRequestParam(AnyMessageConstPtr req, const MemTrackerPtr auto timeout_ms = VERIFY_RESULT(TimeoutMs()); size_t call_id_size = Output::VarintSize32(call_id_); size_t timeout_ms_size = Output::VarintSize32(timeout_ms); - auto serialized_remote_method = remote_method_->serialized(); + auto serialized_remote_method = remote_method_.serialized(); size_t header_pb_len = 1 + call_id_size + serialized_remote_method.size() + 1 + timeout_ms_size; size_t header_size = @@ -260,12 +335,12 @@ Status OutboundCall::SetRequestParam(AnyMessageConstPtr req, const MemTrackerPtr } Status OutboundCall::status() const { - std::lock_guard l(lock_); + std::lock_guard l(mtx_); return status_; } const ErrorStatusPB* OutboundCall::error_pb() const { - std::lock_guard l(lock_); + std::lock_guard l(mtx_); return error_pb_.get(); } @@ -277,54 +352,17 @@ OutboundCall::State OutboundCall::state() const { return state_.load(std::memory_order_acquire); } -bool FinishedState(RpcCallState state) { - switch (state) { - case READY: - case ON_OUTBOUND_QUEUE: - case SENT: - return false; - case TIMED_OUT: - case FINISHED_ERROR: - case FINISHED_SUCCESS: - return true; - } - LOG(FATAL) << "Unknown call state: " << state; - return false; -} - -bool ValidStateTransition(RpcCallState old_state, RpcCallState new_state) { - switch (new_state) { - case ON_OUTBOUND_QUEUE: - return old_state == READY; - case SENT: - return old_state == ON_OUTBOUND_QUEUE; - case TIMED_OUT: - return old_state == SENT || old_state == ON_OUTBOUND_QUEUE; - case FINISHED_SUCCESS: - return old_state == SENT; - case FINISHED_ERROR: - return old_state == SENT || old_state == ON_OUTBOUND_QUEUE || old_state == READY; - default: - // No sanity checks for others. - return true; - } -} - bool OutboundCall::SetState(State new_state) { auto old_state = state_.load(std::memory_order_acquire); // Sanity check state transitions. DVLOG(3) << "OutboundCall " << this << " (" << ToString() << ") switching from " << StateName(old_state) << " to " << StateName(new_state); for (;;) { - if (FinishedState(old_state)) { - VLOG(1) << "Call already finished: " << RpcCallState_Name(old_state) << ", new state: " - << RpcCallState_Name(new_state); - return false; - } - if (!ValidStateTransition(old_state, new_state)) { - LOG(DFATAL) - << "Invalid call state transition: " << RpcCallState_Name(old_state) << " => " - << RpcCallState_Name(new_state); + if (FinishedState(old_state) || !ValidStateTransition(old_state, new_state)) { + LOG_WITH_PREFIX(DFATAL) + << "Invalid call state transition: " << StateName(old_state) << " => " + << StateName(new_state) << ": " << DebugString(); + SetInvalidStateTransition(old_state, new_state); return false; } if (state_.compare_exchange_weak(old_state, new_state, std::memory_order_acq_rel)) { @@ -333,27 +371,67 @@ bool OutboundCall::SetState(State new_state) { } } -void OutboundCall::InvokeCallback() { - invoke_callback_time_ = CoarseMonoClock::Now(); - // Release the connection reference once the callback is invoked. This helps prevent circular - // dependencies for cleanup between OutboundCall and Connection. - connection_.reset(); +bool OutboundCall::UpdateCallbackTime( + std::atomic& time, CoarseTimePoint now, const char* callback_action) { + auto expected = CoarseTimePoint::min(); + if (!time.compare_exchange_strong(expected, now, std::memory_order_acq_rel)) { + LOG_WITH_PREFIX(DFATAL) << "Callback was already " << callback_action << " for " + << DebugString() << " at " << ToStringRelativeToNow(expected, now); + return false; + } + return true; +} + +void OutboundCall::InvokeCallback(std::optional now_optional) { + const auto now = now_optional ? *now_optional : CoarseMonoClock::now(); + if (!UpdateCallbackTime(trigger_callback_time_, now, "triggered")) { + return; + } + + LOG_IF_WITH_PREFIX(DFATAL, !IsFinished()) + << "Invoking callback on an unfinished OutboundCall: " << DebugString(); + if (callback_thread_pool_) { callback_task_.SetOutboundCall(shared_from(this)); - callback_thread_pool_->Enqueue(&callback_task_); - TRACE_TO(trace_, "Callback will be called asynchronously."); - } else { - InvokeCallbackSync(); - TRACE_TO(trace_, "Callback called synchronously."); + if (callback_thread_pool_->Enqueue(&callback_task_)) { + TRACE_TO(trace_, "Callback will be called asynchronously."); + } else { + // In case of a failure to enqueue, the thread pool invokes Done() on the task, which in this + // case calls the callback synchronously. + TRACE_TO(trace_, "Callback has been called synchronously."); + } + return; } + + InvokeCallbackSync(now); + TRACE_TO(trace_, "Callback called synchronously."); } -void OutboundCall::InvokeCallbackSync() { - if (!callback_) { - LOG(DFATAL) << "Callback has been already invoked."; +void OutboundCall::InvokeCallbackSync(std::optional now_optional) { + if (RandomActWithProbability(FLAGS_TEST_outbound_call_skip_callback_probability) && + !callback_invoked() && + !is_local()) { + LOG_WITH_PREFIX(WARNING) << "Skipping OutboundCall callback as a test: " << DebugString(); + return; + } + + const auto now = now_optional ? *now_optional : CoarseMonoClock::now(); + if (!UpdateCallbackTime(invoke_callback_time_, now, "invoked")) { return; } + LOG_IF_WITH_PREFIX(DFATAL, !IsFinished()) + << "Invoking callback synchronously on an unfinished OutboundCall: " << DebugString(); + + // Release the connection reference once the callback is invoked. This helps prevent circular + // dependencies between OutboundCall and Connection. + { + std::lock_guard lock(sent_on_connection_mutex_); + sent_on_connection_.reset(); + } + + // TODO: consider removing the cycle-based mechanism of reporting slow callbacks below. + int64_t start_cycles = CycleClock::Now(); callback_(); // Clear the callback, since it may be holding onto reference counts @@ -373,10 +451,37 @@ void OutboundCall::InvokeCallbackSync() { // Could be destroyed during callback. So reset it. controller_ = nullptr; response_ = nullptr; + + auto completed_call_queue = completed_call_queue_.lock(); + if (completed_call_queue) { + completed_call_queue->AddCompletedCall(call_id_); + } +} + +void OutboundCall::SetConnection(const ConnectionPtr& connection) { + if (!connection_weak_.Set(connection)) { + LOG(WARNING) << "Failed to set connection to " << AsString(connection) << " on " + << DebugString(); + } +} + +void OutboundCall::SetCompletedCallQueue( + const std::shared_ptr& completed_call_queue) { + if (!completed_call_queue_.Set(completed_call_queue)) { + LOG(WARNING) << "Failed to set completed call queue on " << DebugString(); + } +} + +void OutboundCall::SetInvalidStateTransition(RpcCallState old_state, RpcCallState new_state) { + invalid_state_transition_.store(InvalidStateTransition { + .old_state = static_cast(old_state), + .new_state = static_cast(new_state) + }, std::memory_order_release); } void OutboundCall::SetResponse(CallResponse&& resp) { - DCHECK(!IsFinished()); + LOG_IF(DFATAL, IsFinished()) << "SetResponse called on an already finished call: " + << DebugString(); if (test_ignore_response) { LOG_WITH_PREFIX(WARNING) << "Skipping OutboundCall response processing: " << this << " - " @@ -409,18 +514,16 @@ void OutboundCall::SetResponse(CallResponse&& resp) { SetFailed(status); return; } - if (SetState(FINISHED_SUCCESS)) { - InvokeCallback(); - } else { - LOG(DFATAL) << "Success of already finished call: " - << RpcCallState_Name(state_.load(std::memory_order_acquire)); + if (SetState(RpcCallState::FINISHED_SUCCESS)) { + InvokeCallback(now); } } else { // Error auto err = std::make_unique(); if (!pb_util::ParseFromArray(err.get(), r.data(), r.size()).IsOk()) { - SetFailed(STATUS(IOError, "Was an RPC error but could not parse error response", - err->InitializationErrorString())); + SetFailed(STATUS(IOError, + "Was an RPC error but could not parse error response", + err->InitializationErrorString())); return; } auto status = STATUS(RemoteError, err->message()); @@ -432,29 +535,28 @@ void OutboundCall::SetQueued() { auto end_time = CoarseMonoClock::Now(); // Track time taken to be queued. if (outbound_call_metrics_) { - outbound_call_metrics_->queue_time->Increment(MonoDelta(end_time - start_).ToMicroseconds()); + outbound_call_metrics_->queue_time->Increment(MicrosecondsSinceStart(end_time)); } - SetState(ON_OUTBOUND_QUEUE); + auto ignored [[maybe_unused]] = SetState(RpcCallState::ON_OUTBOUND_QUEUE); // NOLINT TRACE_TO_WITH_TIME(trace_, end_time, "Queued."); } -void OutboundCall::SetSent() { +bool OutboundCall::SetSent() { auto end_time = CoarseMonoClock::Now(); // Track time taken to be sent if (outbound_call_metrics_) { - outbound_call_metrics_->send_time->Increment(MonoDelta(end_time - start_).ToMicroseconds()); + outbound_call_metrics_->send_time->Increment(MicrosecondsSinceStart(end_time)); } - SetState(SENT); + auto state_set_successfully = SetState(RpcCallState::SENT); TRACE_TO_WITH_TIME(trace_, end_time, "Call Sent."); + return state_set_successfully; } void OutboundCall::SetFinished() { - DCHECK(!IsFinished()); - // Track time taken to be responded. if (outbound_call_metrics_) { outbound_call_metrics_->time_to_response->Increment( - MonoDelta(CoarseMonoClock::Now() - start_).ToMicroseconds()); + MicrosecondsSinceStart(CoarseMonoClock::Now())); } if (SetState(FINISHED_SUCCESS)) { InvokeCallback(); @@ -462,23 +564,30 @@ void OutboundCall::SetFinished() { } void OutboundCall::SetFailed(const Status &status, std::unique_ptr err_pb) { - DCHECK(!IsFinished()); - TRACE_TO(trace_, "Call Failed."); bool invoke_callback; { - std::lock_guard l(lock_); - status_ = status; - if (status_.IsRemoteError()) { - CHECK(err_pb); - error_pb_ = std::move(err_pb); - if (error_pb_->has_code()) { - status_ = status_.CloneAndAddErrorCode(RpcError(error_pb_->code())); + std::lock_guard l(mtx_); + invoke_callback = SetState(RpcCallState::FINISHED_ERROR); + if (invoke_callback) { + status_ = status; + if (status_.IsRemoteError()) { + CHECK(err_pb); + error_pb_ = std::move(err_pb); + if (error_pb_->has_code()) { + status_ = status_.CloneAndAddErrorCode(RpcError(error_pb_->code())); + } + } else { + CHECK(!err_pb); } } else { - CHECK(!err_pb); + LOG_WITH_PREFIX(ERROR) + << "SetFailed failed to set call state. Existing status_: " << status_ + << ", attempting to set status to: " << status; + if (status_.ok()) { + status_ = status; + } } - invoke_callback = SetState(FINISHED_ERROR); } if (invoke_callback) { InvokeCallback(); @@ -486,19 +595,17 @@ void OutboundCall::SetFailed(const Status &status, std::unique_ptrmethod_name(), + remote_method_.method_name(), conn_id_.remote(), controller_->timeout(), call_id_); - std::lock_guard l(lock_); + std::lock_guard l(mtx_); status_ = std::move(status); invoke_callback = SetState(TIMED_OUT); } @@ -515,21 +622,64 @@ bool OutboundCall::IsFinished() const { return FinishedState(state_.load(std::memory_order_acquire)); } -Result OutboundCall::GetSidecar(size_t idx) const { +Result OutboundCall::GetSidecar(size_t idx) const NO_THREAD_SAFETY_ANALYSIS { return call_response_.GetSidecar(idx); } -Result OutboundCall::GetSidecarHolder(size_t idx) const { +Result OutboundCall::GetSidecarHolder(size_t idx) const NO_THREAD_SAFETY_ANALYSIS { return call_response_.GetSidecarHolder(idx); } string OutboundCall::ToString() const { - return Format("RPC call $0 -> $1 , state=$2.", *remote_method_, conn_id_, StateName(state_)); + std::ostringstream out; + if (is_local()) { + out << "Local "; + } + // Use AsString to get a fixed-with pointer format. + out << "RPC call " << AsString(pointer_cast(this)); + out << ": " << AsString(remote_method_) << " -> " << AsString(conn_id_); + out << ", id: " << call_id_; + out << ", state: " << RpcCallState_Name(state()); + out << ", transfer_state: " << transfer_state(); + return out.str(); +} + +std::string OutboundCall::DebugString() const { + auto now = CoarseMonoClock::Now(); + std::ostringstream out; + + out << ToString(); + out << ", start_time: " << ToStringRelativeToNow(start_, now); + out << ", sent_time: " << ToStringRelativeToNow(sent_time_.load(std::memory_order_acquire), now); + + out << ", trigger_callback_time: "; + out << ToStringRelativeToNow(trigger_callback_time_.load(std::memory_order_acquire), now); + + out << ", invoke_callback_time: "; + out << ToStringRelativeToNow(invoke_callback_time_.load(std::memory_order_acquire), now); + + out << ", expiration_time: "; + out << ToStringRelativeToNow(expires_at_.load(std::memory_order_acquire), now); + + out << ", now: " << AsString(now); + + auto conn = connection_weak_.lock(); + const void* conn_raw_ptr = connection_weak_.raw_ptr_for_logging(); + out << ", connection: "; + out << (conn ? conn->ToString() : (conn_raw_ptr ? AsString(conn_raw_ptr) : "N/A")); + + out << ", active_call_state: " << active_call_state_.load(std::memory_order_acquire); + + auto invalid_state_transition = invalid_state_transition_.load(std::memory_order_acquire); + if (invalid_state_transition) { + out << ", invalid_state_transition: " << invalid_state_transition->ToString(); + } + return out.str(); } bool OutboundCall::DumpPB(const DumpRunningRpcsRequestPB& req, RpcCallInProgressPB* resp) { - std::lock_guard l(lock_); + std::lock_guard l(mtx_); auto state_value = state(); if (!req.dump_timed_out() && state_value == RpcCallState::TIMED_OUT) { return false; @@ -549,7 +699,7 @@ bool OutboundCall::DumpPB(const DumpRunningRpcsRequestPB& req, } std::string OutboundCall::LogPrefix() const { - return Format("{ OutboundCall@$0 } ", this); + return Format("OutboundCall@$0: ", pointer_cast(this)); } Result OutboundCall::TimeoutMs() const { @@ -567,7 +717,7 @@ Result OutboundCall::TimeoutMs() const { Status OutboundCall::InitHeader(RequestHeader* header) { header->set_call_id(call_id_); - remote_method_->ToPB(header->mutable_remote_method()); + remote_method_.ToPB(header->mutable_remote_method()); if (!IsFinished()) { header->set_timeout_millis(VERIFY_RESULT(TimeoutMs())); @@ -575,16 +725,11 @@ Status OutboundCall::InitHeader(RequestHeader* header) { return Status::OK(); } -std::string OutboundCall::DebugString() const { - if (connection_) { - connection_->QueueDumpConnectionState(call_id_, this); +void OutboundCall::QueueDumpConnectionState() const { + auto connection = connection_weak_.lock(); + if (connection) { + connection->QueueDumpConnectionState(call_id_, this); } - - return Format( - "OutboundCall($0): $1, start_time: $2, sent_time: $3, callback_time: $4, now: $5, " - "connection: $6", - this, ToString(), start_, sent_time_, invoke_callback_time_, CoarseMonoClock::Now(), - connection_); } /// @@ -669,5 +814,11 @@ const std::string kRpcErrorCategoryName = "rpc error"; StatusCategoryRegisterer rpc_error_category_registerer( StatusCategoryDescription::Make(&kRpcErrorCategoryName)); +std::string OutboundCall::InvalidStateTransition::ToString() const { + return YB_STRUCT_TO_STRING( + (old_state, RpcCallState_Name(static_cast(old_state))), + (new_state, RpcCallState_Name(static_cast(new_state)))); +} + } // namespace rpc } // namespace yb diff --git a/src/yb/rpc/outbound_call.h b/src/yb/rpc/outbound_call.h index fd5d52686b0..8a8287b902d 100644 --- a/src/yb/rpc/outbound_call.h +++ b/src/yb/rpc/outbound_call.h @@ -61,9 +61,10 @@ #include "yb/rpc/rpc_introspection.pb.h" #include "yb/rpc/service_if.h" #include "yb/rpc/thread_pool.h" +#include "yb/rpc/reactor_thread_role.h" -#include "yb/util/status_fwd.h" #include "yb/util/atomic.h" +#include "yb/util/lockfree.h" #include "yb/util/locks.h" #include "yb/util/mem_tracker.h" #include "yb/util/memory/memory_usage.h" @@ -73,6 +74,7 @@ #include "yb/util/ref_cnt_buffer.h" #include "yb/util/shared_lock.h" #include "yb/util/slice.h" +#include "yb/util/status_fwd.h" #include "yb/util/trace.h" namespace google { @@ -214,6 +216,42 @@ class InvokeCallbackTask : public rpc::ThreadPoolTask { OutboundCallPtr call_; }; +class CompletedCallQueue { + public: + CompletedCallQueue() {} + virtual ~CompletedCallQueue() {} + + // Called when a callback finishes running for a particular call. + void AddCompletedCall(int32_t call_id); + + std::optional Pop() ON_REACTOR_THREAD; + + void Shutdown(); + + private: + struct CompletedCallEntry : MPSCQueueEntry { + explicit CompletedCallEntry(int call_id_) : call_id(call_id_) {} + int32_t call_id; + }; + + // We use this queue to notify the reactor thread that calls have completed so we would stop + // tracking them. + MPSCQueue completed_calls_; + + std::atomic stopping_{false}; +}; + +// Tracks the state of this OutboundCall in relation to the active_calls_ structure in Connection. +// Needed for debugging of stuck OutboundCalls where the callback never gets called. +YB_DEFINE_ENUM(ActiveCallState, + (kNotAdded) // Never added to active calls + (kAdded) // Added to active calls + (kErasedOnResponse) // Erased from active calls after receiving a response + (kResetOnExpiration) // call field in ActiveCall set to nullptr after expiration + (kErasedOnExpiration) // Erased from active calls after expiration + (kErasedOnConnectionShutdown) // Active calls fully cleared at connection shutdown + ); + // Tracks the status of a call on the client side. // // This is an internal-facing class -- clients interact with the @@ -225,7 +263,7 @@ class InvokeCallbackTask : public rpc::ThreadPoolTask { // of different threads, making it tricky to enforce single ownership. class OutboundCall : public RpcCall { public: - OutboundCall(const RemoteMethod* remote_method, + OutboundCall(const RemoteMethod& remote_method, const std::shared_ptr& outbound_call_metrics, std::shared_ptr method_metrics, AnyMessagePtr response_storage, @@ -247,35 +285,32 @@ class OutboundCall : public RpcCall { // is called first. This is called from the Reactor thread. void Serialize(boost::container::small_vector_base* output) override; - // Sets thread pool to be used by `InvokeCallback` for callback execution. - void SetCallbackThreadPool(ThreadPool* callback_thread_pool) { - callback_thread_pool_ = callback_thread_pool; - } - // Callback after the call has been put on the outbound connection queue. void SetQueued(); // Update the call state to show that the request has been sent. // Could be called on already finished call in case it was already timed out. - void SetSent(); + WARN_UNUSED_RESULT bool SetSent(); // Outbound call could be moved to final state only once, // so only one of SetFinished/SetTimedOut/SetFailed/SetResponse can be called. - // Update the call state to show that the call has finished. + // Update the call state to show that the call has finished. Returns true in case of a successful + // state transition. void SetFinished(); // Mark the call as failed. This also triggers the callback to notify // the caller. If the call failed due to a remote error, then err_pb // should be set to the error returned by the remote server. - void SetFailed(const Status& status, std::unique_ptr err_pb = nullptr); + void SetFailed(const Status& status, std::unique_ptr err_pb = nullptr) + EXCLUDES(mtx_); // Mark the call as timed out. This also triggers the callback to notify // the caller. - void SetTimedOut(); + void SetTimedOut() ON_REACTOR_THREAD EXCLUDES(mtx_); // Fill in the call response. - void SetResponse(CallResponse&& resp); + void SetResponse(CallResponse&& resp) ON_REACTOR_THREAD; bool IsTimedOut() const; @@ -284,7 +319,10 @@ class OutboundCall : public RpcCall { std::string ToString() const override; - bool DumpPB(const DumpRunningRpcsRequestPB& req, RpcCallInProgressPB* resp) override; + std::string DebugString() const; + + bool DumpPB(const DumpRunningRpcsRequestPB& req, RpcCallInProgressPB* resp) + EXCLUDES(mtx_) override; std::string LogPrefix() const override; @@ -293,15 +331,28 @@ class OutboundCall : public RpcCall { hostname_ = hostname; } - void SetThreadPoolFailure(const Status& status) { + void SetThreadPoolFailure(const Status& status) EXCLUDES(mtx_) { + std::lock_guard lock(mtx_); thread_pool_failure_ = status; } - const Status& thread_pool_failure() const { + const Status& thread_pool_failure() const EXCLUDES(mtx_) { + std::lock_guard lock(mtx_); return thread_pool_failure_; } - void InvokeCallbackSync(); + void SetActiveCallState(ActiveCallState new_state) { + active_call_state_.store(new_state, std::memory_order_release); + } + + void SetConnection(const ConnectionPtr& connection); + void SetCompletedCallQueue(const std::shared_ptr& completed_call_queue); + + void SetInvalidStateTransition(RpcCallState old_state, RpcCallState new_state); + + void SetExpiration(CoarseTimePoint expires_at) { + expires_at_.store(expires_at, std::memory_order_release); + } //////////////////////////////////////////////////////////// // Getters @@ -309,8 +360,10 @@ class OutboundCall : public RpcCall { const ConnectionId& conn_id() const { return conn_id_; } const std::string& hostname() const { return *hostname_; } - const RemoteMethod& remote_method() const { return *remote_method_; } + + const RemoteMethod& remote_method() const { return remote_method_; } const ResponseCallback& callback() const { return callback_; } + RpcController* controller() { return controller_; } const RpcController* controller() const { return controller_; } AnyMessagePtr response() const { return response_; } @@ -329,18 +382,46 @@ class OutboundCall : public RpcCall { size_t ObjectSize() const override { return sizeof(*this); } - size_t DynamicMemoryUsage() const override { + size_t DynamicMemoryUsage() const override ON_REACTOR_THREAD EXCLUDES(mtx_) { return DynamicMemoryUsageAllowSizeOf(error_pb_) + DynamicMemoryUsageOf(buffer_, call_response_, trace_); } CoarseTimePoint CallStartTime() const { return start_; } - std::string DebugString() const; + // Queues a reactor thread operation to dump the connection state relevant to this call. + void QueueDumpConnectionState() const; // Test only method to reproduce a stuck OutboundCall scenario seen in production. void TEST_ignore_response() { test_ignore_response = true; } + virtual bool is_local() const { return false; } + + // Returns true if the callback has been triggered, e.g. by transitioning the call to a final + // state. + bool callback_triggered() const { + return IsInitialized(trigger_callback_time_.load(std::memory_order_acquire)); + } + + // Returns true if the callback has been invoked and not just submitted to a thread pool. + bool callback_invoked() const { + return IsInitialized(invoke_callback_time_.load(std::memory_order_acquire)); + } + + CoarseTimePoint start_time() { + return start_; + } + + ConnectionPtr connection() const { + return connection_weak_.lock(); + } + + CoarseTimePoint expires_at() const { return expires_at_.load(std::memory_order_acquire); } + + int64_t MicrosecondsSinceStart(CoarseTimePoint now) { + return MonoDelta(now - start_).ToMicroseconds(); + } + protected: friend class RpcController; @@ -349,16 +430,16 @@ class OutboundCall : public RpcCall { ConnectionId conn_id_; const std::string* hostname_; - CoarseTimePoint start_; - CoarseTimePoint sent_time_; - CoarseTimePoint invoke_callback_time_; + + const CoarseTimePoint start_; + RpcController* controller_; // Pointer for the protobuf where the response should be written. // Can be used only while callback_ object is alive. AnyMessagePtr response_; // The trace buffer. - scoped_refptr trace_; + const scoped_refptr trace_; private: friend class RpcController; @@ -369,48 +450,58 @@ class OutboundCall : public RpcCall { void NotifyTransferred(const Status& status, const ConnectionPtr& conn) override; - bool SetState(State new_state); + MUST_USE_RESULT bool SetState(State new_state); State state() const; - // Same as set_state, but requires that the caller already holds - // lock_ - void set_state_unlocked(State new_state); - // return current status - Status status() const; + Status status() const EXCLUDES(mtx_); // Return the error protobuf, if a remote error occurred. // This will only be non-NULL if status().IsRemoteError(). - const ErrorStatusPB* error_pb() const; + const ErrorStatusPB* error_pb() const EXCLUDES(mtx_); Status InitHeader(RequestHeader* header); - // Lock for state_ status_, error_pb_ fields, since they - // may be mutated by the reactor thread while the client thread - // reads them. - mutable simple_spinlock lock_; std::atomic state_ = {READY}; - Status status_; - std::unique_ptr error_pb_; - // Invokes the user-provided callback. Uses callback_thread_pool_ if set. - void InvokeCallback(); + MUST_USE_RESULT bool UpdateCallbackTime( + std::atomic& time, CoarseTimePoint now, const char* callback_action); + + // Invokes the user-provided callback. Uses callback_thread_pool_ if set. This is only invoked + // after a successful transition of the call state to one of the final states, so it should be + // called exactly once. Can be passed in the clock value as an optimization if the clock has + // already been read by the caller. + void InvokeCallback(std::optional now_optional = std::nullopt); + + // Invokes the callback synchronously. Can be passed in the clock value as an optimization if the + // clock has already been read by the caller. + void InvokeCallbackSync(std::optional now_optional = std::nullopt); Result TimeoutMs() const; const int32_t call_id_; // The remote method being called. - const RemoteMethod* remote_method_; + const RemoteMethod& remote_method_; ResponseCallback callback_; - // Only set if the OutboundCall was sent. - ConnectionPtr connection_; + ThreadPool* const callback_thread_pool_; + const std::shared_ptr outbound_call_metrics_; + const std::shared_ptr rpc_metrics_; + const std::shared_ptr method_metrics_; + // ---------------------------------------------------------------------------------------------- + // Fields that may be mutated by the reactor thread while the client thread reads them. + // ---------------------------------------------------------------------------------------------- - InvokeCallbackTask callback_task_; + mutable simple_spinlock mtx_; + Status status_ GUARDED_BY(mtx_); + std::unique_ptr error_pb_ GUARDED_BY(mtx_); + Status thread_pool_failure_ GUARDED_BY(mtx_); + + // ---------------------------------------------------------------------------------------------- - ThreadPool* callback_thread_pool_; + InvokeCallbackTask callback_task_; // Buffers for storing segments of the wire-format request. RefCntBuffer buffer_; @@ -419,18 +510,45 @@ class OutboundCall : public RpcCall { ScopedTrackedConsumption buffer_consumption_; // Once a response has been received for this call, contains that response. - CallResponse call_response_; + CallResponse call_response_ GUARDED_BY_REACTOR_THREAD; // TEST only flag to reproduce stuck OutboundCall scenario. bool test_ignore_response = false; - std::shared_ptr outbound_call_metrics_; + std::atomic active_call_state_{ActiveCallState::kNotAdded}; + + std::atomic sent_time_{CoarseTimePoint::min()}; + std::atomic trigger_callback_time_{CoarseTimePoint::min()}; + std::atomic invoke_callback_time_{CoarseTimePoint::min()}; + + // If we encounter an invalid state transition, we keep track of it so we can log it. + struct InvalidStateTransition { + uint8_t old_state = RpcCallState::READY; + uint8_t new_state = RpcCallState::READY; + std::string ToString() const; + }; + static_assert(sizeof(std::optional) == 3); + std::atomic> invalid_state_transition_{std::nullopt}; + + // This is used in Reactor-based timeout enforcement and for logging. + std::atomic expires_at_{CoarseTimePoint::max()}; + WriteOnceWeakPtr completed_call_queue_; + + // ---------------------------------------------------------------------------------------------- + // Fields with custom synchronization + // ---------------------------------------------------------------------------------------------- + + simple_spinlock sent_on_connection_mutex_; - std::shared_ptr rpc_metrics_; + // Only set if the OutboundCall was sent. Reset when callback is invoked. + ConnectionPtr sent_on_connection_ GUARDED_BY(sent_on_connection_mutex_); - Status thread_pool_failure_; + // Set and hold a weak reference to the connection for the remaining lifetime of the call. + WriteOnceWeakPtr connection_weak_; - std::shared_ptr method_metrics_; + // InvokeCallbackTask should be able to call InvokeCallbackSync and we don't want other that + // method to be public. + friend class InvokeCallbackTask; DISALLOW_COPY_AND_ASSIGN(OutboundCall); }; diff --git a/src/yb/rpc/outbound_data.h b/src/yb/rpc/outbound_data.h index c62bba9c708..f36ace1facd 100644 --- a/src/yb/rpc/outbound_data.h +++ b/src/yb/rpc/outbound_data.h @@ -31,6 +31,7 @@ #include "yb/util/ref_cnt_buffer.h" #include "yb/util/tostring.h" #include "yb/util/type_traits.h" +#include "yb/rpc/reactor_thread_role.h" namespace yb { diff --git a/src/yb/rpc/proxy.cc b/src/yb/rpc/proxy.cc index e65408d8e74..ce78560cf47 100644 --- a/src/yb/rpc/proxy.cc +++ b/src/yb/rpc/proxy.cc @@ -182,7 +182,7 @@ void Proxy::AsyncLocalCall( RpcController* controller, ResponseCallback callback, const bool force_run_callback_on_reactor) { controller->call_ = std::make_shared( - method, outbound_call_metrics_, resp, controller, context_->rpc_metrics(), + *method, outbound_call_metrics_, resp, controller, context_->rpc_metrics(), std::move(callback), GetCallbackThreadPool(force_run_callback_on_reactor, controller->invoke_callback_mode())); if (!PrepareCall(req, controller)) { @@ -197,7 +197,7 @@ void Proxy::AsyncLocalCall( } auto call = controller->call_.get(); call->SetQueued(); - call->SetSent(); + auto ignored [[maybe_unused]] = call->SetSent(); // NOLINT // If currrent thread is RPC worker thread, it is ok to call the handler in the current thread. // Otherwise, enqueue the call to be handled by the service's handler thread. const shared_ptr& local_call = @@ -211,10 +211,12 @@ void Proxy::AsyncRemoteCall( const RemoteMethod* method, std::shared_ptr method_metrics, AnyMessageConstPtr req, AnyMessagePtr resp, RpcController* controller, ResponseCallback callback, const bool force_run_callback_on_reactor) { - controller->call_ = std::make_shared( - method, outbound_call_metrics_, std::move(method_metrics), resp, controller, + // Do not use make_shared to allow for long-lived weak OutboundCall pointers without wasting + // memory. + controller->call_ = std::shared_ptr(new OutboundCall( + *method, outbound_call_metrics_, std::move(method_metrics), resp, controller, context_->rpc_metrics(), std::move(callback), - GetCallbackThreadPool(force_run_callback_on_reactor, controller->invoke_callback_mode())); + GetCallbackThreadPool(force_run_callback_on_reactor, controller->invoke_callback_mode()))); if (!PrepareCall(req, controller)) { return; } diff --git a/src/yb/rpc/reactor-test.cc b/src/yb/rpc/reactor-test.cc index 3790c37f003..3e02e2fde7b 100644 --- a/src/yb/rpc/reactor-test.cc +++ b/src/yb/rpc/reactor-test.cc @@ -70,10 +70,11 @@ class ReactorTest : public RpcTestBase { } void ScheduledTaskScheduleAgain(const Status& status) { - auto task_id = messenger_->ScheduleOnReactor( + auto expected_task_id = messenger_->TEST_next_task_id(); + auto task_id = ASSERT_RESULT(messenger_->ScheduleOnReactor( std::bind(&ReactorTest::ScheduledTaskCheckThread, this, _1, Thread::current_thread()), - 0s, SOURCE_LOCATION(), nullptr /* messenger */); - ASSERT_EQ(task_id, 0); + 0s, SOURCE_LOCATION())); + ASSERT_EQ(expected_task_id, task_id); latch_.CountDown(); } @@ -83,19 +84,21 @@ class ReactorTest : public RpcTestBase { }; TEST_F(ReactorTest, TestFunctionIsCalled) { - auto task_id = messenger_->ScheduleOnReactor( + auto expected_task_id = messenger_->TEST_next_task_id(); + auto task_id = ASSERT_RESULT(messenger_->ScheduleOnReactor( std::bind(&ReactorTest::ScheduledTask, this, _1, Status::OK()), 0s, - SOURCE_LOCATION(), nullptr /* messenger */); - ASSERT_EQ(task_id, 0); + SOURCE_LOCATION())); + ASSERT_EQ(expected_task_id, task_id); latch_.Wait(); } TEST_F(ReactorTest, TestFunctionIsCalledAtTheRightTime) { MonoTime before = MonoTime::Now(); - auto task_id = messenger_->ScheduleOnReactor( + auto expected_task_id = messenger_->TEST_next_task_id(); + auto task_id = ASSERT_RESULT(messenger_->ScheduleOnReactor( std::bind(&ReactorTest::ScheduledTask, this, _1, Status::OK()), - 100ms, SOURCE_LOCATION(), nullptr /* messenger */); - ASSERT_EQ(task_id, 0); + 100ms, SOURCE_LOCATION())); + ASSERT_EQ(expected_task_id, task_id); latch_.Wait(); MonoTime after = MonoTime::Now(); MonoDelta delta = after.GetDeltaSince(before); @@ -103,10 +106,11 @@ TEST_F(ReactorTest, TestFunctionIsCalledAtTheRightTime) { } TEST_F(ReactorTest, TestFunctionIsCalledIfReactorShutdown) { - auto task_id = messenger_->ScheduleOnReactor( + auto expected_task_id = messenger_->TEST_next_task_id(); + auto task_id = ASSERT_RESULT(messenger_->ScheduleOnReactor( std::bind(&ReactorTest::ScheduledTask, this, _1, STATUS(Aborted, "doesn't matter")), - 60s, SOURCE_LOCATION(), nullptr /* messenger */); - ASSERT_EQ(task_id, 0); + 60s, SOURCE_LOCATION())); + ASSERT_EQ(expected_task_id, task_id); messenger_->Shutdown(); latch_.Wait(); } @@ -115,13 +119,15 @@ TEST_F(ReactorTest, TestReschedulesOnSameReactorThread) { // Our scheduled task will schedule yet another task. latch_.Reset(2); - auto task_id = messenger_->ScheduleOnReactor( + auto expected_task_id = messenger_->TEST_next_task_id(); + auto task_id = ASSERT_RESULT(messenger_->ScheduleOnReactor( std::bind(&ReactorTest::ScheduledTaskScheduleAgain, this, _1), 0s, - SOURCE_LOCATION(), nullptr /* messenger */); - ASSERT_EQ(task_id, 0); + SOURCE_LOCATION())); + ASSERT_EQ(expected_task_id, task_id); latch_.Wait(); latch_.Wait(); } + } // namespace rpc } // namespace yb diff --git a/src/yb/rpc/reactor.cc b/src/yb/rpc/reactor.cc index 32574d78af3..2407bec5e2b 100644 --- a/src/yb/rpc/reactor.cc +++ b/src/yb/rpc/reactor.cc @@ -50,30 +50,34 @@ #include "yb/gutil/ref_counted.h" #include "yb/gutil/stringprintf.h" -#include "yb/rpc/connection.h" #include "yb/rpc/connection_context.h" +#include "yb/rpc/connection.h" #include "yb/rpc/messenger.h" #include "yb/rpc/rpc_controller.h" #include "yb/rpc/rpc_introspection.pb.h" +#include "yb/rpc/rpc_metrics.h" #include "yb/rpc/server_event.h" #include "yb/util/atomic.h" #include "yb/util/countdown_latch.h" #include "yb/util/errno.h" +#include "yb/util/flags.h" #include "yb/util/format.h" #include "yb/util/logging.h" #include "yb/util/memory/memory.h" #include "yb/util/metric_entity.h" +#include "yb/util/metrics.h" #include "yb/util/monotime.h" #include "yb/util/net/socket.h" #include "yb/util/scope_exit.h" #include "yb/util/size_literals.h" -#include "yb/util/status.h" #include "yb/util/status_format.h" #include "yb/util/status_log.h" -#include "yb/util/thread.h" +#include "yb/util/status.h" #include "yb/util/thread_restrictions.h" +#include "yb/util/thread.h" #include "yb/util/trace.h" +#include "yb/util/unique_lock.h" using namespace std::literals; @@ -83,6 +87,27 @@ DECLARE_string(local_ip_for_outbound_sockets); DECLARE_int32(num_connections_to_server); DECLARE_int32(socket_receive_buffer_size); +DEFINE_bool(reactor_check_current_thread, true, + "Enforce the requirement that operations that require running on a reactor " + "thread are always running on the correct reactor thread."); + +DEFINE_int32(stuck_outbound_call_default_timeout_sec, 120, + "Default timeout for reporting purposes for the Reactor-based stuck OutboundCall tracking and " + "expiration mechanism. That mechanism itself is controlled by the " + "reactor_based_outbound_call_expiration_delay_ms flag. Note that this flag does not force a " + "call to be timed out, it just specifies the interval after which the call is logged."); +TAG_FLAG(stuck_outbound_call_default_timeout_sec, advanced); + +DEFINE_int32(stuck_outbound_call_check_interval_sec, 30, + "Check and report each stuck outbound call at most once per this number of seconds."); +TAG_FLAG(stuck_outbound_call_check_interval_sec, advanced); + +DEFINE_int32(reactor_based_outbound_call_expiration_delay_ms, 1000, + "Expire OutboundCalls using Reactor-level logic with this delay after the timeout, as an " + "additional layer of protection against stuck outbound calls. This safety mechanism is " + "disabled if this flag is set to 0."); +TAG_FLAG(reactor_based_outbound_call_expiration_delay_ms, advanced); + namespace yb { namespace rpc { @@ -125,6 +150,26 @@ size_t PatchReceiveBufferSize(size_t receive_buffer_size) { 64_KB, FLAGS_rpc_read_buffer_size ? FLAGS_rpc_read_buffer_size : receive_buffer_size); } +inline bool ShouldCheckCurrentThread() { +#ifdef NDEBUG + // Release mode, use the flag. + return FLAGS_reactor_check_current_thread; +#else + // Debug mode, always check. + return true; +#endif +} + +bool ShouldTrackOutboundCalls() { + return FLAGS_reactor_based_outbound_call_expiration_delay_ms > 0; +} + +CoarseTimePoint ExpirationEnforcementTime(CoarseTimePoint expires_at) { + return expires_at == CoarseTimePoint::max() + ? CoarseTimePoint::max() + : expires_at + 1ms * FLAGS_reactor_based_outbound_call_expiration_delay_ms; +} + } // anonymous namespace // ------------------------------------------------------------------------------------------------ @@ -134,15 +179,18 @@ size_t PatchReceiveBufferSize(size_t receive_buffer_size) { Reactor::Reactor(Messenger* messenger, int index, const MessengerBuilder &bld) - : messenger_(messenger), + : messenger_(*messenger), name_(StringPrintf("%s_R%03d", messenger->name().c_str(), index)), log_prefix_(name_ + ": "), loop_(kDefaultLibEvFlags), - cur_time_(CoarseMonoClock::Now()), - last_unused_tcp_scan_(cur_time_), connection_keepalive_time_(bld.connection_keepalive_time()), coarse_timer_granularity_(bld.coarse_timer_granularity()), - num_connections_to_server_(bld.num_connections_to_server()) { + process_outbound_queue_task_( + MakeFunctorReactorTask(std::bind(&Reactor::ProcessOutboundQueue, this), + SOURCE_LOCATION())), + num_connections_to_server_(bld.num_connections_to_server()), + completed_call_queue_(std::make_shared()), + cur_time_(CoarseMonoClock::Now()) { static std::once_flag libev_once; std::call_once(libev_once, DoInitLibEv); @@ -150,8 +198,6 @@ Reactor::Reactor(Messenger* messenger, << yb::ToString(connection_keepalive_time_) << ", coarse timer granularity: " << yb::ToString(coarse_timer_granularity_); - process_outbound_queue_task_ = - MakeFunctorReactorTask(std::bind(&Reactor::ProcessOutboundQueue, this), SOURCE_LOCATION()); } Reactor::~Reactor() { @@ -160,9 +206,20 @@ Reactor::~Reactor() { } Status Reactor::Init() { - DCHECK(thread_.get() == nullptr) << "Already started"; DVLOG_WITH_PREFIX(6) << "Called Reactor::Init()"; - // Register to get async notifications in our epoll loop. + auto old_state = ReactorState::kNotStarted; + { + // Even though state_ is atomic, we still need to take the lock to make sure state_ + // and pending_tasks_ are being modified in a consistent way. + std::lock_guard pending_tasks_lock(pending_tasks_mtx_); + SCHECK( + state_.compare_exchange_strong( + old_state, ReactorState::kRunning, std::memory_order_acq_rel), + IllegalState, + "State was expected to be kNotStarted but was $1 during reactor initialization", + old_state); + } + // Register to get async notifications in our libev loop. async_.set(loop_); async_.set(this); async_.start(); @@ -176,42 +233,48 @@ Status Reactor::Init() { ToSeconds(coarse_timer_granularity_)); // Create Reactor thread. - const std::string group_name = messenger_->name() + "_reactor"; + const std::string group_name = messenger_.name() + "_reactor"; return yb::Thread::Create(group_name, group_name, &Reactor::RunThread, this, &thread_); } -void Reactor::Shutdown() { - ReactorState old_state = ReactorState::kRunning; - do { - if (state_.compare_exchange_weak(old_state, - ReactorState::kClosing, - std::memory_order_acq_rel)) { - VLOG_WITH_PREFIX(1) << "shutting down Reactor thread."; - WakeThread(); - } - } while (!HasReactorStartedClosing(old_state)); - - // Another thread already switched the state to closing before us. +void Reactor::StartShutdown() { + LOG_IF_WITH_PREFIX(FATAL, IsCurrentThread()) + << __PRETTY_FUNCTION__ << " cannot be called from the reactor thread"; + auto old_state = ReactorState::kRunning; + bool state_change_success; + { + // Even though state_ is atomic, we still need to take the lock to make sure state_ + // and pending_tasks_ are being modified in a consistent way. + std::lock_guard pending_tasks_lock(pending_tasks_mtx_); + state_change_success = state_.compare_exchange_strong( + old_state, ReactorState::kClosing, std::memory_order_acq_rel); + } + if (state_change_success) { + VLOG_WITH_PREFIX(1) << "shutting down Reactor thread."; + WakeThread(); + } else if (!HasReactorStartedClosing(old_state)) { + LOG_WITH_PREFIX(WARNING) << __PRETTY_FUNCTION__ << " called with an unexpected reactor state: " + << old_state; + } } void Reactor::ShutdownConnection(const ConnectionPtr& conn) { - DCHECK(IsCurrentThread()); - + CheckCurrentThread(); VLOG_WITH_PREFIX(1) << "shutting down " << conn->ToString(); - conn->Shutdown(ServiceUnavailableError()); + if (!conn->shutdown_completed()) { + conn->Shutdown(ServiceUnavailableError()); + } if (!conn->context().Idle()) { VLOG_WITH_PREFIX(1) << "connection is not idle: " << conn->ToString(); - std::weak_ptr weak_conn(conn); - conn->context().ListenIdle([this, weak_conn]() { - DCHECK(IsCurrentThreadOrStartedClosing()); + conn->context().ListenIdle([this, weak_conn = std::weak_ptr(conn)]() { auto conn = weak_conn.lock(); if (conn) { VLOG_WITH_PREFIX(1) << "connection became idle " << conn->ToString(); - // The access to waiting_conns_ is safe here, because this code can only be called on the - // reactor thread or when holding final_abort_mutex_ during shutdown. + std::lock_guard lock(waiting_conns_mtx_); waiting_conns_.erase(conn); } }); + std::lock_guard lock(waiting_conns_mtx_); waiting_conns_.insert(conn); } else { VLOG_WITH_PREFIX(1) << "connection is idle: " << conn->ToString(); @@ -219,10 +282,8 @@ void Reactor::ShutdownConnection(const ConnectionPtr& conn) { } void Reactor::ShutdownInternal() { - DCHECK(IsCurrentThread()); - + CheckCurrentThread(); stopping_ = true; - stop_start_time_ = CoarseMonoClock::Now(); // Tear down any outbound TCP connections. VLOG_WITH_PREFIX(1) << "tearing down outbound TCP connections..."; @@ -252,25 +313,28 @@ void Reactor::ShutdownInternal() { // async_handler_tasks_ are the tasks added by ScheduleReactorTask. VLOG_WITH_PREFIX(1) << "aborting async handler tasks"; - for (const auto& task : async_handler_tasks_) { + for (const auto& task : pending_tasks_being_processed_) { task->Abort(aborted); } VLOG_WITH_PREFIX(1) << "aborting outbound calls"; CHECK(processing_outbound_queue_.empty()) << yb::ToString(processing_outbound_queue_); { - std::lock_guard lock(outbound_queue_lock_); + std::lock_guard lock(outbound_queue_mtx_); outbound_queue_stopped_ = true; outbound_queue_.swap(processing_outbound_queue_); } for (auto& call : processing_outbound_queue_) { - call->Transferred(aborted, nullptr); + call->Transferred(aborted, /* conn= */ nullptr); } processing_outbound_queue_.clear(); + + completed_call_queue_->Shutdown(); } Status Reactor::GetMetrics(ReactorMetrics *metrics) { return RunOnReactorThread([metrics](Reactor* reactor) { + ReactorThreadRoleGuard guard; metrics->num_client_connections = reactor->client_conns_.size(); metrics->num_server_connections = reactor->server_conns_.size(); return Status::OK(); @@ -292,24 +356,31 @@ void Reactor::Join() { thread_->Join(); } -void Reactor::QueueEventOnAllConnections( +Status Reactor::QueueEventOnAllConnections( ServerEventListPtr server_event, const SourceLocation& source_location) { - ScheduleReactorFunctor([server_event = std::move(server_event)](Reactor* reactor) { + return ScheduleReactorFunctor([server_event = std::move(server_event)](Reactor* reactor) { + ReactorThreadRoleGuard guard; for (const ConnectionPtr& conn : reactor->server_conns_) { - conn->QueueOutboundData(server_event); + auto queuing_status = conn->QueueOutboundData(server_event); + LOG_IF(DFATAL, !queuing_status.ok()) + << "Could not queue a server event in " + << __PRETTY_FUNCTION__ << ": " << queuing_status; } }, source_location); } -void Reactor::QueueEventOnFilteredConnections( +Status Reactor::QueueEventOnFilteredConnections( ServerEventListPtr server_event, const SourceLocation& source_location, ConnectionFilter connection_filter) { - ScheduleReactorFunctor( - [server_event = std::move(server_event), - connection_filter = std::move(connection_filter)](Reactor* reactor) { + return ScheduleReactorFunctor([server_event = std::move(server_event), + connection_filter = std::move(connection_filter)](Reactor* reactor) { + ReactorThreadRoleGuard guard; for (const ConnectionPtr& conn : reactor->server_conns_) { if (connection_filter(conn)) { - conn->QueueOutboundData(server_event); + auto queuing_status = conn->QueueOutboundData(server_event); + LOG_IF(DFATAL, !queuing_status.ok()) + << "Could not queue a server event in " + << __PRETTY_FUNCTION__ << ": " << queuing_status; } } }, @@ -319,6 +390,7 @@ void Reactor::QueueEventOnFilteredConnections( Status Reactor::DumpRunningRpcs(const DumpRunningRpcsRequestPB& req, DumpRunningRpcsResponsePB* resp) { return RunOnReactorThread([&req, resp](Reactor* reactor) -> Status { + ReactorThreadRoleGuard guard; for (const ConnectionPtr& conn : reactor->server_conns_) { RETURN_NOT_OK(conn->DumpPB(req, resp->add_inbound_connections())); } @@ -335,25 +407,31 @@ void Reactor::WakeThread() { } void Reactor::CheckReadyToStop() { - DCHECK(IsCurrentThread()); - - VLOG_WITH_PREFIX(4) << "Check ready to stop: " << thread_->ToString() << ", " - << "waiting connections: " << waiting_conns_.size(); + CheckCurrentThread(); - if (VLOG_IS_ON(4)) { - for (const auto& conn : waiting_conns_) { - VLOG_WITH_PREFIX(4) << "Connection: " << conn->ToString() << ", idle=" << conn->Idle() - << ", why: " << conn->ReasonNotIdle(); + bool all_connections_idle; + { + std::lock_guard lock(waiting_conns_mtx_); + all_connections_idle = waiting_conns_.empty(); + if (VLOG_IS_ON(4)) { + VLOG_WITH_PREFIX(4) << "Check ready to stop: " << thread_->ToString() << ", " + << "waiting connections: " << waiting_conns_.size(); + + for (const auto& conn : waiting_conns_) { + VLOG_WITH_PREFIX(4) << "Connection: " << conn->ToString() + << ", idle=" << conn->Idle() + << ", why: " << conn->ReasonNotIdle(); + } } } - if (waiting_conns_.empty()) { + if (all_connections_idle) { VLOG_WITH_PREFIX(4) << "Reactor ready to stop, breaking loop: " << this; VLOG_WITH_PREFIX(2) << "Marking reactor as closed: " << thread_.get()->ToString(); ReactorTasks final_tasks; { - std::lock_guard lock(pending_tasks_mtx_); + std::lock_guard pending_tasks_lock(pending_tasks_mtx_); state_.store(ReactorState::kClosed, std::memory_order_release); final_tasks.swap(pending_tasks_); } @@ -372,26 +450,34 @@ void Reactor::CheckReadyToStop() { void Reactor::AsyncHandler(ev::async &watcher, int revents) { VLOG_WITH_PREFIX_AND_FUNC(4) << "Events: " << revents; - DCHECK(IsCurrentThread()); - - auto se = ScopeExit([this] { - async_handler_tasks_.clear(); + auto scope_exit = ScopeExit([this] { + // Thread safety analysis does not realize that this scope exit closure only runs in this + // function that is already running on reactor thread. + ReactorThreadRoleGuard guard; + pending_tasks_being_processed_.clear(); }); - if (PREDICT_FALSE(DrainTaskQueueAndCheckIfClosing())) { + CHECK(pending_tasks_being_processed_.empty()); + + { + std::lock_guard pending_tasks_lock(pending_tasks_mtx_); + pending_tasks_being_processed_.swap(pending_tasks_); + } + + if (PREDICT_FALSE(HasReactorStartedClosing(state()))) { + // ShutdownInternal will abort every task in pending_tasks_being_processed_. ShutdownInternal(); CheckReadyToStop(); return; } - for (const auto &task : async_handler_tasks_) { + for (const auto &task : pending_tasks_being_processed_) { task->Run(this); } } void Reactor::RegisterConnection(const ConnectionPtr& conn) { - DCHECK(IsCurrentThread()); - + CheckCurrentThread(); Status s = conn->Start(&loop_); if (s.ok()) { server_conns_.push_back(conn); @@ -401,33 +487,31 @@ void Reactor::RegisterConnection(const ConnectionPtr& conn) { } ConnectionPtr Reactor::AssignOutboundCall(const OutboundCallPtr& call) { - DCHECK(IsCurrentThread()); + CheckCurrentThread(); ConnectionPtr conn; - - // TODO: Move call deadline timeout computation into OutboundCall constructor. - const MonoDelta &timeout = call->controller()->timeout(); - MonoTime deadline; - if (!timeout.Initialized()) { - LOG_WITH_PREFIX(WARNING) << "Client call " << call->remote_method().ToString() - << " has no timeout set for connection id: " - << call->conn_id().ToString(); - deadline = MonoTime::Max(); - } else { - deadline = MonoTime::Now(); - deadline.AddDelta(timeout); - } - - Status s = FindOrStartConnection(call->conn_id(), call->hostname(), deadline, &conn); + Status s = FindOrStartConnection(call->conn_id(), call->hostname(), &conn); if (PREDICT_FALSE(!s.ok())) { call->SetFailed(s); return ConnectionPtr(); } + call->SetConnection(conn); + call->SetCompletedCallQueue(completed_call_queue_); conn->QueueOutboundCall(call); + + if (ShouldTrackOutboundCalls()) { + auto expires_at = call->expires_at(); + tracked_outbound_calls_.insert(TrackedOutboundCall { + .call_id = call->call_id(), + .call_weak = call, + .next_check_time = expires_at == CoarseTimePoint::max() + ? call->start_time() + FLAGS_stuck_outbound_call_default_timeout_sec * 1s + : expires_at + FLAGS_reactor_based_outbound_call_expiration_delay_ms * 1ms + }); + } return conn; } -// // Handles timer events. The periodic timer: // // 1. updates Reactor::cur_time_ @@ -435,7 +519,7 @@ ConnectionPtr Reactor::AssignOutboundCall(const OutboundCallPtr& call) { // tcp_conn_timeo_ seconds. // void Reactor::TimerHandler(ev::timer &watcher, int revents) { - DCHECK(IsCurrentThread()); + CheckCurrentThread(); if (EV_ERROR & revents) { LOG_WITH_PREFIX(WARNING) << "Reactor got an error in the timer handler."; @@ -449,32 +533,33 @@ void Reactor::TimerHandler(ev::timer &watcher, int revents) { auto now = CoarseMonoClock::Now(); VLOG_WITH_PREFIX(4) << "timer tick at " << ToSeconds(now.time_since_epoch()); - cur_time_ = now; + cur_time_.store(now, std::memory_order_release); ScanIdleConnections(); + + ScanForStuckOutboundCalls(now); } void Reactor::ScanIdleConnections() { - DCHECK(IsCurrentThread()); if (connection_keepalive_time_ == CoarseMonoClock::Duration::zero()) { VLOG_WITH_PREFIX(3) << "Skipping Idle connections check since connection_keepalive_time_ = 0"; return; } // enforce TCP connection timeouts - auto c = server_conns_.begin(); - auto c_end = server_conns_.end(); + auto conn_iter = server_conns_.begin(); + auto conn_iter_end = server_conns_.end(); uint64_t timed_out = 0; - for (; c != c_end; ) { - const ConnectionPtr& conn = *c; + for (; conn_iter != conn_iter_end; ) { + const ConnectionPtr& conn = *conn_iter; if (!conn->Idle()) { VLOG_WITH_PREFIX(3) << "Connection " << conn->ToString() << " not idle"; - ++c; // TODO: clean up this loop + ++conn_iter; // TODO: clean up this loop continue; } - auto last_activity_time = conn->last_activity_time(); - auto connection_delta = cur_time_ - last_activity_time; + auto conn_last_activity_time = conn->last_activity_time(); + auto connection_delta = cur_time() - conn_last_activity_time; if (connection_delta > connection_keepalive_time_) { conn->Shutdown(STATUS_FORMAT( NetworkError, "Connection timed out after $0", ToSeconds(connection_delta))); @@ -482,12 +567,13 @@ void Reactor::ScanIdleConnections() { << "DEBUG: Closing idle connection: " << conn->ToString() << " - it has been idle for " << ToSeconds(connection_delta) << "s"; VLOG(1) << "(delta: " << ToSeconds(connection_delta) - << ", current time: " << ToSeconds(cur_time_.time_since_epoch()) - << ", last activity time: " << ToSeconds(last_activity_time.time_since_epoch()) << ")"; - server_conns_.erase(c++); + << ", current time: " << ToSeconds(cur_time().time_since_epoch()) + << ", last activity time: " + << ToSeconds(conn_last_activity_time.time_since_epoch()) << ")"; + server_conns_.erase(conn_iter++); ++timed_out; } else { - ++c; + ++conn_iter; } } @@ -501,11 +587,6 @@ bool Reactor::IsCurrentThread() const { return thread_.get() == yb::Thread::current_thread(); } -bool Reactor::IsCurrentThreadOrStartedClosing() const { - return thread_.get() == yb::Thread::current_thread() || - HasReactorStartedClosing(state_.load(std::memory_order_acquire)); -} - void Reactor::RunThread() { ThreadRestrictions::SetWaitAllowed(false); ThreadRestrictions::SetIOAllowed(false); @@ -548,16 +629,17 @@ Result> CreateStream( Status Reactor::FindOrStartConnection(const ConnectionId &conn_id, const std::string& hostname, - const MonoTime &deadline, ConnectionPtr* conn) { - DCHECK(IsCurrentThread()); - auto c = client_conns_.find(conn_id); - if (c != client_conns_.end()) { - *conn = (*c).second; - return Status::OK(); + CheckCurrentThread(); + { + auto conn_iter = client_conns_.find(conn_id); + if (conn_iter != client_conns_.end()) { + *conn = (*conn_iter).second; + return Status::OK(); + } } - if (HasReactorStartedClosing(state_.load(std::memory_order_acquire))) { + if (state() != ReactorState::kRunning) { return ServiceUnavailableError(); } @@ -567,9 +649,9 @@ Status Reactor::FindOrStartConnection(const ConnectionId &conn_id, // Create a new socket and start connecting to the remote. auto sock = VERIFY_RESULT(CreateClientSocket(conn_id.remote())); - if (messenger_->has_outbound_ip_base_.load(std::memory_order_acquire) && - !messenger_->test_outbound_ip_base_.is_unspecified()) { - auto address_bytes(messenger_->test_outbound_ip_base_.to_v4().to_bytes()); + if (messenger_.has_outbound_ip_base_.load(std::memory_order_acquire) && + !messenger_.test_outbound_ip_base_.is_unspecified()) { + auto address_bytes(messenger_.test_outbound_ip_base_.to_v4().to_bytes()); // Use different addresses for public/private endpoints. // Private addresses are even, and public are odd. // So if base address is "private" and destination address is "public" we will modify @@ -584,8 +666,8 @@ Status Reactor::FindOrStartConnection(const ConnectionId &conn_id, << status; } else if (FLAGS_local_ip_for_outbound_sockets.empty()) { auto outbound_address = conn_id.remote().address().is_v6() - ? messenger_->outbound_address_v6() - : messenger_->outbound_address_v4(); + ? messenger_.outbound_address_v6() + : messenger_.outbound_address_v4(); if (!outbound_address.is_unspecified()) { auto status = sock.SetReuseAddr(true); if (status.ok()) { @@ -604,23 +686,23 @@ Status Reactor::FindOrStartConnection(const ConnectionId &conn_id, auto receive_buffer_size = PatchReceiveBufferSize(VERIFY_RESULT(sock.GetReceiveBufferSize())); auto stream = VERIFY_RESULT(CreateStream( - messenger_->stream_factories_, conn_id.protocol(), + messenger_.stream_factories_, conn_id.protocol(), StreamCreateData { .remote = conn_id.remote(), .remote_hostname = hostname, .socket = &sock, .receive_buffer_size = receive_buffer_size, - .mem_tracker = messenger_->connection_context_factory_->buffer_tracker(), - .metric_entity = messenger_->metric_entity(), + .mem_tracker = messenger_.connection_context_factory_->buffer_tracker(), + .metric_entity = messenger_.metric_entity(), })); - auto context = messenger_->connection_context_factory_->Create(receive_buffer_size); + auto context = messenger_.connection_context_factory_->Create(receive_buffer_size); // Register the new connection in our map. auto connection = std::make_shared( this, std::move(stream), ConnectionDirection::CLIENT, - messenger()->rpc_metrics().get(), + messenger_.rpc_metrics().get(), std::move(context)); RETURN_NOT_OK(connection->Start(&loop_)); @@ -653,7 +735,7 @@ void Reactor::DropWithRemoteAddress(const IpAddress& address) { } void Reactor::DropIncomingWithRemoteAddress(const IpAddress& address) { - DCHECK(IsCurrentThread()); + CheckCurrentThread(); VLOG_WITH_PREFIX(1) << "Dropping Incoming connections from " << address; for (auto& conn : server_conns_) { @@ -662,7 +744,7 @@ void Reactor::DropIncomingWithRemoteAddress(const IpAddress& address) { } void Reactor::DropOutgoingWithRemoteAddress(const IpAddress& address) { - DCHECK(IsCurrentThread()); + CheckCurrentThread(); VLOG_WITH_PREFIX(1) << "Dropping Outgoing connections to " << address; for (auto& pair : client_conns_) { ShutdownIfRemoteAddressIs(pair.second, address); @@ -670,7 +752,7 @@ void Reactor::DropOutgoingWithRemoteAddress(const IpAddress& address) { } void Reactor::DestroyConnection(Connection *conn, const Status &conn_status) { - DCHECK(IsCurrentThread()); + CheckCurrentThread(); VLOG_WITH_PREFIX(3) << "DestroyConnection(" << conn->ToString() << ", " << conn_status.ToString() << ")"; @@ -713,17 +795,25 @@ void Reactor::DestroyConnection(Connection *conn, const Status &conn_status) { void Reactor::ProcessOutboundQueue() { CHECK(processing_outbound_queue_.empty()) << yb::ToString(processing_outbound_queue_); { - std::lock_guard lock(outbound_queue_lock_); + std::lock_guard lock(outbound_queue_mtx_); outbound_queue_.swap(processing_outbound_queue_); } if (processing_outbound_queue_.empty()) { return; } + // Find the set of unique connections assigned to the given set of calls, and notify each + // connection that outbound data has been queued. Create new connections as needed. + + CHECK(processing_connections_.empty()); processing_connections_.reserve(processing_outbound_queue_.size()); for (auto& call : processing_outbound_queue_) { auto conn = AssignOutboundCall(call); - processing_connections_.push_back(std::move(conn)); + // The returned connection can be null in case of an error. The call should have already been + // set as failed in that case. + if (conn) { + processing_connections_.push_back(std::move(conn)); + } } processing_outbound_queue_.clear(); @@ -732,7 +822,8 @@ void Reactor::ProcessOutboundQueue() { processing_connections_.erase(new_end, processing_connections_.end()); for (auto& conn : processing_connections_) { if (conn) { - conn->OutboundQueued(); + // If this fails, the connection will be destroyed. + auto ignored_status = conn->OutboundQueued(); } } processing_connections_.clear(); @@ -745,7 +836,7 @@ void Reactor::QueueOutboundCall(OutboundCallPtr call) { bool was_empty = false; bool closing = false; { - std::lock_guard lock(outbound_queue_lock_); + std::lock_guard lock(outbound_queue_mtx_); if (!outbound_queue_stopped_) { was_empty = outbound_queue_.empty(); outbound_queue_.push_back(call); @@ -758,8 +849,12 @@ void Reactor::QueueOutboundCall(OutboundCallPtr call) { return; } if (was_empty) { - auto scheduled = ScheduleReactorTask(process_outbound_queue_task_); - LOG_IF_WITH_PREFIX(WARNING, !scheduled) << "Failed to schedule process outbound queue task"; + auto scheduling_status = ScheduleReactorTask(process_outbound_queue_task_); + // We should not call call->Transferred(scheduling_status, ...) here, because the call has + // already been added to the outbound queue. ShutdownInternal will abort it. + LOG_IF_WITH_PREFIX(WARNING, !scheduling_status.ok()) + << "Failed to schedule outbound queue processing task: " + << scheduling_status; } TRACE_TO(call->trace(), "Scheduled."); } @@ -795,12 +890,13 @@ DelayedTask::DelayedTask(StatusFunctor func, MonoDelta when, int64_t id, func_(std::move(func)), when_(when), id_(id), - messenger_(messenger) { + messenger_(*messenger) { } void DelayedTask::Run(Reactor* reactor) { - DCHECK(reactor_ == nullptr) << "Task has already been scheduled"; - DCHECK(reactor->IsCurrentThread()); + reactor->CheckCurrentThread(); + DCHECK(reactor_atomic_.load(std::memory_order_acquire) == nullptr) + << "Task has already been scheduled"; const auto reactor_state = reactor->state(); if (reactor_state != ReactorState::kRunning) { @@ -823,6 +919,7 @@ void DelayedTask::Run(Reactor* reactor) { // Schedule the task to run later. reactor_ = reactor; + reactor_atomic_.store(reactor, std::memory_order_release); timer_.set(reactor->loop_); // timer_ is owned by this task and will be stopped through AbortTask/Abort before this task @@ -860,16 +957,25 @@ void DelayedTask::AbortTask(const Status& abort_status) { << "Status: " << abort_status << ", " << AsString(mark_as_done_result); if (mark_as_done_result == MarkAsDoneResult::kSuccess) { + auto* reactor = reactor_atomic_.load(std::memory_order_acquire); // Stop the libev timer. We don't need to do this in the kNotScheduled case, because the timer // has not started in that case. - if (reactor_->IsCurrentThread()) { + if (reactor->IsCurrentThread()) { + ReactorThreadRoleGuard guard; timer_.stop(); } else { // Must call timer_.stop() on the reactor thread. Keep a refcount to prevent this DelayedTask // from being deleted. If the reactor thread has already been shut down, this will be a no-op. - reactor_->ScheduleReactorFunctor([this, holder = shared_from(this)](Reactor* reactor) { - timer_.stop(); - }, SOURCE_LOCATION()); + auto scheduling_status = + reactor->ScheduleReactorFunctor([this, holder = shared_from(this)](Reactor* reactor) { + ReactorThreadRoleGuard guard; + timer_.stop(); + }, SOURCE_LOCATION()); + LOG_IF(DFATAL, !scheduling_status.ok()) + << "Could not schedule a reactor task to stop the libev timer in " << __PRETTY_FUNCTION__ + << ": " + << "scheduling_status=" << scheduling_status + << ", abort_status=" << abort_status; } } if (mark_as_done_result != MarkAsDoneResult::kAlreadyDone) { @@ -880,30 +986,27 @@ void DelayedTask::AbortTask(const Status& abort_status) { } void DelayedTask::DoAbort(const Status& abort_status) { - if (messenger_ != nullptr) { - messenger_->RemoveScheduledTask(id_); - } + messenger_.RemoveScheduledTask(id_); AbortTask(abort_status); } void DelayedTask::TimerHandler(ev::timer& watcher, int revents) { - DCHECK(reactor_->IsCurrentThread()); + auto* reactor = reactor_atomic_.load(std::memory_order_acquire); auto mark_as_done_result = MarkAsDone(); if (mark_as_done_result != MarkAsDoneResult::kSuccess) { - DCHECK_EQ(MarkAsDoneResult::kAlreadyDone, mark_as_done_result) - << "Can't get kNotScheduled here, because the timer handler is already being called"; + LOG_IF_WITH_PREFIX(DFATAL, mark_as_done_result != MarkAsDoneResult::kAlreadyDone) + << "Unexpected state: " << mark_as_done_result << " (expected kAlreadyDone): " + << "the timer handler is already being called"; return; } // Hold shared_ptr, so this task wouldn't be destroyed upon removal below until func_ is called. auto holder = shared_from(this); - reactor_->scheduled_tasks_.erase(holder); - if (messenger_ != nullptr) { - messenger_->RemoveScheduledTask(id_); - } + reactor->scheduled_tasks_.erase(holder); + messenger_.RemoveScheduledTask(id_); if (EV_ERROR & revents) { std::string msg = "Delayed task got an error in its timer handler"; @@ -914,6 +1017,8 @@ void DelayedTask::TimerHandler(ev::timer& watcher, int revents) { VLOG_WITH_PREFIX_AND_FUNC(4) << "Execute"; func_(Status::OK()); } + // Clear the function to remove all captured resources. + func_.clear(); } // ------------------------------------------------------------------------------------------------ @@ -927,14 +1032,14 @@ void Reactor::RegisterInboundSocket( receive_buffer_size = PatchReceiveBufferSize(receive_buffer_size); auto stream = CreateStream( - messenger_->stream_factories_, messenger_->listen_protocol_, + messenger_.stream_factories_, messenger_.listen_protocol_, StreamCreateData { .remote = remote, .remote_hostname = std::string(), .socket = socket, .receive_buffer_size = receive_buffer_size, .mem_tracker = factory->buffer_tracker(), - .metric_entity = messenger_->metric_entity() + .metric_entity = messenger_.metric_entity() }); if (!stream.ok()) { LOG_WITH_PREFIX(DFATAL) << "Failed to create stream for " << remote << ": " << stream.status(); @@ -943,41 +1048,49 @@ void Reactor::RegisterInboundSocket( auto conn = std::make_shared(this, std::move(*stream), ConnectionDirection::SERVER, - messenger()->rpc_metrics().get(), + messenger_.rpc_metrics().get(), factory->Create(receive_buffer_size)); - ScheduleReactorFunctor([conn = std::move(conn)](Reactor* reactor) { + + auto scheduling_status = ScheduleReactorFunctor([conn = std::move(conn)](Reactor* reactor) { + ReactorThreadRoleGuard guard; reactor->RegisterConnection(conn); }, SOURCE_LOCATION()); + LOG_IF_WITH_PREFIX(DFATAL, !scheduling_status.ok()) + << "Could not schedule a reactor task in " << __PRETTY_FUNCTION__ + << ": " << scheduling_status; } -bool Reactor::ScheduleReactorTask(ReactorTaskPtr task, bool schedule_even_closing) { +Status Reactor::ScheduleReactorTask(ReactorTaskPtr task, bool even_if_not_running) { + // task should never be null, so not using an SCHECK here. + CHECK_NOTNULL(task); bool was_empty; { // Even though state_ is atomic, we still need to take the lock to make sure state_ - // and pending_tasks_mtx_ are being modified in a consistent way. - std::unique_lock pending_lock(pending_tasks_mtx_); - auto state = state_.load(std::memory_order_acquire); - bool failure = schedule_even_closing ? state == ReactorState::kClosed - : HasReactorStartedClosing(state); + // and pending_tasks_ are being modified in a consistent way. + std::lock_guard pending_tasks_lock(pending_tasks_mtx_); + auto current_state = state(); + bool failure = even_if_not_running ? current_state == ReactorState::kClosed + : current_state != ReactorState::kRunning; if (failure) { - return false; + auto msg = Format("$0: Cannot schedule a reactor task. " + "Current state: $1, even_if_not_running: $2", + log_prefix_, current_state, even_if_not_running); + if (HasReactorStartedClosing(current_state)) { + // It is important to use Aborted status here, because it is used by downstream code to + // distinguish between the case when the reactor is closed and the case when the reactor is + // not running yet. + return STATUS_FORMAT(Aborted, msg); + } + return STATUS_FORMAT(ServiceUnavailable, msg); } was_empty = pending_tasks_.empty(); - pending_tasks_.emplace_back(std::move(task)); + pending_tasks_.push_back(std::move(task)); } if (was_empty) { WakeThread(); } - return true; -} - -bool Reactor::DrainTaskQueueAndCheckIfClosing() { - CHECK(async_handler_tasks_.empty()); - - std::lock_guard lock(pending_tasks_mtx_); - async_handler_tasks_.swap(pending_tasks_); - return HasReactorStartedClosing(state_.load(std::memory_order_acquire)); + return Status::OK(); } // Task to call an arbitrary function within the reactor thread. @@ -1009,12 +1122,80 @@ class RunFunctionTask : public ReactorTask { CountDownLatch latch_{1}; }; +void Reactor::ScanForStuckOutboundCalls(CoarseTimePoint now) { + if (!ShouldTrackOutboundCalls()) { + return; + } + + auto& index_by_call_id = tracked_outbound_calls_.get(); + while (auto call_id_opt = completed_call_queue_->Pop()) { + index_by_call_id.erase(*call_id_opt); + } + + auto& index_by_next_check_time = tracked_outbound_calls_.get(); + while (!index_by_next_check_time.empty()) { + auto& entry = *index_by_next_check_time.begin(); + // It is useful to check the next entry even if its scheduled next check time is later than + // now, to erase entries corresponding to completed calls as soon as possible. This alone, + // even without the completed call queue, mostly solves #19090. + auto call = entry.call_weak.lock(); + if (!call || call->callback_invoked()) { + index_by_next_check_time.erase(index_by_next_check_time.begin()); + continue; + } + if (entry.next_check_time > now) { + break; + } + + // Normally, timeout should be enforced at the connection level. Here, we will catch failures + // to do that. + const bool forcing_timeout = + !call->callback_triggered() && now >= ExpirationEnforcementTime(call->expires_at()); + + auto call_str = call->DebugString(); + + auto conn = call->connection(); + + LOG_WITH_PREFIX(WARNING) << "Stuck OutboundCall: " << call_str + << (forcing_timeout ? " (forcing a timeout)" : ""); + IncrementCounter(messenger_.rpc_metrics()->outbound_calls_stuck); + + if (forcing_timeout) { + // Only do this after we've logged the call, so that the log would capture the call state + // before the forced timeout. + if (conn) { + // This calls SetTimedOut so we don't need to do it directly. + conn->ForceCallExpiration(call); + } else { + LOG_WITH_PREFIX(WARNING) << "Connection is not set for a call that is being forcefully " + << "expired: " << call_str; + call->SetTimedOut(); + } + } + + index_by_next_check_time.modify(index_by_next_check_time.begin(), [now](auto& tracked_call) { + tracked_call.next_check_time = now + FLAGS_stuck_outbound_call_check_interval_sec * 1s; + }); + } + + if (tracked_outbound_calls_.size() >= 1000) { + YB_LOG_WITH_PREFIX_EVERY_N_SECS(WARNING, 1) + << "tracked_outbound_calls_ has a large number of entries: " + << tracked_outbound_calls_.size(); + } +} + +void Reactor::CheckCurrentThread() const { + if (ShouldCheckCurrentThread()) { + CHECK_EQ(thread_.get(), yb::Thread::current_thread()) + << "Operation is not running on the correct reactor thread"; + } +} + template Status Reactor::RunOnReactorThread(const F& f, const SourceLocation& source_location) { auto task = std::make_shared>(f, source_location); - if (!ScheduleReactorTask(task)) { - return ServiceUnavailableError(); - } + RETURN_NOT_OK(ScheduleReactorTask(task)); return task->Wait(); } diff --git a/src/yb/rpc/reactor.h b/src/yb/rpc/reactor.h index a2ca07335ad..da33f3b6be4 100644 --- a/src/yb/rpc/reactor.h +++ b/src/yb/rpc/reactor.h @@ -47,6 +47,10 @@ #include #include +#include +#include +#include +#include #include #include // NOLINT #include @@ -57,17 +61,19 @@ #include "yb/gutil/ref_counted.h" #include "yb/rpc/outbound_call.h" +#include "yb/rpc/reactor_thread_role.h" -#include "yb/util/status_fwd.h" #include "yb/util/async_util.h" #include "yb/util/condition_variable.h" #include "yb/util/countdown_latch.h" #include "yb/util/locks.h" +#include "yb/util/mem_tracker.h" #include "yb/util/monotime.h" #include "yb/util/mutex.h" #include "yb/util/net/socket.h" #include "yb/util/shared_lock.h" #include "yb/util/source_location.h" +#include "yb/util/status_fwd.h" namespace yb { namespace rpc { @@ -241,7 +247,7 @@ class DelayedTask : public ReactorTask { const SourceLocation& source_location, Messenger* messenger); // Schedules the task for running later but doesn't actually run it yet. - void Run(Reactor* reactor) override; + void Run(Reactor* reactor) ON_REACTOR_THREAD override; // Could be called from non-reactor thread even before reactor thread shutdown. void AbortTask(const Status& abort_status); @@ -256,10 +262,10 @@ class DelayedTask : public ReactorTask { void DoAbort(const Status& abort_status) override; // Set done_ to true if not set and return true. If done_ is already set, return false. - MarkAsDoneResult MarkAsDone(); + MarkAsDoneResult MarkAsDone() EXCLUDES(lock_); // libev callback for when the registered timer fires. - void TimerHandler(ev::timer& rwatcher, int revents); // NOLINT + void TimerHandler(ev::timer& rwatcher, int revents) ON_REACTOR_THREAD; // NOLINT // User function to invoke when timer fires or when task is aborted. StatusFunctor func_; @@ -268,19 +274,21 @@ class DelayedTask : public ReactorTask { const MonoDelta when_; // Link back to registering reactor thread. - Reactor* reactor_ = nullptr; + Reactor* reactor_ GUARDED_BY(lock_) = nullptr; + + // Same as reactor_, but can be accessed without holding lock_. + std::atomic reactor_atomic_{nullptr}; // libev timer. Set when Run() is invoked. - ev::timer timer_; + ev::timer timer_ GUARDED_BY_REACTOR_THREAD; // This task's id. const int64_t id_; - Messenger* const messenger_; + Messenger& messenger_; // Set to true whenever a Run or Abort methods are called. - // Guarded by lock_. - bool done_ = false; + bool done_ GUARDED_BY(lock_) = false; typedef simple_spinlock LockType; mutable LockType lock_; @@ -288,7 +296,17 @@ class DelayedTask : public ReactorTask { typedef std::vector ReactorTasks; -YB_DEFINE_ENUM(ReactorState, (kRunning)(kClosing)(kClosed)); +YB_DEFINE_ENUM(ReactorState, + // The reactor is not fully initialized. + (kNotStarted) + // The reactor has been initialized and started running. + (kRunning) + // Shutdown() has been called. + (kClosing) + // The internal shutdown sequence (running on reactor thread) has completed. + // Immediately after transitioning to this state, it might still take a small amount + // of time for the final tasks to be aborted, and for the libev loop to be broken. + (kClosed)); class Reactor { public: @@ -301,53 +319,66 @@ class Reactor { ~Reactor(); - Reactor(const Reactor&) = delete; - void operator=(const Reactor&) = delete; + // libev callback for handling async notifications in our epoll thread. + void AsyncHandler(ev::async &watcher, int revents) ON_REACTOR_THREAD; // NOLINT + + // libev callback for handling timer events in our epoll thread. + void TimerHandler(ev::timer &watcher, int revents) ON_REACTOR_THREAD; // NOLINT // This may be called from another thread. - Status Init(); + const std::string &name() const { return name_; } - // Add any connections on this reactor thread into the given status dump. - // May be called from another thread. - Status DumpRunningRpcs( - const DumpRunningRpcsRequestPB& req, DumpRunningRpcsResponsePB* resp); + const std::string& LogPrefix() { return log_prefix_; } - // Block until the Reactor thread is shut down - // - // This must be called from another thread. - void Shutdown(); + Messenger& messenger() const { return messenger_; } - // This method is thread-safe. - void WakeThread(); + // Returns the latest current time updated by this reactor's thread. + CoarseTimePoint cur_time() const { return cur_time_.load(std::memory_order_acquire); } - // libev callback for handling async notifications in our epoll thread. - void AsyncHandler(ev::async &watcher, int revents); // NOLINT + // Schedule the given task's Run() method to be called on the reactor thread. If the reactor shuts + // down before it is run, the Abort method will be called. + // + // Can be called on any thread, including the reactor thread. + Status ScheduleReactorTask(ReactorTaskPtr task) { + return ScheduleReactorTask(std::move(task), false /* even_if_not_running */); + } - // libev callback for handling timer events in our epoll thread. - void TimerHandler(ev::timer &watcher, int revents); // NOLINT + template + Status ScheduleReactorFunctor(const F& f, const SourceLocation& source_location) { + return ScheduleReactorTask(MakeFunctorReactorTask(f, source_location)); + } - // This may be called from another thread. - const std::string &name() const { return name_; } + // This should not be called on reactor thread, as the reactor thread should not even be running + // yet. + Status Init() EXCLUDES_REACTOR_THREAD; - const std::string& LogPrefix() { return log_prefix_; } + // Start the shutdown process by changing the reactor state to kClosing if it is kRunning. When + // this function returns, the reactor might still be shutting down. + void StartShutdown() EXCLUDES_REACTOR_THREAD; - Messenger *messenger() const { return messenger_; } + // Add any connections on this reactor thread into the given status dump. + Status DumpRunningRpcs( + const DumpRunningRpcsRequestPB& req, DumpRunningRpcsResponsePB* resp) + EXCLUDES_REACTOR_THREAD; - CoarseTimePoint cur_time() const { return cur_time_; } + // Causes AsyncHandler() to be called on the reactor thread. We should never need to call this + // function from the reactor thread. + void WakeThread() EXCLUDES_REACTOR_THREAD; // Drop all connections with remote address. Used in tests with broken connectivity. - void DropIncomingWithRemoteAddress(const IpAddress& address); - void DropOutgoingWithRemoteAddress(const IpAddress& address); - void DropWithRemoteAddress(const IpAddress& address); + void DropIncomingWithRemoteAddress(const IpAddress& address) ON_REACTOR_THREAD; + void DropOutgoingWithRemoteAddress(const IpAddress& address) ON_REACTOR_THREAD; + void DropWithRemoteAddress(const IpAddress& address) ON_REACTOR_THREAD; // Return true if this reactor thread is the thread currently // running. Should be used in DCHECK assertions. bool IsCurrentThread() const; - // Return true if this reactor thread is the thread currently running, or the reactor is closing. - // This is the condition under which the Abort method can be called for tasks. Should be used in - // DCHECK assertions. - bool IsCurrentThreadOrStartedClosing() const; + // Checks that the current thread is this reactor's thread. The check is always performed in + // debug mode, and in release mode only if FLAGS_reactor_check_current_thread is set. + // This function should be called from functions that are scheduled on the reactor thread as + // callbacks and are not supposed to be called directly. + void CheckCurrentThread() const; // Shut down the given connection, removing it from the connection tracking // structures of this reactor. @@ -355,24 +386,26 @@ class Reactor { // The connection is not explicitly deleted -- shared_ptr reference counting // may hold on to the object after this, but callers should assume that it // _may_ be deleted by this call. - void DestroyConnection(Connection *conn, const Status &conn_status); + void DestroyConnection(Connection *conn, const Status &conn_status) ON_REACTOR_THREAD; // Queue a new call to be sent. If the reactor is already shut down, marks // the call as failed. - void QueueOutboundCall(OutboundCallPtr call); + void QueueOutboundCall(OutboundCallPtr call) + EXCLUDES_REACTOR_THREAD EXCLUDES(outbound_queue_mtx_); // Collect metrics. // Must be called from the reactor thread. - Status GetMetrics(ReactorMetrics *metrics); + Status GetMetrics(ReactorMetrics *metrics) EXCLUDES_REACTOR_THREAD; - void Join(); + void Join() EXCLUDES_REACTOR_THREAD; // Queues a server event on all the connections, such that every client receives it. - void QueueEventOnAllConnections( - ServerEventListPtr server_event, const SourceLocation& source_location); + Status QueueEventOnAllConnections( + ServerEventListPtr server_event, const SourceLocation& source_location) + EXCLUDES_REACTOR_THREAD; // Queues a server event on all the connections which pass the filter. - void QueueEventOnFilteredConnections( + Status QueueEventOnFilteredConnections( ServerEventListPtr server_event, const SourceLocation& source_location, ConnectionFilter connection_filter); @@ -381,21 +414,12 @@ class Reactor { // If the reactor is already shut down, takes care of closing the socket. void RegisterInboundSocket( Socket *socket, size_t receive_buffer_size, const Endpoint& remote, - const ConnectionContextFactoryPtr& factory); - - // Schedule the given task's Run() method to be called on the reactor thread. If the reactor shuts - // down before it is run, the Abort method will be called. - // Returns true if task was scheduled. - MUST_USE_RESULT bool ScheduleReactorTask(ReactorTaskPtr task) { - return ScheduleReactorTask(std::move(task), false /* schedule_even_closing */); - } + const ConnectionContextFactoryPtr& factory) + EXCLUDES_REACTOR_THREAD; - template - bool ScheduleReactorFunctor(const F& f, const SourceLocation& source_location) { - return ScheduleReactorTask(MakeFunctorReactorTask(f, source_location)); - } - - ReactorState state() { + // We use NO_THREAD_SAFETY_ANALYSIS here because state_ is atomic, despite only being written + // when pending_tasks_mtx_ is locked. + ReactorState state() const NO_THREAD_SAFETY_ANALYSIS { return state_.load(std::memory_order_acquire); } @@ -407,74 +431,84 @@ class Reactor { // Run the main event loop of the reactor. void RunThread(); - MUST_USE_RESULT bool ScheduleReactorTask(ReactorTaskPtr task, bool schedule_even_closing); + // Schedules a task on the reactor thread. Returns an ServiceUnavailalbe if the reactor is not + // in a valid state, defined as: + // - If even_if_not_running is true, the only invalid state is kClosed. + // - If even_if_not_running is false, the only valid state is kRunning. + // Can be called on any thread, including the reactor thread. + Status ScheduleReactorTask(ReactorTaskPtr task, bool even_if_not_running); // Find or create a new connection to the given remote. // If such a connection already exists, returns that, otherwise creates a new one. // May return a bad Status if the connect() call fails. // The resulting connection object is managed internally by the reactor thread. // Deadline specifies latest time allowed for initializing the connection. - Status FindOrStartConnection(const ConnectionId &conn_id, - const std::string& hostname, - const MonoTime &deadline, - ConnectionPtr* conn); + Status FindOrStartConnection( + const ConnectionId &conn_id, + const std::string& hostname, + ConnectionPtr* conn) ON_REACTOR_THREAD; // Scan any open connections for idle ones that have been idle longer than // connection_keepalive_time_ - void ScanIdleConnections(); + void ScanIdleConnections() ON_REACTOR_THREAD; + + // Invoked periodically. Scans tracked_outbound_calls_, checks if any calls have completed + // (callback has been called) or deleted altogether. For calls that stay in that map for too long, + // reports them to the log once in a few seconds. + void ScanForStuckOutboundCalls(CoarseTimePoint now) ON_REACTOR_THREAD; // Assign a new outbound call to the appropriate connection object. // If this fails, the call is marked failed and completed. - ConnectionPtr AssignOutboundCall(const OutboundCallPtr &call); + ConnectionPtr AssignOutboundCall(const OutboundCallPtr &call) ON_REACTOR_THREAD; // Register a new connection. - void RegisterConnection(const ConnectionPtr& conn); + void RegisterConnection(const ConnectionPtr& conn) ON_REACTOR_THREAD; // Actually perform shutdown of the thread, tearing down any connections, // etc. This is called from within the thread. - void ShutdownInternal(); + void ShutdownInternal() ON_REACTOR_THREAD; - void ProcessOutboundQueue(); + void ProcessOutboundQueue() ON_REACTOR_THREAD; - void CheckReadyToStop(); - - // Drains the pending_tasks_ queue into async_handler_tasks_. Returns true if the reactor is - // closing. - bool DrainTaskQueueAndCheckIfClosing(); + void CheckReadyToStop() ON_REACTOR_THREAD; template - Status RunOnReactorThread(const F& f, const SourceLocation& source_location); + Status RunOnReactorThread(const F& f, const SourceLocation& source_location) + EXCLUDES_REACTOR_THREAD; + + void ShutdownConnection(const ConnectionPtr& conn) ON_REACTOR_THREAD; - void ShutdownConnection(const ConnectionPtr& conn); + // ---------------------------------------------------------------------------------------------- + // Fields set in the constructor + // ---------------------------------------------------------------------------------------------- - // parent messenger - Messenger* const messenger_; + Messenger& messenger_; const std::string name_; const std::string log_prefix_; - mutable simple_spinlock pending_tasks_mtx_; + // our epoll object (or kqueue, etc). + ev::dynamic_loop loop_; - // Reactor status, mostly used when shutting down. Guarded by pending_tasks_mtx_, but also read - // without a lock for sanity checking. - std::atomic state_{ReactorState::kRunning}; + // If a connection has been idle for this much time, it is torn down. + const CoarseMonoClock::Duration connection_keepalive_time_; - // This mutex is used to make sure that multiple threads that end up running Abort() in case the - // reactor has already shut down will not have data races accessing data that is normally only - // accessed from the reactor thread. We are using a recursive mutex because Abort() could try to - // submit another reactor task, which will result in Abort() being called on that other task - // as well. - std::recursive_mutex final_abort_mutex_; + // Scan for idle connections on this granularity. + const CoarseMonoClock::Duration coarse_timer_granularity_; - // Tasks to be run within the reactor thread. - // Guarded by pending_tasks_mtx_. - ReactorTasks pending_tasks_; + const ReactorTaskPtr process_outbound_queue_task_; - scoped_refptr thread_; + // Number of outbound connections to create per each destination server address. + const int num_connections_to_server_; - // our epoll object (or kqueue, etc). - ev::dynamic_loop loop_; + const std::shared_ptr completed_call_queue_; + + // ---------------------------------------------------------------------------------------------- + // Fields initialized in Init() + // ---------------------------------------------------------------------------------------------- + + scoped_refptr thread_; // Used by other threads to notify the reactor thread ev::async async_; @@ -482,52 +516,107 @@ class Reactor { // Handles the periodic timer. ev::timer timer_; - // Scheduled (but not yet run) delayed tasks. - std::set> scheduled_tasks_; + // ---------------------------------------------------------------------------------------------- + // Fields protected by pending_tasks_mtx_ + // ---------------------------------------------------------------------------------------------- - ReactorTasks async_handler_tasks_; + mutable simple_spinlock pending_tasks_mtx_; - // The current monotonic time. Updated every coarse_timer_granularity_. - CoarseTimePoint cur_time_; + // Tasks to be run within the reactor thread. + ReactorTasks pending_tasks_ GUARDED_BY(pending_tasks_mtx_); - // last time we did TCP timeouts. - CoarseTimePoint last_unused_tcp_scan_; + // Reactor status, mostly used when shutting down. Guarded by pending_tasks_mtx_, but also read + // without a lock for sanity checking. + std::atomic state_ GUARDED_BY(pending_tasks_mtx_) {ReactorState::kNotStarted}; - // Map of sockaddrs to Connection objects for outbound (client) connections. - ConnectionMap client_conns_; + // ---------------------------------------------------------------------------------------------- + // Fields protected by outbound_queue_mtx_ + // ---------------------------------------------------------------------------------------------- - // List of current connections coming into the server. - ConnectionList server_conns_; + simple_spinlock outbound_queue_mtx_; + + bool outbound_queue_stopped_ GUARDED_BY(outbound_queue_mtx_) = false; + + std::vector outbound_queue_ GUARDED_BY(outbound_queue_mtx_); + + // ---------------------------------------------------------------------------------------------- + // Fields protected by waiting_conns_mtx_ + // ---------------------------------------------------------------------------------------------- + + simple_spinlock waiting_conns_mtx_; // Set of connections that should be completed before we can stop this thread. - std::unordered_set waiting_conns_; + std::unordered_set waiting_conns_ GUARDED_BY(waiting_conns_mtx_); - // If a connection has been idle for this much time, it is torn down. - CoarseMonoClock::Duration connection_keepalive_time_; + // ---------------------------------------------------------------------------------------------- + // Atomic fields + // ---------------------------------------------------------------------------------------------- - // Scan for idle connections on this granularity. - CoarseMonoClock::Duration coarse_timer_granularity_; + // The current monotonic time. Updated every coarse_timer_granularity_. + std::atomic cur_time_; - simple_spinlock outbound_queue_lock_; - bool outbound_queue_stopped_ = false; + // ---------------------------------------------------------------------------------------------- + // Fields that are only accessed on the reactor thread + // ---------------------------------------------------------------------------------------------- - // We found that we should shut down, but not all connections are ready for it. Only accessed in - // the reactor thread. - bool stopping_ = false; + // Scheduled (but not yet run) delayed tasks. + std::set> scheduled_tasks_ GUARDED_BY_REACTOR_THREAD; - CoarseTimePoint stop_start_time_; + // Map of sockaddrs to Connection objects for outbound (client) connections. + ConnectionMap client_conns_ GUARDED_BY_REACTOR_THREAD; - std::vector outbound_queue_; + // List of current connections coming into the server. + ConnectionList server_conns_ GUARDED_BY_REACTOR_THREAD; + + // We found that we should shut down, but not all connections are ready for it. Only accessed in + // the reactor thread. + bool stopping_ GUARDED_BY_REACTOR_THREAD = false; // Outbound calls currently being processed. Only accessed on the reactor thread. Could be a local // variable, but implemented as a member field as an optimization to avoid memory allocation. - std::vector processing_outbound_queue_; + std::vector processing_outbound_queue_ GUARDED_BY_REACTOR_THREAD; - std::vector processing_connections_; - ReactorTaskPtr process_outbound_queue_task_; + // Used in ProcessOutboundQueue to group outbound calls by connection and find the set of unique + // connections that we should try to write to. Should always be empty except when + // ProcessOutboundQueue is executing. An optimization to avoid memory allocation. + std::vector processing_connections_ GUARDED_BY_REACTOR_THREAD; - // Number of outbound connections to create per each destination server address. - int num_connections_to_server_; + // Tasks moved from pending_tasks_ that are currently being processed by AsyncHandler. + ReactorTasks pending_tasks_being_processed_ GUARDED_BY_REACTOR_THREAD; + + ReactorTasks async_handler_tasks_; + + // ---------------------------------------------------------------------------------------------- + // A subsystem for proactively tracking stuck OutboundCalls where the callback has never been + // called. We add calls to tracked_outbound_calls_ in AssignOutboundCall and remove them as soon + // as we find out the callback has been invoked. + + struct TrackedOutboundCall { + int32_t call_id = 0; + OutboundCallWeakPtr call_weak; // If this cannot be locked, we remove the entry. + CoarseTimePoint next_check_time; // Next time to check the callback status. + }; + + class CallIdTag; + class NextCheckTimeTag; + using TrackedOutboundCalls = boost::multi_index_container< + TrackedOutboundCall, + boost::multi_index::indexed_by< + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::member>, + boost::multi_index::ordered_non_unique< + boost::multi_index::tag, + boost::multi_index::member + >>>; + + TrackedOutboundCalls tracked_outbound_calls_ GUARDED_BY_REACTOR_THREAD; + + DISALLOW_COPY_AND_ASSIGN(Reactor); }; } // namespace rpc diff --git a/src/yb/rpc/reactor_thread_role.h b/src/yb/rpc/reactor_thread_role.h new file mode 100644 index 00000000000..b27ac04780a --- /dev/null +++ b/src/yb/rpc/reactor_thread_role.h @@ -0,0 +1,48 @@ +// +// Copyright (c) YugaByte, 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. +// +// + +#pragma once + +#include "yb/util/thread_annotations_util.h" + +namespace yb { +namespace rpc { + +#if defined(__clang__) && __clang_major__ >= 13 + +struct ReactorThreadRole { + static constexpr ThreadRole kReactor{}; + static ThreadRole Alias() RETURN_CAPABILITY(kReactor); +}; + +using ReactorThreadRoleGuard = CapabilityGuard; + +#define ON_REACTOR_THREAD REQUIRES(::yb::rpc::ReactorThreadRole::kReactor) +#define EXCLUDES_REACTOR_THREAD EXCLUDES(::yb::rpc::ReactorThreadRole::kReactor) +#define GUARDED_BY_REACTOR_THREAD GUARDED_BY(::yb::rpc::ReactorThreadRole::kReactor) + +#else + +struct [[maybe_unused]] ReactorThreadRoleGuard { // NOLINT +}; + +#define ON_REACTOR_THREAD +#define EXCLUDES_REACTOR_THREAD +#define GUARDED_BY_REACTOR_THREAD + +#endif + +} // namespace rpc +} // namespace yb diff --git a/src/yb/rpc/refined_stream.cc b/src/yb/rpc/refined_stream.cc index 5f2e9d57126..3ede7cd35f4 100644 --- a/src/yb/rpc/refined_stream.cc +++ b/src/yb/rpc/refined_stream.cc @@ -78,15 +78,15 @@ void RefinedStream::Shutdown(const Status& status) { Result RefinedStream::Send(OutboundDataPtr data) { switch (state_) { - case RefinedStreamState::kInitial: - case RefinedStreamState::kHandshake: - pending_data_.push_back(std::move(data)); - return std::numeric_limits::max(); - case RefinedStreamState::kEnabled: - RETURN_NOT_OK(refiner_->Send(std::move(data))); - return std::numeric_limits::max(); - case RefinedStreamState::kDisabled: - return lower_stream_->Send(std::move(data)); + case RefinedStreamState::kInitial: + case RefinedStreamState::kHandshake: + pending_data_.push_back(std::move(data)); + return kUnknownCallHandle; + case RefinedStreamState::kEnabled: + RETURN_NOT_OK(refiner_->Send(std::move(data))); + return kUnknownCallHandle; + case RefinedStreamState::kDisabled: + return lower_stream_->Send(std::move(data)); } FATAL_INVALID_ENUM_VALUE(RefinedStreamState, state_); diff --git a/src/yb/rpc/refined_stream.h b/src/yb/rpc/refined_stream.h index d327643644d..14a1ac4a413 100644 --- a/src/yb/rpc/refined_stream.h +++ b/src/yb/rpc/refined_stream.h @@ -16,6 +16,7 @@ #include "yb/rpc/circular_read_buffer.h" #include "yb/rpc/stream.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/mem_tracker.h" @@ -29,9 +30,9 @@ YB_DEFINE_ENUM(LocalSide, (kClient)(kServer)); class StreamRefiner { public: virtual void Start(RefinedStream* stream) = 0; - virtual Status ProcessHeader() = 0; + virtual Status ProcessHeader() ON_REACTOR_THREAD = 0; virtual Status Send(OutboundDataPtr data) = 0; - virtual Status Handshake() = 0; + virtual Status Handshake() ON_REACTOR_THREAD = 0; virtual Result Read(StreamReadBuffer* out) = 0; virtual const Protocol* GetProtocol() = 0; @@ -60,7 +61,7 @@ class RefinedStream : public Stream, public StreamContext { const Endpoint& Local() const override; Status Start(bool connect, ev::loop_ref* loop, StreamContext* context) override; void Shutdown(const Status& status) override; - Result Send(OutboundDataPtr data) override; + Result Send(OutboundDataPtr data) ON_REACTOR_THREAD override; bool Cancelled(size_t handle) override; bool IsConnected() override; const Protocol* GetProtocol() override; @@ -68,8 +69,8 @@ class RefinedStream : public Stream, public StreamContext { std::string ToString() const override; // Implementation StreamContext - Result ProcessReceived(ReadBufferFull read_buffer_full) override; - void Connected() override; + Result ProcessReceived(ReadBufferFull read_buffer_full) ON_REACTOR_THREAD override; + void Connected() ON_REACTOR_THREAD override; void UpdateLastActivity() override; void UpdateLastRead() override; @@ -77,8 +78,8 @@ class RefinedStream : public Stream, public StreamContext { void Transferred(const OutboundDataPtr& data, const Status& status) override; void Destroy(const Status& status) override; - Status Established(RefinedStreamState state); - Status SendToLower(OutboundDataPtr data); + Status Established(RefinedStreamState state) ON_REACTOR_THREAD; + Status SendToLower(OutboundDataPtr data) ON_REACTOR_THREAD; Status StartHandshake(); StreamContext& context() const { @@ -94,7 +95,7 @@ class RefinedStream : public Stream, public StreamContext { } private: - Result Handshake(); + Result Handshake() ON_REACTOR_THREAD; Result Read(); std::unique_ptr lower_stream_; diff --git a/src/yb/rpc/rpc.cc b/src/yb/rpc/rpc.cc index 8044e6fb688..5e308c2a58a 100644 --- a/src/yb/rpc/rpc.cc +++ b/src/yb/rpc/rpc.cc @@ -163,21 +163,21 @@ Status RpcRetrier::DoDelayedRetry(RpcCommand* rpc, const Status& why_status) { } auto retain_rpc = rpc->shared_from_this(); - task_id_ = messenger_->ScheduleOnReactor( + auto task_id_result = messenger_->ScheduleOnReactor( std::bind(&RpcRetrier::DoRetry, this, rpc, _1), retry_delay_ + MonoDelta::FromMilliseconds(RandomUniformInt(0, 4)), - SOURCE_LOCATION(), messenger_); + SOURCE_LOCATION()); - // Scheduling state can be changed only in this method, so we expected both - // exchanges below to succeed. + // Scheduling state can be changed only in this method, so we expected both exchanges below to + // succeed. expected_state = RpcRetrierState::kScheduling; - if (task_id_.load(std::memory_order_acquire) == kInvalidTaskId) { - auto result = STATUS_FORMAT(Aborted, "Failed to schedule: $0", rpc); - LOG(WARNING) << result; + if (!task_id_result.ok()) { + task_id_.store(kInvalidTaskId, std::memory_order_release); CHECK(state_.compare_exchange_strong( expected_state, RpcRetrierState::kFinished, std::memory_order_acq_rel)); - return result; + return task_id_result.status(); } + task_id_.store(*task_id_result, std::memory_order_release); CHECK(state_.compare_exchange_strong( expected_state, RpcRetrierState::kWaiting, std::memory_order_acq_rel)); return Status::OK(); diff --git a/src/yb/rpc/rpc_call.cc b/src/yb/rpc/rpc_call.cc index 6b50a2b1bec..30e6c1337f1 100644 --- a/src/yb/rpc/rpc_call.cc +++ b/src/yb/rpc/rpc_call.cc @@ -14,18 +14,28 @@ // #include "yb/rpc/rpc_call.h" +#include "yb/util/logging.h" #include "yb/util/status.h" namespace yb { namespace rpc { void RpcCall::Transferred(const Status& status, const ConnectionPtr& conn) { - if (state_ != TransferState::PENDING) { - LOG(DFATAL) << "Transferred for " << ToString() << " executed twice, state: " - << yb::ToString(state_) << ", status: " << status; - return; + auto transfer_state = transfer_state_.load(std::memory_order_acquire); + for (;;) { + if (transfer_state != TransferState::PENDING) { + LOG_WITH_PREFIX(DFATAL) << __PRETTY_FUNCTION__ << " executed more than once on call " + << static_cast(this) << ", current state: " + << yb::ToString(transfer_state) << " (expected to be PENDING), " + << "status passed to the current Transferred call: " << status << ", " + << "this RpcCall: " << ToString(); + return; + } + auto new_state = status.ok() ? TransferState::FINISHED : TransferState::ABORTED; + if (transfer_state_.compare_exchange_strong(transfer_state, new_state)) { + break; + } } - state_ = status.ok() ? TransferState::FINISHED : TransferState::ABORTED; NotifyTransferred(status, conn); } diff --git a/src/yb/rpc/rpc_call.h b/src/yb/rpc/rpc_call.h index fbd98162f98..095b06097f4 100644 --- a/src/yb/rpc/rpc_call.h +++ b/src/yb/rpc/rpc_call.h @@ -45,10 +45,12 @@ class RpcCall : public OutboundData { return ""; } + TransferState transfer_state() const { return transfer_state_.load(std::memory_order_acquire); } + private: virtual void NotifyTransferred(const Status& status, const ConnectionPtr& conn) = 0; - TransferState state_ = TransferState::PENDING; + std::atomic transfer_state_{TransferState::PENDING}; }; } // namespace rpc diff --git a/src/yb/rpc/rpc_context.cc b/src/yb/rpc/rpc_context.cc index 238125cc966..9dd6b261274 100644 --- a/src/yb/rpc/rpc_context.cc +++ b/src/yb/rpc/rpc_context.cc @@ -250,9 +250,12 @@ void RpcContext::Panic(const char* filepath, int line_number, const string& mess void RpcContext::CloseConnection() { auto connection = call_->connection(); - connection->reactor()->ScheduleReactorFunctor([connection](Reactor*) { - connection->Close(); - }, SOURCE_LOCATION()); + auto closing_status = + connection->reactor()->ScheduleReactorFunctor([connection](Reactor*) { + connection->Close(); + }, SOURCE_LOCATION()); + LOG_IF(DFATAL, !closing_status.ok()) + << "Could not schedule a reactor task to close a connection: " << closing_status; } std::string RpcContext::ToString() const { diff --git a/src/yb/rpc/rpc_controller.cc b/src/yb/rpc/rpc_controller.cc index 8d2067e9c99..943813ee19a 100644 --- a/src/yb/rpc/rpc_controller.cc +++ b/src/yb/rpc/rpc_controller.cc @@ -147,6 +147,7 @@ int32_t RpcController::call_id() const { std::string RpcController::CallStateDebugString() const { std::lock_guard l(lock_); if (call_) { + call_->QueueDumpConnectionState(); return call_->DebugString(); } return "call not set"; diff --git a/src/yb/rpc/rpc_fwd.h b/src/yb/rpc/rpc_fwd.h index fa605cd7e0f..69ac35523bb 100644 --- a/src/yb/rpc/rpc_fwd.h +++ b/src/yb/rpc/rpc_fwd.h @@ -104,6 +104,7 @@ class Messenger; class OutboundCall; typedef std::shared_ptr OutboundCallPtr; +typedef std::weak_ptr OutboundCallWeakPtr; class OutboundData; typedef std::shared_ptr OutboundDataPtr; diff --git a/src/yb/rpc/rpc_metrics.cc b/src/yb/rpc/rpc_metrics.cc index e119c770b99..ac3985c5749 100644 --- a/src/yb/rpc/rpc_metrics.cc +++ b/src/yb/rpc/rpc_metrics.cc @@ -45,6 +45,12 @@ METRIC_DEFINE_counter(server, rpc_outbound_calls_created, yb::MetricUnit::kRequests, "Number of created RPC outbound calls."); +METRIC_DEFINE_counter(server, rpc_outbound_calls_stuck, + "Number of stuck outbound RPC calls.", + yb::MetricUnit::kRequests, + "Number of events where we detected an unreasonably long running outbound " + "RPC call."); + namespace yb { namespace rpc { @@ -56,6 +62,7 @@ RpcMetrics::RpcMetrics(const scoped_refptr& metric_entity) { inbound_calls_created = METRIC_rpc_inbound_calls_created.Instantiate(metric_entity); outbound_calls_alive = METRIC_rpc_outbound_calls_alive.Instantiate(metric_entity, 0); outbound_calls_created = METRIC_rpc_outbound_calls_created.Instantiate(metric_entity); + outbound_calls_stuck = METRIC_rpc_outbound_calls_stuck.Instantiate(metric_entity); } } diff --git a/src/yb/rpc/rpc_metrics.h b/src/yb/rpc/rpc_metrics.h index 0b4c3f71888..b7b806c5e1a 100644 --- a/src/yb/rpc/rpc_metrics.h +++ b/src/yb/rpc/rpc_metrics.h @@ -28,6 +28,7 @@ struct RpcMetrics { scoped_refptr inbound_calls_created; scoped_refptr> outbound_calls_alive; scoped_refptr outbound_calls_created; + scoped_refptr outbound_calls_stuck; }; } // namespace rpc diff --git a/src/yb/rpc/rpc_with_call_id.cc b/src/yb/rpc/rpc_with_call_id.cc index 00546fe630f..67e688a9e05 100644 --- a/src/yb/rpc/rpc_with_call_id.cc +++ b/src/yb/rpc/rpc_with_call_id.cc @@ -62,8 +62,6 @@ void ConnectionContextWithCallId::Shutdown(const Status& status) { } void ConnectionContextWithCallId::CallProcessed(InboundCall* call) { - DCHECK(call->connection()->reactor()->IsCurrentThreadOrStartedClosing()); - ++processed_call_count_; auto id = ExtractCallId(call); auto it = calls_being_handled_.find(id); @@ -79,9 +77,8 @@ void ConnectionContextWithCallId::CallProcessed(InboundCall* call) { } } -void ConnectionContextWithCallId::QueueResponse(const ConnectionPtr& conn, - InboundCallPtr call) { - conn->QueueOutboundData(std::move(call)); +Status ConnectionContextWithCallId::QueueResponse(const ConnectionPtr& conn, InboundCallPtr call) { + return conn->QueueOutboundData(std::move(call)); } } // namespace rpc diff --git a/src/yb/rpc/rpc_with_call_id.h b/src/yb/rpc/rpc_with_call_id.h index 750cd2ce4e7..ddfaffe3518 100644 --- a/src/yb/rpc/rpc_with_call_id.h +++ b/src/yb/rpc/rpc_with_call_id.h @@ -50,7 +50,7 @@ class ConnectionContextWithCallId : public ConnectionContextBase, bool Idle(std::string* reason_not_idle = nullptr) override; void CallProcessed(InboundCall* call) override; - void QueueResponse(const ConnectionPtr& conn, InboundCallPtr call) override; + Status QueueResponse(const ConnectionPtr& conn, InboundCallPtr call) override; // Calls which have been received on the server and are currently // being handled. diff --git a/src/yb/rpc/rpc_with_queue.cc b/src/yb/rpc/rpc_with_queue.cc index 55aeaf522df..d656c7f43db 100644 --- a/src/yb/rpc/rpc_with_queue.cc +++ b/src/yb/rpc/rpc_with_queue.cc @@ -67,7 +67,7 @@ void ConnectionContextWithQueue::Enqueue(std::shared_ptr c first_without_reply_.store(call.get(), std::memory_order_release); } if (size <= max_concurrent_calls_) { - reactor->messenger()->Handle(call, Queue::kTrue); + reactor->messenger().Handle(call, Queue::kTrue); } } @@ -99,7 +99,7 @@ void ConnectionContextWithQueue::CallProcessed(InboundCall* call) { --replies_being_sent_; if (calls_queue_.size() >= max_concurrent_calls_) { auto call_ptr = calls_queue_[max_concurrent_calls_ - 1]; - reactor->messenger()->Handle(call_ptr, Queue::kTrue); + reactor->messenger().Handle(call_ptr, Queue::kTrue); } if (Idle() && idle_listener_) { idle_listener_(); @@ -110,19 +110,17 @@ void ConnectionContextWithQueue::CallProcessed(InboundCall* call) { } } -void ConnectionContextWithQueue::QueueResponse(const ConnectionPtr& conn, - InboundCallPtr call) { +Status ConnectionContextWithQueue::QueueResponse(const ConnectionPtr& conn, InboundCallPtr call) { QueueableInboundCall* queueable_call = down_cast(call.get()); queueable_call->SetHasReply(); if (queueable_call == first_without_reply_.load(std::memory_order_acquire)) { - auto scheduled = conn->reactor()->ScheduleReactorTask(flush_outbound_queue_task_); - LOG_IF(WARNING, !scheduled) << "Failed to schedule flush outbound queue"; + return conn->reactor()->ScheduleReactorTask(flush_outbound_queue_task_); } + return Status::OK(); } void ConnectionContextWithQueue::FlushOutboundQueue(Connection* conn) { - DCHECK(conn->reactor()->IsCurrentThread()); - + conn->reactor()->CheckCurrentThread(); const size_t begin = replies_being_sent_; size_t end = begin; for (;;) { @@ -151,10 +149,11 @@ void ConnectionContextWithQueue::FlushOutboundQueue(Connection* conn) { } } -void ConnectionContextWithQueue::AssignConnection(const ConnectionPtr& conn) { +Status ConnectionContextWithQueue::AssignConnection(const ConnectionPtr& conn) { flush_outbound_queue_task_ = MakeFunctorReactorTask( std::bind(&ConnectionContextWithQueue::FlushOutboundQueue, this, conn.get()), conn, SOURCE_LOCATION()); + return Status::OK(); } } // namespace rpc diff --git a/src/yb/rpc/rpc_with_queue.h b/src/yb/rpc/rpc_with_queue.h index 72a1559e0c6..c6039da6576 100644 --- a/src/yb/rpc/rpc_with_queue.h +++ b/src/yb/rpc/rpc_with_queue.h @@ -25,6 +25,7 @@ #include "yb/rpc/connection_context.h" #include "yb/rpc/inbound_call.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/size_literals.h" @@ -77,23 +78,24 @@ class ConnectionContextWithQueue : public ConnectionContextBase, return queued_bytes_ <= max_queued_bytes_; } - void Enqueue(std::shared_ptr call); + void Enqueue(std::shared_ptr call) ON_REACTOR_THREAD; uint64_t ProcessedCallCount() override { return processed_call_count_.load(std::memory_order_acquire); } - void Shutdown(const Status& status) override; + void Shutdown(const Status& status) ON_REACTOR_THREAD override; private: - void AssignConnection(const ConnectionPtr& conn) override; - void DumpPB(const DumpRunningRpcsRequestPB& req, RpcConnectionPB* resp) override; - bool Idle(std::string* reason_not_idle = nullptr) override; - void QueueResponse(const ConnectionPtr& conn, InboundCallPtr call) override; + Status AssignConnection(const ConnectionPtr& conn) override; + void DumpPB(const DumpRunningRpcsRequestPB& req, RpcConnectionPB* resp) + ON_REACTOR_THREAD override; + bool Idle(std::string* reason_not_idle = nullptr) ON_REACTOR_THREAD override; + Status QueueResponse(const ConnectionPtr& conn, InboundCallPtr call) override; void ListenIdle(IdleListener listener) override { idle_listener_ = std::move(listener); } - void CallProcessed(InboundCall* call) override; - void FlushOutboundQueue(Connection* conn); + void CallProcessed(InboundCall* call) ON_REACTOR_THREAD override; + void FlushOutboundQueue(Connection* conn) ON_REACTOR_THREAD; void FlushOutboundQueueAborted(const Status& status); const size_t max_concurrent_calls_; @@ -107,7 +109,8 @@ class ConnectionContextWithQueue : public ConnectionContextBase, // first_without_reply_ points to the first of them. // There are not more than max_concurrent_calls_ entries in first two groups. // After end of queue there are calls that we received but processing did not start for them. - std::deque> calls_queue_; + std::deque> calls_queue_ + GUARDED_BY_REACTOR_THREAD; std::shared_ptr flush_outbound_queue_task_; // First call that does not have reply yet. diff --git a/src/yb/rpc/secure_stream.cc b/src/yb/rpc/secure_stream.cc index cf880c0677f..3e4781cbdfc 100644 --- a/src/yb/rpc/secure_stream.cc +++ b/src/yb/rpc/secure_stream.cc @@ -25,6 +25,7 @@ #include "yb/rpc/outbound_data.h" #include "yb/rpc/refined_stream.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/enums.h" #include "yb/util/errno.h" @@ -297,7 +298,7 @@ class SecureContext::Impl { // Generates and uses temporary keys, should be used only during testing. Status TEST_GenerateKeys(int bits, const std::string& common_name, - MatchingCertKeyPair matching_cert_key_pair) EXCLUDES(mutex_); + MatchingCertKeyPair matching_cert_key_pair) EXCLUDES(mutex_); Result Create(rpc::UseCertificateKeyPair use_certificate_key_pair) const EXCLUDES(mutex_); @@ -516,11 +517,11 @@ class SecureRefiner : public StreamRefiner { stream_ = stream; } - Status Handshake() override; + Status Handshake() ON_REACTOR_THREAD override; Status Init(); - Status Send(OutboundDataPtr data) override; - Status ProcessHeader() override; + Status Send(OutboundDataPtr data) ON_REACTOR_THREAD override; + Status ProcessHeader() ON_REACTOR_THREAD override; Result Read(StreamReadBuffer* out) override; std::string ToString() const override { @@ -536,10 +537,10 @@ class SecureRefiner : public StreamRefiner { bool MatchEndpoint(X509* cert, GENERAL_NAMES* gens); bool MatchUid(X509* cert, GENERAL_NAMES* gens); bool MatchUidEntry(const Slice& value, const char* name); - Result WriteEncrypted(OutboundDataPtr data); + Result WriteEncrypted(OutboundDataPtr data) ON_REACTOR_THREAD; void DecryptReceived(); - Status Established(RefinedStreamState state) { + Status Established(RefinedStreamState state) ON_REACTOR_THREAD { VLOG_WITH_PREFIX(4) << "Established with state: " << state << ", used cipher: " << SSL_get_cipher_name(ssl_.get()); diff --git a/src/yb/rpc/stream.h b/src/yb/rpc/stream.h index 12d9833b475..a1ba218f1a9 100644 --- a/src/yb/rpc/stream.h +++ b/src/yb/rpc/stream.h @@ -15,6 +15,7 @@ #define YB_RPC_STREAM_H #include "yb/rpc/rpc_fwd.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/status_fwd.h" #include "yb/util/net/socket.h" @@ -32,6 +33,11 @@ class MetricEntity; namespace rpc { +using CallHandle = size_t; + +// A value we use instead of a call handle in case there is a failure to queue a call. +constexpr CallHandle kUnknownCallHandle = std::numeric_limits::max(); + class StreamReadBuffer { public: // Returns whether we could read appended data from this buffer. It is NOT always !Empty(). @@ -99,9 +105,9 @@ class Stream { virtual void Shutdown(const Status& status) = 0; // Returns handle to block associated with this data. This handle could be used to cancel - // transfer of this block using Cancelled. - // For instance when unsent call times out. - virtual Result Send(OutboundDataPtr data) = 0; + // transfer of this block using Cancelled, e.g. when a unsent call times out. + // May return kUnknownCallHandle. + virtual Result Send(OutboundDataPtr data) ON_REACTOR_THREAD = 0; virtual Status TryWrite() = 0; virtual void ParseReceived() = 0; diff --git a/src/yb/rpc/yb_rpc.cc b/src/yb/rpc/yb_rpc.cc index cbac75d0c46..6eb4d6b05af 100644 --- a/src/yb/rpc/yb_rpc.cc +++ b/src/yb/rpc/yb_rpc.cc @@ -193,7 +193,7 @@ Status YBInboundConnectionContext::HandleCall( return Status::OK(); } - reactor->messenger()->Handle(call, Queue::kTrue); + reactor->messenger().Handle(call, Queue::kTrue); return Status::OK(); } @@ -237,11 +237,19 @@ void YBInboundConnectionContext::HandleTimeout(ev::timer& watcher, int revents) // for sending is still in queue due to RPC/networking issues, so no need to queue // another one. VLOG(4) << connection->ToString() << ": " << "Sending heartbeat, now: " << AsString(now) - << ", deadline: " << AsString(deadline) + << ", deadline: " << ToStringRelativeToNow(deadline, now) << ", last_write_time_: " << AsString(last_write_time_) << ", last_heartbeat_sending_time_: " << AsString(last_heartbeat_sending_time_); - connection->QueueOutboundData(HeartbeatOutboundData::Instance()); - last_heartbeat_sending_time_ = now; + auto queuing_status = connection->QueueOutboundData(HeartbeatOutboundData::Instance()); + if (queuing_status.ok()) { + last_heartbeat_sending_time_ = now; + } else { + // Do not DFATAL here. This happens during shutdown and should not result in frequent + // log messages. + LOG(WARNING) << "Could not queue an inbound connection heartbeat message: " + << queuing_status; + // We will try again at the next timer event. + } } timer_.Start(HeartbeatPeriod()); } else { @@ -527,8 +535,8 @@ void YBOutboundConnectionContext::Connected(const ConnectionPtr& connection) { } } -void YBOutboundConnectionContext::AssignConnection(const ConnectionPtr& connection) { - connection->QueueOutboundData(ConnectionHeaderInstance()); +Status YBOutboundConnectionContext::AssignConnection(const ConnectionPtr& connection) { + return connection->QueueOutboundData(ConnectionHeaderInstance()); } Result YBOutboundConnectionContext::ProcessCalls( diff --git a/src/yb/rpc/yb_rpc.h b/src/yb/rpc/yb_rpc.h index 8b57e029679..52355bb6fdf 100644 --- a/src/yb/rpc/yb_rpc.h +++ b/src/yb/rpc/yb_rpc.h @@ -31,6 +31,7 @@ #include "yb/rpc/connection_context.h" #include "yb/rpc/rpc_with_call_id.h" #include "yb/rpc/serialization.h" +#include "yb/rpc/reactor_thread_role.h" #include "yb/util/ev_util.h" #include "yb/util/net/net_fwd.h" @@ -85,16 +86,18 @@ class YBInboundConnectionContext : public YBConnectionContext { static std::string Name() { return "Inbound RPC"; } private: // Takes ownership of call_data content. - Status HandleCall(const ConnectionPtr& connection, CallData* call_data) override; + Status HandleCall(const ConnectionPtr& connection, CallData* call_data) + ON_REACTOR_THREAD override; void Connected(const ConnectionPtr& connection) override; - Result ProcessCalls(const ConnectionPtr& connection, - const IoVecs& data, - ReadBufferFull read_buffer_full) override; + Result ProcessCalls( + const ConnectionPtr& connection, + const IoVecs& data, + ReadBufferFull read_buffer_full) ON_REACTOR_THREAD override; // Takes ownership of call_data content. Status HandleInboundCall(const ConnectionPtr& connection, std::vector* call_data); - void HandleTimeout(ev::timer& watcher, int revents); // NOLINT + void HandleTimeout(ev::timer& watcher, int revents) ON_REACTOR_THREAD; // NOLINT RpcConnectionPB::StateType State() override { return state_; } @@ -245,16 +248,18 @@ class YBOutboundConnectionContext : public YBConnectionContext { } // Takes ownership of call_data content. - Status HandleCall(const ConnectionPtr& connection, CallData* call_data) override; + Status HandleCall(const ConnectionPtr& connection, CallData* call_data) + ON_REACTOR_THREAD override; void Connected(const ConnectionPtr& connection) override; - void AssignConnection(const ConnectionPtr& connection) override; - Result ProcessCalls(const ConnectionPtr& connection, - const IoVecs& data, - ReadBufferFull read_buffer_full) override; + Status AssignConnection(const ConnectionPtr& connection) override; + Result ProcessCalls( + const ConnectionPtr& connection, + const IoVecs& data, + ReadBufferFull read_buffer_full) ON_REACTOR_THREAD override; void UpdateLastRead(const ConnectionPtr& connection) override; - void HandleTimeout(ev::timer& watcher, int revents); // NOLINT + void HandleTimeout(ev::timer& watcher, int revents) ON_REACTOR_THREAD; // NOLINT std::weak_ptr connection_; diff --git a/src/yb/util/lockfree-test.cc b/src/yb/util/lockfree-test.cc index bce36f11e2a..3da2f0e3c9b 100644 --- a/src/yb/util/lockfree-test.cc +++ b/src/yb/util/lockfree-test.cc @@ -487,4 +487,39 @@ TEST(LockfreeTest, Stack) { } } +TEST(LockfreeTest, WriteOnceWeakPtr) { + std::shared_ptr hello = std::make_shared("Hello"); + std::shared_ptr world = std::make_shared("world"); + + { + WriteOnceWeakPtr wowp(hello); + ASSERT_TRUE(wowp.IsInitialized()); + ASSERT_FALSE(wowp.Set(world)); + auto* hello_ptr = hello.get(); + ASSERT_EQ(wowp.raw_ptr_for_logging(), hello_ptr); + hello.reset(); + ASSERT_EQ(wowp.lock(), nullptr); + // Still initialized, even though the object has been destroyed. + ASSERT_TRUE(wowp.IsInitialized()); + // The weak pointer still stores the same raw pointer. + ASSERT_EQ(wowp.raw_ptr_for_logging(), hello_ptr); + } + + { + WriteOnceWeakPtr wowp; + ASSERT_FALSE(wowp.IsInitialized()); + ASSERT_FALSE(wowp.Set(nullptr)); + ASSERT_TRUE(!wowp.IsInitialized()); // Setting to nullptr was a no-op. + ASSERT_TRUE(wowp.Set(world)); + ASSERT_TRUE(wowp.IsInitialized()); + // Setting the pointer the second time, even to the same value, will fail. + ASSERT_FALSE(wowp.Set(world)); + auto* world_ptr = world.get(); + ASSERT_EQ(wowp.raw_ptr_for_logging(), world_ptr); + world.reset(); + ASSERT_TRUE(wowp.IsInitialized()); + ASSERT_EQ(wowp.raw_ptr_for_logging(), world_ptr); + } +} + } // namespace yb diff --git a/src/yb/util/lockfree.h b/src/yb/util/lockfree.h index 969067bb47d..582ccfc8e05 100644 --- a/src/yb/util/lockfree.h +++ b/src/yb/util/lockfree.h @@ -51,6 +51,12 @@ class MPSCQueue { return result; } + void Drain() { + while (auto* entry = Pop()) { + delete entry; + } + } + private: void PreparePop() { T* current = push_head_.exchange(nullptr, std::memory_order_acq_rel); @@ -144,6 +150,66 @@ class LockFreeStack { boost::atomic head_{Head{nullptr, 0}}; }; +// A weak pointer that can only be written to once, but can be read and written in a lock-free way. +template +class WriteOnceWeakPtr { + public: + WriteOnceWeakPtr() {} + + explicit WriteOnceWeakPtr(const std::shared_ptr& p) + : state_(p ? State::kSet : State::kUnset), + weak_ptr_(p) { + } + + // Set the pointer to the given value. Return true if successful. Setting the value to a null + // pointer is never considered successful. + MUST_USE_RESULT bool Set(const std::shared_ptr& p) { + if (!p) + return false; + auto expected_state = State::kUnset; + if (!state_.compare_exchange_strong( + expected_state, State::kSetting, std::memory_order_acq_rel)) { + return false; + } + // Only one thread will ever get here. + weak_ptr_ = p; + // Use sequential consistency here to prevent unexpected reorderings of future operations before + // this one. + state_ = State::kSet; + return true; + } + + std::shared_ptr lock() const { + return IsInitialized() ? weak_ptr_.lock() : nullptr; + } + + // This always returns a const T*, because the object is not guaranteed to exist, and the return + // value of this function should only be used for logging/debugging. + const T* raw_ptr_for_logging() const { + // This uses the fact that the weak pointer stores the raw pointer as its first word, and avoids + // storing the raw pointer separately. This is true for libc++ and libstdc++. + return IsInitialized() ? *reinterpret_cast(&weak_ptr_) : nullptr; + } + + bool IsInitialized() const { + return state_.load(std::memory_order_acquire) == State::kSet; + } + + private: + enum class State : uint8_t { + kUnset, + kSetting, + kSet + }; + + std::atomic state_{State::kUnset}; + std::weak_ptr weak_ptr_; + + static_assert(sizeof(weak_ptr_) == 2 * sizeof(void*)); + + DISALLOW_COPY_AND_ASSIGN(WriteOnceWeakPtr); +}; + } // namespace yb #endif // YB_UTIL_LOCKFREE_H diff --git a/src/yb/util/monotime-test.cc b/src/yb/util/monotime-test.cc index 92e4af1ef43..f24070ca674 100644 --- a/src/yb/util/monotime-test.cc +++ b/src/yb/util/monotime-test.cc @@ -37,6 +37,7 @@ #include #include "yb/util/test_util.h" +#include "yb/util/format.h" using namespace std::literals; @@ -83,6 +84,14 @@ TEST(TestMonoTime, TestComparison) { ASSERT_TRUE(mil.LessThan(sec)); ASSERT_TRUE(mil.MoreThan(nano)); ASSERT_TRUE(sec.MoreThan(mil)); + + ASSERT_TRUE(IsInitialized(CoarseMonoClock::Now())); + ASSERT_FALSE(IsInitialized(CoarseTimePoint::min())); + ASSERT_TRUE(IsInitialized(CoarseTimePoint::max())); + + ASSERT_FALSE(IsExtremeValue(CoarseMonoClock::Now())); + ASSERT_TRUE(IsExtremeValue(CoarseTimePoint::min())); + ASSERT_TRUE(IsExtremeValue(CoarseTimePoint::max())); } TEST(TestMonoTime, TestTimeVal) { @@ -269,4 +278,26 @@ TEST(TestMonoTime, TestSubtractDelta) { ASSERT_EQ(start_copy + delta, start); } +TEST(TestMonoTime, ToStringRelativeToNow) { + auto now = CoarseMonoClock::Now(); + + auto t = now + 2s; + ASSERT_EQ(Format("$0 (2.000s from now)", t), ToStringRelativeToNow(t, now)); + ASSERT_EQ("2.000s from now", ToStringRelativeToNowOnly(t, now)); + + t = now; + ASSERT_EQ(Format("$0 (now)", t), ToStringRelativeToNow(t, now)); + ASSERT_EQ("now", ToStringRelativeToNowOnly(t, now)); + + t = now - 2s; + ASSERT_EQ(Format("$0 (2.000s ago)", t), ToStringRelativeToNow(t, now)); + ASSERT_EQ("2.000s ago", ToStringRelativeToNowOnly(t, now)); + + ASSERT_EQ("-inf", ToStringRelativeToNow(CoarseTimePoint::min(), now)); + ASSERT_EQ("+inf", ToStringRelativeToNow(CoarseTimePoint::max(), now)); + + ASSERT_EQ(ToString(t), ToStringRelativeToNow(t, CoarseTimePoint::min())); + ASSERT_EQ(ToString(t), ToStringRelativeToNow(t, CoarseTimePoint::max())); +} + } // namespace yb diff --git a/src/yb/util/monotime.cc b/src/yb/util/monotime.cc index f34a54396b4..a9198031ce6 100644 --- a/src/yb/util/monotime.cc +++ b/src/yb/util/monotime.cc @@ -408,6 +408,12 @@ CoarseMonoClock::Duration ClockResolution() { } std::string ToString(CoarseMonoClock::TimePoint time_point) { + if (time_point == CoarseTimePoint::min()) { + return "-inf"; + } + if (time_point == CoarseTimePoint::max()) { + return "+inf"; + } return MonoDelta(time_point.time_since_epoch()).ToString(); } @@ -423,4 +429,31 @@ bool IsInitialized(CoarseTimePoint time_point) { return MonoDelta(time_point.time_since_epoch()).Initialized(); } +bool IsExtremeValue(CoarseTimePoint time_point) { + return time_point == CoarseTimePoint::min() || time_point == CoarseTimePoint::max(); +} + +std::string ToStringRelativeToNow(CoarseTimePoint t, CoarseTimePoint now) { + if (IsExtremeValue(t) || IsExtremeValue(now)) { + return ToString(t); + } + return Format("$0 ($1)", t, ToStringRelativeToNowOnly(t, now)); +} + +std::string ToStringRelativeToNow(CoarseTimePoint t, std::optional now) { + if (now) + return ToStringRelativeToNow(t, *now); + return ToString(t); +} + +std::string ToStringRelativeToNowOnly(CoarseTimePoint t, CoarseTimePoint now) { + if (t < now) { + return Format("$0 ago", now - t); + } + if (t > now) { + return Format("$0 from now", t - now); + } + return "now"; +} + } // namespace yb diff --git a/src/yb/util/monotime.h b/src/yb/util/monotime.h index 8197acd5656..36755561d3e 100644 --- a/src/yb/util/monotime.h +++ b/src/yb/util/monotime.h @@ -34,6 +34,7 @@ #include #include +#include #include #include @@ -341,8 +342,28 @@ std::string ToString(CoarseMonoClock::TimePoint value); CoarseTimePoint ToCoarse(MonoTime monotime); std::chrono::steady_clock::time_point ToSteady(CoarseTimePoint time_point); +// Returns false if the given time point is the minimum possible value of CoarseTimePoint. The +// implementation is consistent with MonoDelta's notion of being initialized, looking at the time +// since epoch. Note that CoarseTimePoint::min() is not the default value of a CoarseTimePoint. +// Its default value is a time point represented by zero, which may be an arbitrary point in time, +// since CLOCK_MONOTONIC represents monotonic time since some unspecified starting point. bool IsInitialized(CoarseTimePoint time_point); +// Returns true if the given time point is either the minimum or maximum possible value. +bool IsExtremeValue(CoarseTimePoint time_point); + +// Formats the given time point in the form "