client channel: don't hold mutexes while calling the ConfigSelector or the LB picker (#31973)

* WIP

* use the same lock for queue and picker -- still need to figure out how to drain queue safely

* working!

* always unref pickers in WorkSerializer

* simplify tracking of queued calls

* fix sanity

* simplify resolver queueing code

* fix tsan failure

* simplify queued LB pick reprocessing

* minor cleanups

* remove unnecessary wrap-around checks

* clang-format

* generate_projects

* fix pollset bug

* use absl::flat_hash_set<> instead of std::set<>

* fix use-after-free in retry code

* add missing BUILD dep
pull/32223/head
Mark D. Roth 2 years ago committed by GitHub
parent 1d8fac36b3
commit e699e0135e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 1
      BUILD
  2. 5
      src/core/BUILD
  3. 757
      src/core/ext/filters/client_channel/client_channel.cc
  4. 85
      src/core/ext/filters/client_channel/client_channel.h
  5. 17
      src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc
  6. 55
      src/core/ext/filters/client_channel/lb_policy/ring_hash/ring_hash.cc
  7. 14
      src/core/ext/filters/client_channel/lb_policy/rls/rls.cc
  8. 24
      src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc
  9. 14
      src/core/ext/filters/client_channel/lb_policy/weighted_target/weighted_target.cc
  10. 3
      src/core/ext/filters/client_channel/retry_filter.cc
  11. 7
      src/core/ext/xds/xds_endpoint.cc
  12. 10
      src/core/ext/xds/xds_endpoint.h
  13. 22
      src/core/lib/load_balancing/lb_policy.cc
  14. 6
      src/core/lib/load_balancing/lb_policy.h

@ -2768,6 +2768,7 @@ grpc_cc_library(
],
external_deps = [
"absl/base:core_headers",
"absl/container:flat_hash_set",
"absl/container:inlined_vector",
"absl/status",
"absl/status:statusor",

@ -2470,6 +2470,7 @@ grpc_cc_library(
srcs = ["lib/load_balancing/lb_policy.cc"],
hdrs = ["lib/load_balancing/lb_policy.h"],
external_deps = [
"absl/base:core_headers",
"absl/status",
"absl/status:statusor",
"absl/strings",
@ -2489,6 +2490,7 @@ grpc_cc_library(
"//:debug_location",
"//:event_engine_base_hdrs",
"//:exec_ctx",
"//:gpr",
"//:gpr_platform",
"//:grpc_trace",
"//:orphanable",
@ -3812,6 +3814,7 @@ grpc_cc_library(
"absl/base:core_headers",
"absl/functional:bind_front",
"absl/memory",
"absl/random",
"absl/status",
"absl/status:statusor",
"absl/strings",
@ -4409,6 +4412,7 @@ grpc_cc_library(
"ext/filters/client_channel/lb_policy/round_robin/round_robin.cc",
],
external_deps = [
"absl/random",
"absl/status",
"absl/status:statusor",
"absl/strings",
@ -4595,6 +4599,7 @@ grpc_cc_library(
"ext/filters/client_channel/lb_policy/weighted_target/weighted_target.cc",
],
external_deps = [
"absl/base:core_headers",
"absl/random",
"absl/status",
"absl/status:statusor",

File diff suppressed because it is too large Load Diff

@ -24,11 +24,11 @@
#include <atomic>
#include <map>
#include <memory>
#include <set>
#include <string>
#include <utility>
#include "absl/base/thread_annotations.h"
#include "absl/container/flat_hash_set.h"
#include "absl/status/status.h"
#include "absl/strings/string_view.h"
#include "absl/types/optional.h"
@ -222,15 +222,6 @@ class ClientChannel {
std::atomic<bool> done_{false};
};
struct ResolverQueuedCall {
grpc_call_element* elem;
ResolverQueuedCall* next = nullptr;
};
struct LbQueuedCall {
LoadBalancedCall* lb_call;
LbQueuedCall* next = nullptr;
};
ClientChannel(grpc_channel_element_args* args, grpc_error_handle* error);
~ClientChannel();
@ -246,6 +237,9 @@ class ClientChannel {
// Note: All methods with "Locked" suffix must be invoked from within
// work_serializer_.
void ReprocessQueuedResolverCalls()
ABSL_EXCLUSIVE_LOCKS_REQUIRED(&resolution_mu_);
void OnResolverResultChangedLocked(Resolver::Result result)
ABSL_EXCLUSIVE_LOCKS_REQUIRED(*work_serializer_);
void OnResolverErrorLocked(absl::Status status)
@ -284,20 +278,6 @@ class ClientChannel {
void TryToConnectLocked() ABSL_EXCLUSIVE_LOCKS_REQUIRED(*work_serializer_);
// These methods all require holding resolution_mu_.
void AddResolverQueuedCall(ResolverQueuedCall* call,
grpc_polling_entity* pollent)
ABSL_EXCLUSIVE_LOCKS_REQUIRED(resolution_mu_);
void RemoveResolverQueuedCall(ResolverQueuedCall* to_remove,
grpc_polling_entity* pollent)
ABSL_EXCLUSIVE_LOCKS_REQUIRED(resolution_mu_);
// These methods all require holding data_plane_mu_.
void AddLbQueuedCall(LbQueuedCall* call, grpc_polling_entity* pollent)
ABSL_EXCLUSIVE_LOCKS_REQUIRED(data_plane_mu_);
void RemoveLbQueuedCall(LbQueuedCall* to_remove, grpc_polling_entity* pollent)
ABSL_EXCLUSIVE_LOCKS_REQUIRED(data_plane_mu_);
//
// Fields set at construction and never modified.
//
@ -316,9 +296,9 @@ class ClientChannel {
// Fields related to name resolution. Guarded by resolution_mu_.
//
mutable Mutex resolution_mu_;
// Linked list of calls queued waiting for resolver result.
ResolverQueuedCall* resolver_queued_calls_ ABSL_GUARDED_BY(resolution_mu_) =
nullptr;
// List of calls queued waiting for resolver result.
absl::flat_hash_set<grpc_call_element*> resolver_queued_calls_
ABSL_GUARDED_BY(resolution_mu_);
// Data from service config.
absl::Status resolver_transient_failure_error_
ABSL_GUARDED_BY(resolution_mu_);
@ -330,13 +310,13 @@ class ClientChannel {
ABSL_GUARDED_BY(resolution_mu_);
//
// Fields used in the data plane. Guarded by data_plane_mu_.
// Fields related to LB picks. Guarded by lb_mu_.
//
mutable Mutex data_plane_mu_;
mutable Mutex lb_mu_;
RefCountedPtr<LoadBalancingPolicy::SubchannelPicker> picker_
ABSL_GUARDED_BY(data_plane_mu_);
// Linked list of calls queued waiting for LB pick.
LbQueuedCall* lb_queued_calls_ ABSL_GUARDED_BY(data_plane_mu_) = nullptr;
ABSL_GUARDED_BY(lb_mu_);
absl::flat_hash_set<LoadBalancedCall*> lb_queued_calls_
ABSL_GUARDED_BY(lb_mu_);
//
// Fields used in the control plane. Guarded by work_serializer.
@ -360,7 +340,7 @@ class ClientChannel {
// The set of SubchannelWrappers that currently exist.
// No need to hold a ref, since the map is updated in the control-plane
// work_serializer when the SubchannelWrappers are created and destroyed.
std::set<SubchannelWrapper*> subchannel_wrappers_
absl::flat_hash_set<SubchannelWrapper*> subchannel_wrappers_
ABSL_GUARDED_BY(*work_serializer_);
int keepalive_time_ ABSL_GUARDED_BY(*work_serializer_) = -1;
grpc_error_handle disconnect_error_ ABSL_GUARDED_BY(*work_serializer_);
@ -422,16 +402,11 @@ class ClientChannel::LoadBalancedCall
void StartTransportStreamOpBatch(grpc_transport_stream_op_batch* batch);
// Invoked by channel for queued LB picks when the picker is updated.
static void PickSubchannel(void* arg, grpc_error_handle error);
// Helper function for performing an LB pick while holding the data plane
// mutex. Returns true if the pick is complete, in which case the caller
// must invoke PickDone() or AsyncPickDone() with the returned error.
bool PickSubchannelLocked(grpc_error_handle* error)
ABSL_EXCLUSIVE_LOCKS_REQUIRED(&ClientChannel::data_plane_mu_);
// Schedules a callback to process the completed pick. The callback
// will not run until after this method returns.
void AsyncPickDone(grpc_error_handle error);
void PickSubchannel(bool was_queued);
// Called by channel when removing a call from the list of queued calls.
void RemoveCallFromLbQueuedCallsLocked()
ABSL_EXCLUSIVE_LOCKS_REQUIRED(&ClientChannel::lb_mu_);
RefCountedPtr<SubchannelCall> subchannel_call() const {
return subchannel_call_;
@ -479,14 +454,14 @@ class ClientChannel::LoadBalancedCall
void RecordCallCompletion(absl::Status status);
void CreateSubchannelCall();
// Invoked when a pick is completed, on both success or failure.
static void PickDone(void* arg, grpc_error_handle error);
// Removes the call from the channel's list of queued picks if present.
void MaybeRemoveCallFromLbQueuedCallsLocked()
ABSL_EXCLUSIVE_LOCKS_REQUIRED(&ClientChannel::data_plane_mu_);
// Helper function for performing an LB pick with a specified picker.
// Returns true if the pick is complete.
bool PickSubchannelImpl(LoadBalancingPolicy::SubchannelPicker* picker,
grpc_error_handle* error);
// Adds the call to the channel's list of queued picks if not already present.
void MaybeAddCallToLbQueuedCallsLocked()
ABSL_EXCLUSIVE_LOCKS_REQUIRED(&ClientChannel::data_plane_mu_);
void AddCallToLbQueuedCallsLocked()
ABSL_EXCLUSIVE_LOCKS_REQUIRED(&ClientChannel::lb_mu_);
ClientChannel* chand_;
@ -513,15 +488,9 @@ class ClientChannel::LoadBalancedCall
// Set when we fail inside the LB call.
grpc_error_handle failure_error_;
grpc_closure pick_closure_;
// Accessed while holding ClientChannel::data_plane_mu_.
ClientChannel::LbQueuedCall queued_call_
ABSL_GUARDED_BY(&ClientChannel::data_plane_mu_);
bool queued_pending_lb_pick_ ABSL_GUARDED_BY(&ClientChannel::data_plane_mu_) =
false;
// Accessed while holding ClientChannel::lb_mu_.
LbQueuedCallCanceller* lb_call_canceller_
ABSL_GUARDED_BY(&ClientChannel::data_plane_mu_) = nullptr;
ABSL_GUARDED_BY(&ClientChannel::lb_mu_) = nullptr;
RefCountedPtr<ConnectedSubchannel> connected_subchannel_;
const BackendMetricData* backend_metric_data_ = nullptr;

@ -61,6 +61,7 @@
#include <string.h>
#include <algorithm>
#include <atomic>
#include <initializer_list>
#include <map>
#include <memory>
@ -389,19 +390,15 @@ class GrpcLb : public LoadBalancingPolicy {
// Returns the LB token to use for a drop, or null if the call
// should not be dropped.
//
// Note: This is called from the picker, so it will be invoked in
// the channel's data plane mutex, NOT the control plane
// work_serializer. It should not be accessed by any other part of the LB
// policy.
// Note: This is called from the picker, NOT from inside the control
// plane work_serializer.
const char* ShouldDrop();
private:
std::vector<GrpcLbServer> serverlist_;
// Guarded by the channel's data plane mutex, NOT the control
// plane work_serializer. It should not be accessed by anything but the
// picker via the ShouldDrop() method.
size_t drop_index_ = 0;
// Accessed from the picker, so needs synchronization.
std::atomic<size_t> drop_index_{0};
};
class Picker : public SubchannelPicker {
@ -717,8 +714,8 @@ bool GrpcLb::Serverlist::ContainsAllDropEntries() const {
const char* GrpcLb::Serverlist::ShouldDrop() {
if (serverlist_.empty()) return nullptr;
GrpcLbServer& server = serverlist_[drop_index_];
drop_index_ = (drop_index_ + 1) % serverlist_.size();
size_t index = drop_index_.fetch_add(1, std::memory_order_relaxed);
GrpcLbServer& server = serverlist_[index % serverlist_.size()];
return server.drop ? server.load_balance_token : nullptr;
}

@ -254,19 +254,14 @@ class RingHash : public LoadBalancingPolicy {
explicit Picker(RefCountedPtr<RingHashSubchannelList> subchannel_list)
: subchannel_list_(std::move(subchannel_list)) {}
~Picker() override {
// Hop into WorkSerializer to unref the subchannel list, since that may
// trigger the unreffing of the underlying subchannels.
MakeOrphanable<WorkSerializerRunner>(std::move(subchannel_list_));
}
PickResult Pick(PickArgs args) override;
private:
// An interface for running a callback in the control plane WorkSerializer.
class WorkSerializerRunner : public Orphanable {
// A fire-and-forget class that schedules subchannel connection attempts
// on the control plane WorkSerializer.
class SubchannelConnectionAttempter : public Orphanable {
public:
explicit WorkSerializerRunner(
explicit SubchannelConnectionAttempter(
RefCountedPtr<RingHashSubchannelList> subchannel_list)
: subchannel_list_(std::move(subchannel_list)) {
GRPC_CLOSURE_INIT(&closure_, RunInExecCtx, this, nullptr);
@ -278,17 +273,26 @@ class RingHash : public LoadBalancingPolicy {
ExecCtx::Run(DEBUG_LOCATION, &closure_, absl::OkStatus());
}
void AddSubchannel(RefCountedPtr<SubchannelInterface> subchannel) {
subchannels_.push_back(std::move(subchannel));
}
// Will be invoked inside of the WorkSerializer.
virtual void Run() {}
void Run() {
if (!ring_hash_lb()->shutdown_) {
for (auto& subchannel : subchannels_) {
subchannel->RequestConnection();
}
}
}
protected:
private:
RingHash* ring_hash_lb() const {
return static_cast<RingHash*>(subchannel_list_->policy());
}
private:
static void RunInExecCtx(void* arg, grpc_error_handle /*error*/) {
auto* self = static_cast<WorkSerializerRunner*>(arg);
auto* self = static_cast<SubchannelConnectionAttempter*>(arg);
self->ring_hash_lb()->work_serializer()->Run(
[self]() {
self->Run();
@ -298,31 +302,8 @@ class RingHash : public LoadBalancingPolicy {
}
RefCountedPtr<RingHashSubchannelList> subchannel_list_;
grpc_closure closure_;
};
// A fire-and-forget class that schedules subchannel connection attempts
// on the control plane WorkSerializer.
class SubchannelConnectionAttempter : public WorkSerializerRunner {
public:
explicit SubchannelConnectionAttempter(
RefCountedPtr<RingHashSubchannelList> subchannel_list)
: WorkSerializerRunner(std::move(subchannel_list)) {}
void AddSubchannel(RefCountedPtr<SubchannelInterface> subchannel) {
subchannels_.push_back(std::move(subchannel));
}
void Run() override {
if (!ring_hash_lb()->shutdown_) {
for (auto& subchannel : subchannels_) {
subchannel->RequestConnection();
}
}
}
private:
std::vector<RefCountedPtr<SubchannelInterface>> subchannels_;
grpc_closure closure_;
};
RefCountedPtr<RingHashSubchannelList> subchannel_list_;

@ -365,7 +365,6 @@ class RlsLb : public LoadBalancingPolicy {
class Picker : public LoadBalancingPolicy::SubchannelPicker {
public:
explicit Picker(RefCountedPtr<RlsLb> lb_policy);
~Picker() override;
PickResult Pick(PickArgs args) override;
@ -1008,19 +1007,6 @@ RlsLb::Picker::Picker(RefCountedPtr<RlsLb> lb_policy)
}
}
RlsLb::Picker::~Picker() {
// It's not safe to unref the default child policy in the picker,
// since that needs to be done in the WorkSerializer.
if (default_child_policy_ != nullptr) {
auto* default_child_policy = default_child_policy_.release();
lb_policy_->work_serializer()->Run(
[default_child_policy]() {
default_child_policy->Unref(DEBUG_LOCATION, "Picker");
},
DEBUG_LOCATION);
}
}
LoadBalancingPolicy::PickResult RlsLb::Picker::Pick(PickArgs args) {
// Construct key for request.
RequestKey key = {BuildKeyMap(config_->key_builder_map(), args.path,

@ -18,13 +18,16 @@
#include <inttypes.h>
#include <stdlib.h>
#include <string.h>
#include <algorithm>
#include <atomic>
#include <memory>
#include <string>
#include <utility>
#include <vector>
#include "absl/random/random.h"
#include "absl/status/status.h"
#include "absl/status/statusor.h"
#include "absl/strings/str_cat.h"
@ -174,7 +177,7 @@ class RoundRobin : public LoadBalancingPolicy {
// Using pointer value only, no ref held -- do not dereference!
RoundRobin* parent_;
size_t last_picked_index_;
std::atomic<size_t> last_picked_index_;
std::vector<RefCountedPtr<SubchannelInterface>> subchannels_;
};
@ -189,6 +192,8 @@ class RoundRobin : public LoadBalancingPolicy {
RefCountedPtr<RoundRobinSubchannelList> latest_pending_subchannel_list_;
bool shutdown_ = false;
absl::BitGen bit_gen_;
};
//
@ -207,27 +212,26 @@ RoundRobin::Picker::Picker(RoundRobin* parent,
}
// For discussion on why we generate a random starting index for
// the picker, see https://github.com/grpc/grpc-go/issues/2580.
// TODO(roth): rand(3) is not thread-safe. This should be replaced with
// something better as part of https://github.com/grpc/grpc/issues/17891.
last_picked_index_ = rand() % subchannels_.size();
size_t index =
absl::Uniform<size_t>(parent->bit_gen_, 0, subchannels_.size());
last_picked_index_.store(index, std::memory_order_relaxed);
if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) {
gpr_log(GPR_INFO,
"[RR %p picker %p] created picker from subchannel_list=%p "
"with %" PRIuPTR " READY subchannels; last_picked_index_=%" PRIuPTR,
parent_, this, subchannel_list, subchannels_.size(),
last_picked_index_);
parent_, this, subchannel_list, subchannels_.size(), index);
}
}
RoundRobin::PickResult RoundRobin::Picker::Pick(PickArgs /*args*/) {
last_picked_index_ = (last_picked_index_ + 1) % subchannels_.size();
size_t index = last_picked_index_.fetch_add(1, std::memory_order_relaxed) %
subchannels_.size();
if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) {
gpr_log(GPR_INFO,
"[RR %p picker %p] returning index %" PRIuPTR ", subchannel=%p",
parent_, this, last_picked_index_,
subchannels_[last_picked_index_].get());
parent_, this, index, subchannels_[index].get());
}
return PickResult::Complete(subchannels_[last_picked_index_]);
return PickResult::Complete(subchannels_[index]);
}
//

@ -26,6 +26,7 @@
#include <utility>
#include <vector>
#include "absl/base/thread_annotations.h"
#include "absl/random/random.h"
#include "absl/status/status.h"
#include "absl/status/statusor.h"
@ -46,6 +47,7 @@
#include "src/core/lib/gprpp/debug_location.h"
#include "src/core/lib/gprpp/orphanable.h"
#include "src/core/lib/gprpp/ref_counted_ptr.h"
#include "src/core/lib/gprpp/sync.h"
#include "src/core/lib/gprpp/time.h"
#include "src/core/lib/gprpp/validation_errors.h"
#include "src/core/lib/gprpp/work_serializer.h"
@ -138,7 +140,11 @@ class WeightedTargetLb : public LoadBalancingPolicy {
private:
PickerList pickers_;
absl::BitGen bit_gen_;
// TODO(roth): Consider using a separate thread-local BitGen for each CPU
// to avoid the need for this mutex.
Mutex mu_;
absl::BitGen bit_gen_ ABSL_GUARDED_BY(&mu_);
};
// Each WeightedChild holds a ref to its parent WeightedTargetLb.
@ -247,8 +253,10 @@ class WeightedTargetLb : public LoadBalancingPolicy {
WeightedTargetLb::PickResult WeightedTargetLb::WeightedPicker::Pick(
PickArgs args) {
// Generate a random number in [0, total weight).
const uint64_t key =
absl::Uniform<uint64_t>(bit_gen_, 0, pickers_.back().first);
const uint64_t key = [&]() {
MutexLock lock(&mu_);
return absl::Uniform<uint64_t>(bit_gen_, 0, pickers_.back().first);
}();
// Find the index in pickers_ corresponding to key.
size_t mid = 0;
size_t start_index = 0;

@ -1351,8 +1351,9 @@ RetryFilter::CallData::CallAttempt::BatchData::~BatchData() {
this);
}
CallAttempt* call_attempt = std::exchange(call_attempt_, nullptr);
GRPC_CALL_STACK_UNREF(call_attempt->calld_->owning_call_, "Retry BatchData");
grpc_call_stack* owning_call = call_attempt->calld_->owning_call_;
call_attempt->Unref(DEBUG_LOCATION, "~BatchData");
GRPC_CALL_STACK_UNREF(owning_call, "Retry BatchData");
}
void RetryFilter::CallData::CallAttempt::BatchData::

@ -93,11 +93,14 @@ std::string XdsEndpointResource::Priority::ToString() const {
}
bool XdsEndpointResource::DropConfig::ShouldDrop(
const std::string** category_name) const {
const std::string** category_name) {
for (size_t i = 0; i < drop_category_list_.size(); ++i) {
const auto& drop_category = drop_category_list_[i];
// Generate a random number in [0, 1000000).
const uint32_t random = static_cast<uint32_t>(rand()) % 1000000;
const uint32_t random = [&]() {
MutexLock lock(&mu_);
return absl::Uniform<uint32_t>(bit_gen_, 0, 1000000);
}();
if (random < drop_category.parts_per_million) {
*category_name = &drop_category.name;
return true;

@ -28,6 +28,8 @@
#include <utility>
#include <vector>
#include "absl/base/thread_annotations.h"
#include "absl/random/random.h"
#include "absl/strings/string_view.h"
#include "envoy/config/endpoint/v3/endpoint.upbdefs.h"
#include "upb/def.h"
@ -38,6 +40,7 @@
#include "src/core/ext/xds/xds_resource_type_impl.h"
#include "src/core/lib/gprpp/ref_counted.h"
#include "src/core/lib/gprpp/ref_counted_ptr.h"
#include "src/core/lib/gprpp/sync.h"
#include "src/core/lib/resolver/server_address.h"
namespace grpc_core {
@ -90,7 +93,7 @@ struct XdsEndpointResource : public XdsResourceType::ResourceData {
// The only method invoked from outside the WorkSerializer (used in
// the data plane).
bool ShouldDrop(const std::string** category_name) const;
bool ShouldDrop(const std::string** category_name);
const DropCategoryList& drop_category_list() const {
return drop_category_list_;
@ -108,6 +111,11 @@ struct XdsEndpointResource : public XdsResourceType::ResourceData {
private:
DropCategoryList drop_category_list_;
bool drop_all_ = false;
// TODO(roth): Consider using a separate thread-local BitGen for each CPU
// to avoid the need for this mutex.
Mutex mu_;
absl::BitGen bit_gen_ ABSL_GUARDED_BY(&mu_);
};
PriorityList priorities;

@ -69,19 +69,15 @@ LoadBalancingPolicy::SubchannelPicker::SubchannelPicker()
LoadBalancingPolicy::PickResult LoadBalancingPolicy::QueuePicker::Pick(
PickArgs /*args*/) {
// We invoke the parent's ExitIdleLocked() via a closure instead
// of doing it directly here, for two reasons:
// 1. ExitIdleLocked() may cause the policy's state to change and
// a new picker to be delivered to the channel. If that new
// picker is delivered before ExitIdleLocked() returns, then by
// the time this function returns, the pick will already have
// been processed, and we'll be trying to re-process the same
// pick again, leading to a crash.
// 2. We are currently running in the data plane mutex, but we
// need to bounce into the control plane work_serializer to call
// ExitIdleLocked().
if (!exit_idle_called_ && parent_ != nullptr) {
exit_idle_called_ = true;
auto* parent = parent_->Ref().release(); // ref held by lambda.
// of doing it directly here because ExitIdleLocked() may cause the
// policy's state to change and a new picker to be delivered to the
// channel. If that new picker is delivered before ExitIdleLocked()
// returns, then by the time this function returns, the pick will already
// have been processed, and we'll be trying to re-process the same pick
// again, leading to a crash.
MutexLock lock(&mu_);
if (parent_ != nullptr) {
auto* parent = parent_.release(); // ref held by lambda.
ExecCtx::Run(DEBUG_LOCATION,
GRPC_CLOSURE_CREATE(
[](void* arg, grpc_error_handle /*error*/) {

@ -27,6 +27,7 @@
#include <utility>
#include <vector>
#include "absl/base/thread_annotations.h"
#include "absl/status/status.h"
#include "absl/status/statusor.h"
#include "absl/strings/string_view.h"
@ -44,6 +45,7 @@
#include "src/core/lib/gprpp/orphanable.h"
#include "src/core/lib/gprpp/ref_counted.h"
#include "src/core/lib/gprpp/ref_counted_ptr.h"
#include "src/core/lib/gprpp/sync.h"
#include "src/core/lib/gprpp/work_serializer.h"
#include "src/core/lib/iomgr/iomgr_fwd.h"
#include "src/core/lib/load_balancing/subchannel_interface.h"
@ -392,8 +394,8 @@ class LoadBalancingPolicy : public InternallyRefCounted<LoadBalancingPolicy> {
PickResult Pick(PickArgs args) override;
private:
RefCountedPtr<LoadBalancingPolicy> parent_;
bool exit_idle_called_ = false;
Mutex mu_;
RefCountedPtr<LoadBalancingPolicy> parent_ ABSL_GUARDED_BY(&mu_);
};
// A picker that returns PickResult::Fail for all picks.

Loading…
Cancel
Save