mirror of https://github.com/grpc/grpc.git
Move retry code into its own filter in the DynamicFilter stack (#25820)
* rename ChannelData to ClientChannel * make ClientChannel class definition public * move retry code to its own filter * move LB call factory method to ClientChannel class * move dynamic termination filter out of ClientChannel class * update comments * remove retry parsing from client channel service config parser * fix clang-tidy * fix service_config_test * clang-formatpull/25848/head
parent
efd2ed8ae6
commit
3f19333ced
29 changed files with 3803 additions and 3463 deletions
File diff suppressed because it is too large
Load Diff
@ -1,76 +1,496 @@ |
||||
/*
|
||||
* |
||||
* Copyright 2015 gRPC authors. |
||||
* |
||||
* 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. |
||||
* |
||||
*/ |
||||
//
|
||||
// Copyright 2015 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#ifndef GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_CLIENT_CHANNEL_H |
||||
#define GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_CLIENT_CHANNEL_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include "src/core/ext/filters/client_channel/client_channel_channelz.h" |
||||
#include <map> |
||||
#include <memory> |
||||
#include <set> |
||||
#include <string> |
||||
|
||||
#include "absl/status/status.h" |
||||
#include "absl/types/optional.h" |
||||
|
||||
#include <grpc/support/log.h> |
||||
|
||||
#include "src/core/ext/filters/client_channel/client_channel_factory.h" |
||||
#include "src/core/ext/filters/client_channel/config_selector.h" |
||||
#include "src/core/ext/filters/client_channel/dynamic_filters.h" |
||||
#include "src/core/ext/filters/client_channel/lb_policy.h" |
||||
#include "src/core/ext/filters/client_channel/resolver.h" |
||||
#include "src/core/lib/channel/channel_stack.h" |
||||
#include "src/core/ext/filters/client_channel/resolver_result_parsing.h" |
||||
#include "src/core/ext/filters/client_channel/retry_throttle.h" |
||||
#include "src/core/ext/filters/client_channel/service_config.h" |
||||
#include "src/core/ext/filters/client_channel/subchannel.h" |
||||
#include "src/core/ext/filters/client_channel/subchannel_pool_interface.h" |
||||
#include "src/core/lib/gprpp/sync.h" |
||||
#include "src/core/lib/iomgr/error.h" |
||||
#include "src/core/lib/iomgr/polling_entity.h" |
||||
#include "src/core/lib/iomgr/work_serializer.h" |
||||
#include "src/core/lib/surface/channel.h" |
||||
#include "src/core/lib/transport/connectivity_state.h" |
||||
|
||||
//
|
||||
// Client channel filter
|
||||
//
|
||||
|
||||
// A client channel is a channel that begins disconnected, and can connect
|
||||
// to some endpoint on demand. If that endpoint disconnects, it will be
|
||||
// connected to again later.
|
||||
//
|
||||
// Calls on a disconnected client channel are queued until a connection is
|
||||
// established.
|
||||
|
||||
// Channel arg key for server URI string.
|
||||
#define GRPC_ARG_SERVER_URI "grpc.server_uri" |
||||
|
||||
/* A client channel is a channel that begins disconnected, and can connect
|
||||
to some endpoint on demand. If that endpoint disconnects, it will be |
||||
connected to again later. |
||||
// Channel arg containing a pointer to the ClientChannel object.
|
||||
#define GRPC_ARG_CLIENT_CHANNEL "grpc.internal.client_channel" |
||||
|
||||
// Channel arg containing a pointer to the ServiceConfig object.
|
||||
#define GRPC_ARG_SERVICE_CONFIG_OBJ "grpc.internal.service_config_obj" |
||||
|
||||
// Max number of batches that can be pending on a call at any given
|
||||
// time. This includes one batch for each of the following ops:
|
||||
// recv_initial_metadata
|
||||
// send_initial_metadata
|
||||
// recv_message
|
||||
// send_message
|
||||
// recv_trailing_metadata
|
||||
// send_trailing_metadata
|
||||
#define MAX_PENDING_BATCHES 6 |
||||
|
||||
namespace grpc_core { |
||||
|
||||
class ClientChannel { |
||||
public: |
||||
static const grpc_channel_filter kFilterVtable; |
||||
|
||||
class LoadBalancedCall; |
||||
|
||||
// Returns the ClientChannel object from channel, or null if channel
|
||||
// is not a client channel.
|
||||
static ClientChannel* GetFromChannel(grpc_channel* channel); |
||||
|
||||
grpc_connectivity_state CheckConnectivityState(bool try_to_connect); |
||||
|
||||
// Starts a one-time connectivity state watch. When the channel's state
|
||||
// becomes different from *state, sets *state to the new state and
|
||||
// schedules on_complete. The watcher_timer_init callback is invoked as
|
||||
// soon as the watch is actually started (i.e., after hopping into the
|
||||
// client channel combiner). I/O will be serviced via pollent.
|
||||
//
|
||||
// This is intended to be used when starting a watch from outside of C-core
|
||||
// via grpc_channel_watch_connectivity_state(). It should not be used
|
||||
// by other callers.
|
||||
void AddExternalConnectivityWatcher(grpc_polling_entity pollent, |
||||
grpc_connectivity_state* state, |
||||
grpc_closure* on_complete, |
||||
grpc_closure* watcher_timer_init) { |
||||
new ExternalConnectivityWatcher(this, pollent, state, on_complete, |
||||
watcher_timer_init); |
||||
} |
||||
|
||||
// Cancels a pending external watcher previously added by
|
||||
// AddExternalConnectivityWatcher().
|
||||
void CancelExternalConnectivityWatcher(grpc_closure* on_complete) { |
||||
ExternalConnectivityWatcher::RemoveWatcherFromExternalWatchersMap( |
||||
this, on_complete, /*cancel=*/true); |
||||
} |
||||
|
||||
int NumExternalConnectivityWatchers() const { |
||||
MutexLock lock(&external_watchers_mu_); |
||||
return static_cast<int>(external_watchers_.size()); |
||||
} |
||||
|
||||
// Starts and stops a connectivity watch. The watcher will be initially
|
||||
// notified as soon as the state changes from initial_state and then on
|
||||
// every subsequent state change until either the watch is stopped or
|
||||
// it is notified that the state has changed to SHUTDOWN.
|
||||
//
|
||||
// This is intended to be used when starting watches from code inside of
|
||||
// C-core (e.g., for a nested control plane channel for things like xds).
|
||||
void AddConnectivityWatcher( |
||||
grpc_connectivity_state initial_state, |
||||
OrphanablePtr<AsyncConnectivityStateWatcherInterface> watcher); |
||||
void RemoveConnectivityWatcher( |
||||
AsyncConnectivityStateWatcherInterface* watcher); |
||||
|
||||
RefCountedPtr<LoadBalancedCall> CreateLoadBalancedCall( |
||||
const grpc_call_element_args& args, grpc_polling_entity* pollent, |
||||
size_t parent_data_size); |
||||
|
||||
private: |
||||
class CallData; |
||||
class ResolverResultHandler; |
||||
class SubchannelWrapper; |
||||
class ClientChannelControlHelper; |
||||
class ConnectivityWatcherAdder; |
||||
class ConnectivityWatcherRemover; |
||||
|
||||
// Represents a pending connectivity callback from an external caller
|
||||
// via grpc_client_channel_watch_connectivity_state().
|
||||
class ExternalConnectivityWatcher : public ConnectivityStateWatcherInterface { |
||||
public: |
||||
ExternalConnectivityWatcher(ClientChannel* chand, |
||||
grpc_polling_entity pollent, |
||||
grpc_connectivity_state* state, |
||||
grpc_closure* on_complete, |
||||
grpc_closure* watcher_timer_init); |
||||
|
||||
~ExternalConnectivityWatcher() override; |
||||
|
||||
// Removes the watcher from the external_watchers_ map.
|
||||
static void RemoveWatcherFromExternalWatchersMap(ClientChannel* chand, |
||||
grpc_closure* on_complete, |
||||
bool cancel); |
||||
|
||||
void Notify(grpc_connectivity_state state, |
||||
const absl::Status& /* status */) override; |
||||
|
||||
void Cancel(); |
||||
|
||||
private: |
||||
// Adds the watcher to state_tracker_. Consumes the ref that is passed to it
|
||||
// from Start().
|
||||
void AddWatcherLocked(); |
||||
void RemoveWatcherLocked(); |
||||
|
||||
ClientChannel* chand_; |
||||
grpc_polling_entity pollent_; |
||||
grpc_connectivity_state initial_state_; |
||||
grpc_connectivity_state* state_; |
||||
grpc_closure* on_complete_; |
||||
grpc_closure* watcher_timer_init_; |
||||
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** error); |
||||
~ClientChannel(); |
||||
|
||||
// Filter vtable functions.
|
||||
static grpc_error* Init(grpc_channel_element* elem, |
||||
grpc_channel_element_args* args); |
||||
static void Destroy(grpc_channel_element* elem); |
||||
static void StartTransportOp(grpc_channel_element* elem, |
||||
grpc_transport_op* op); |
||||
static void GetChannelInfo(grpc_channel_element* elem, |
||||
const grpc_channel_info* info); |
||||
|
||||
// Note: Does NOT return a new ref.
|
||||
grpc_error* disconnect_error() const { |
||||
return disconnect_error_.Load(MemoryOrder::ACQUIRE); |
||||
} |
||||
|
||||
// Note: All methods with "Locked" suffix must be invoked from within
|
||||
// work_serializer_.
|
||||
|
||||
Calls on a disconnected client channel are queued until a connection is |
||||
established. */ |
||||
void OnResolverResultChangedLocked(Resolver::Result result); |
||||
void OnResolverErrorLocked(grpc_error* error); |
||||
|
||||
extern const grpc_channel_filter grpc_client_channel_filter; |
||||
void CreateOrUpdateLbPolicyLocked( |
||||
RefCountedPtr<LoadBalancingPolicy::Config> lb_policy_config, |
||||
Resolver::Result result); |
||||
OrphanablePtr<LoadBalancingPolicy> CreateLbPolicyLocked( |
||||
const grpc_channel_args& args); |
||||
|
||||
grpc_connectivity_state grpc_client_channel_check_connectivity_state( |
||||
grpc_channel_element* elem, int try_to_connect); |
||||
void UpdateStateAndPickerLocked( |
||||
grpc_connectivity_state state, const absl::Status& status, |
||||
const char* reason, |
||||
std::unique_ptr<LoadBalancingPolicy::SubchannelPicker> picker); |
||||
|
||||
int grpc_client_channel_num_external_connectivity_watchers( |
||||
grpc_channel_element* elem); |
||||
void UpdateServiceConfigInControlPlaneLocked( |
||||
RefCountedPtr<ServiceConfig> service_config, |
||||
RefCountedPtr<ConfigSelector> config_selector, |
||||
const internal::ClientChannelGlobalParsedConfig* parsed_service_config, |
||||
const char* lb_policy_name); |
||||
|
||||
void UpdateServiceConfigInDataPlaneLocked(); |
||||
|
||||
void CreateResolverLocked(); |
||||
void DestroyResolverAndLbPolicyLocked(); |
||||
|
||||
grpc_error* DoPingLocked(grpc_transport_op* op); |
||||
|
||||
void StartTransportOpLocked(grpc_transport_op* op); |
||||
|
||||
void TryToConnectLocked(); |
||||
|
||||
// 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_); |
||||
RefCountedPtr<ConnectedSubchannel> GetConnectedSubchannelInDataPlane( |
||||
SubchannelInterface* subchannel) const |
||||
ABSL_EXCLUSIVE_LOCKS_REQUIRED(data_plane_mu_); |
||||
|
||||
//
|
||||
// Fields set at construction and never modified.
|
||||
//
|
||||
const bool deadline_checking_enabled_; |
||||
const bool enable_retries_; |
||||
const size_t per_rpc_retry_buffer_size_; |
||||
grpc_channel_stack* owning_stack_; |
||||
ClientChannelFactory* client_channel_factory_; |
||||
const grpc_channel_args* channel_args_; |
||||
RefCountedPtr<ServiceConfig> default_service_config_; |
||||
std::string server_name_; |
||||
UniquePtr<char> target_uri_; |
||||
channelz::ChannelNode* channelz_node_; |
||||
|
||||
//
|
||||
// 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; |
||||
// Data from service config.
|
||||
grpc_error* resolver_transient_failure_error_ |
||||
ABSL_GUARDED_BY(resolution_mu_) = GRPC_ERROR_NONE; |
||||
bool received_service_config_data_ ABSL_GUARDED_BY(resolution_mu_) = false; |
||||
RefCountedPtr<ServiceConfig> service_config_ ABSL_GUARDED_BY(resolution_mu_); |
||||
RefCountedPtr<ConfigSelector> config_selector_ |
||||
ABSL_GUARDED_BY(resolution_mu_); |
||||
RefCountedPtr<DynamicFilters> dynamic_filters_ |
||||
ABSL_GUARDED_BY(resolution_mu_); |
||||
|
||||
//
|
||||
// Fields used in the data plane. Guarded by data_plane_mu_.
|
||||
//
|
||||
mutable Mutex data_plane_mu_; |
||||
std::unique_ptr<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; |
||||
|
||||
//
|
||||
// Fields used in the control plane. Guarded by work_serializer.
|
||||
//
|
||||
std::shared_ptr<WorkSerializer> work_serializer_; |
||||
grpc_pollset_set* interested_parties_; |
||||
ConnectivityStateTracker state_tracker_; |
||||
OrphanablePtr<Resolver> resolver_; |
||||
bool previous_resolution_contained_addresses_ = false; |
||||
RefCountedPtr<ServiceConfig> saved_service_config_; |
||||
RefCountedPtr<ConfigSelector> saved_config_selector_; |
||||
absl::optional<std::string> health_check_service_name_; |
||||
OrphanablePtr<LoadBalancingPolicy> lb_policy_; |
||||
RefCountedPtr<SubchannelPoolInterface> subchannel_pool_; |
||||
// The number of SubchannelWrapper instances referencing a given Subchannel.
|
||||
std::map<Subchannel*, int> subchannel_refcount_map_; |
||||
// 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_; |
||||
// Pending ConnectedSubchannel updates for each SubchannelWrapper.
|
||||
// Updates are queued here in the control plane work_serializer and then
|
||||
// applied in the data plane mutex when the picker is updated.
|
||||
std::map<RefCountedPtr<SubchannelWrapper>, RefCountedPtr<ConnectedSubchannel>> |
||||
pending_subchannel_updates_; |
||||
int keepalive_time_ = -1; |
||||
|
||||
//
|
||||
// Fields accessed from both data plane mutex and control plane
|
||||
// work_serializer.
|
||||
//
|
||||
Atomic<grpc_error*> disconnect_error_; |
||||
|
||||
//
|
||||
// Fields guarded by a mutex, since they need to be accessed
|
||||
// synchronously via get_channel_info().
|
||||
//
|
||||
Mutex info_mu_; |
||||
UniquePtr<char> info_lb_policy_name_ ABSL_GUARDED_BY(info_mu_); |
||||
UniquePtr<char> info_service_config_json_ ABSL_GUARDED_BY(info_mu_); |
||||
|
||||
//
|
||||
// Fields guarded by a mutex, since they need to be accessed
|
||||
// synchronously via grpc_channel_num_external_connectivity_watchers().
|
||||
//
|
||||
mutable Mutex external_watchers_mu_; |
||||
std::map<grpc_closure*, RefCountedPtr<ExternalConnectivityWatcher>> |
||||
external_watchers_ ABSL_GUARDED_BY(external_watchers_mu_); |
||||
}; |
||||
|
||||
// Starts a one-time connectivity state watch. When the channel's state
|
||||
// becomes different from *state, sets *state to the new state and
|
||||
// schedules on_complete. The watcher_timer_init callback is invoked as
|
||||
// soon as the watch is actually started (i.e., after hopping into the
|
||||
// client channel combiner). I/O will be serviced via pollent.
|
||||
//
|
||||
// This is intended to be used when starting a watch from outside of C-core
|
||||
// via grpc_channel_watch_connectivity_state(). It should not be used
|
||||
// by other callers.
|
||||
void grpc_client_channel_watch_connectivity_state( |
||||
grpc_channel_element* elem, grpc_polling_entity pollent, |
||||
grpc_connectivity_state* state, grpc_closure* on_complete, |
||||
grpc_closure* watcher_timer_init); |
||||
|
||||
// Starts and stops a connectivity watch. The watcher will be initially
|
||||
// notified as soon as the state changes from initial_state and then on
|
||||
// every subsequent state change until either the watch is stopped or
|
||||
// it is notified that the state has changed to SHUTDOWN.
|
||||
// ClientChannel::LoadBalancedCall
|
||||
//
|
||||
// This is intended to be used when starting watches from code inside of
|
||||
// C-core (e.g., for a nested control plane channel for things like xds).
|
||||
void grpc_client_channel_start_connectivity_watch( |
||||
grpc_channel_element* elem, grpc_connectivity_state initial_state, |
||||
grpc_core::OrphanablePtr<grpc_core::AsyncConnectivityStateWatcherInterface> |
||||
watcher); |
||||
void grpc_client_channel_stop_connectivity_watch( |
||||
grpc_channel_element* elem, |
||||
grpc_core::AsyncConnectivityStateWatcherInterface* watcher); |
||||
|
||||
// This object is ref-counted, but it cannot inherit from RefCounted<>,
|
||||
// because it is allocated on the arena and can't free its memory when
|
||||
// its refcount goes to zero. So instead, it manually implements the
|
||||
// same API as RefCounted<>, so that it can be used with RefCountedPtr<>.
|
||||
class ClientChannel::LoadBalancedCall { |
||||
public: |
||||
LoadBalancedCall(ClientChannel* chand, const grpc_call_element_args& args, |
||||
grpc_polling_entity* pollent); |
||||
~LoadBalancedCall(); |
||||
|
||||
// Interface of RefCounted<>.
|
||||
RefCountedPtr<LoadBalancedCall> Ref() GRPC_MUST_USE_RESULT; |
||||
RefCountedPtr<LoadBalancedCall> Ref(const DebugLocation& location, |
||||
const char* reason) GRPC_MUST_USE_RESULT; |
||||
// When refcount drops to 0, destroys itself and the associated call stack,
|
||||
// but does NOT free the memory because it's in the call arena.
|
||||
void Unref(); |
||||
void Unref(const DebugLocation& location, const char* reason); |
||||
|
||||
void* GetParentData(); |
||||
|
||||
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* 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** 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* error); |
||||
|
||||
RefCountedPtr<SubchannelCall> subchannel_call() const { |
||||
return subchannel_call_; |
||||
} |
||||
|
||||
private: |
||||
// Allow RefCountedPtr<> to access IncrementRefCount().
|
||||
template <typename T> |
||||
friend class ::grpc_core::RefCountedPtr; |
||||
|
||||
class LbQueuedCallCanceller; |
||||
class Metadata; |
||||
class LbCallState; |
||||
|
||||
// Interface of RefCounted<>.
|
||||
void IncrementRefCount(); |
||||
void IncrementRefCount(const DebugLocation& location, const char* reason); |
||||
|
||||
// Returns the index into pending_batches_ to be used for batch.
|
||||
static size_t GetBatchIndex(grpc_transport_stream_op_batch* batch); |
||||
void PendingBatchesAdd(grpc_transport_stream_op_batch* batch); |
||||
static void FailPendingBatchInCallCombiner(void* arg, grpc_error* error); |
||||
// A predicate type and some useful implementations for PendingBatchesFail().
|
||||
typedef bool (*YieldCallCombinerPredicate)( |
||||
const CallCombinerClosureList& closures); |
||||
static bool YieldCallCombiner(const CallCombinerClosureList& /*closures*/) { |
||||
return true; |
||||
} |
||||
static bool NoYieldCallCombiner(const CallCombinerClosureList& /*closures*/) { |
||||
return false; |
||||
} |
||||
static bool YieldCallCombinerIfPendingBatchesFound( |
||||
const CallCombinerClosureList& closures) { |
||||
return closures.size() > 0; |
||||
} |
||||
// Fails all pending batches.
|
||||
// If yield_call_combiner_predicate returns true, assumes responsibility for
|
||||
// yielding the call combiner.
|
||||
void PendingBatchesFail( |
||||
grpc_error* error, |
||||
YieldCallCombinerPredicate yield_call_combiner_predicate); |
||||
static void ResumePendingBatchInCallCombiner(void* arg, grpc_error* ignored); |
||||
// Resumes all pending batches on subchannel_call_.
|
||||
void PendingBatchesResume(); |
||||
|
||||
static void RecvTrailingMetadataReadyForLoadBalancingPolicy( |
||||
void* arg, grpc_error* error); |
||||
void InjectRecvTrailingMetadataReadyForLoadBalancingPolicy( |
||||
grpc_transport_stream_op_batch* batch); |
||||
|
||||
void CreateSubchannelCall(); |
||||
// Invoked when a pick is completed, on both success or failure.
|
||||
static void PickDone(void* arg, grpc_error* error); |
||||
// Removes the call from the channel's list of queued picks if present.
|
||||
void MaybeRemoveCallFromLbQueuedCallsLocked() |
||||
ABSL_EXCLUSIVE_LOCKS_REQUIRED(&ClientChannel::data_plane_mu_); |
||||
// 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_); |
||||
|
||||
RefCount refs_; |
||||
|
||||
ClientChannel* chand_; |
||||
|
||||
// TODO(roth): Instead of duplicating these fields in every filter
|
||||
// that uses any one of them, we should store them in the call
|
||||
// context. This will save per-call memory overhead.
|
||||
grpc_slice path_; // Request path.
|
||||
gpr_cycle_counter call_start_time_; |
||||
grpc_millis deadline_; |
||||
Arena* arena_; |
||||
grpc_call_stack* owning_call_; |
||||
CallCombiner* call_combiner_; |
||||
grpc_call_context_element* call_context_; |
||||
|
||||
// Set when we get a cancel_stream op.
|
||||
grpc_error* cancel_error_ = GRPC_ERROR_NONE; |
||||
|
||||
grpc_polling_entity* pollent_ = nullptr; |
||||
|
||||
grpc_closure pick_closure_; |
||||
|
||||
// Accessed while holding ClientChannel::data_plane_mu_.
|
||||
ClientChannel::LbQueuedCall queued_call_; |
||||
bool queued_pending_lb_pick_ = false; |
||||
const LoadBalancingPolicy::BackendMetricData* backend_metric_data_ = nullptr; |
||||
RefCountedPtr<ConnectedSubchannel> connected_subchannel_; |
||||
std::function<void(grpc_error*, LoadBalancingPolicy::MetadataInterface*, |
||||
LoadBalancingPolicy::CallState*)> |
||||
lb_recv_trailing_metadata_ready_; |
||||
LbQueuedCallCanceller* lb_call_canceller_ = nullptr; |
||||
|
||||
RefCountedPtr<SubchannelCall> subchannel_call_; |
||||
|
||||
// For intercepting recv_trailing_metadata_ready for the LB policy.
|
||||
grpc_metadata_batch* recv_trailing_metadata_ = nullptr; |
||||
grpc_closure recv_trailing_metadata_ready_; |
||||
grpc_closure* original_recv_trailing_metadata_ready_ = nullptr; |
||||
|
||||
// Batches are added to this list when received from above.
|
||||
// They are removed when we are done handling the batch (i.e., when
|
||||
// either we have invoked all of the batch's callbacks or we have
|
||||
// passed the batch down to the subchannel call and are not
|
||||
// intercepting any of its callbacks).
|
||||
grpc_transport_stream_op_batch* pending_batches_[MAX_PENDING_BATCHES] = {}; |
||||
}; |
||||
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_CLIENT_CHANNEL_H
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,30 @@ |
||||
//
|
||||
// Copyright 2021 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#ifndef GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_RETRY_FILTER_H |
||||
#define GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_RETRY_FILTER_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include "src/core/lib/channel/channel_stack.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
extern const grpc_channel_filter kRetryFilterVtable; |
||||
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_RETRY_FILTER_H
|
@ -0,0 +1,285 @@ |
||||
//
|
||||
// Copyright 2018 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include "src/core/ext/filters/client_channel/retry_service_config.h" |
||||
|
||||
#include <ctype.h> |
||||
#include <stdio.h> |
||||
#include <string.h> |
||||
|
||||
#include "absl/strings/str_cat.h" |
||||
#include "absl/types/optional.h" |
||||
|
||||
#include <grpc/support/alloc.h> |
||||
#include <grpc/support/log.h> |
||||
#include <grpc/support/string_util.h> |
||||
|
||||
#include "src/core/ext/filters/client_channel/client_channel.h" |
||||
#include "src/core/ext/filters/client_channel/lb_policy_registry.h" |
||||
#include "src/core/ext/filters/client_channel/server_address.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/channel/status_util.h" |
||||
#include "src/core/lib/gpr/string.h" |
||||
#include "src/core/lib/gprpp/memory.h" |
||||
#include "src/core/lib/json/json_util.h" |
||||
#include "src/core/lib/uri/uri_parser.h" |
||||
|
||||
// As per the retry design, we do not allow more than 5 retry attempts.
|
||||
#define MAX_MAX_RETRY_ATTEMPTS 5 |
||||
|
||||
namespace grpc_core { |
||||
namespace internal { |
||||
|
||||
namespace { |
||||
size_t g_retry_service_config_parser_index; |
||||
} |
||||
|
||||
size_t RetryServiceConfigParser::ParserIndex() { |
||||
return g_retry_service_config_parser_index; |
||||
} |
||||
|
||||
void RetryServiceConfigParser::Register() { |
||||
g_retry_service_config_parser_index = ServiceConfigParser::RegisterParser( |
||||
absl::make_unique<RetryServiceConfigParser>()); |
||||
} |
||||
|
||||
namespace { |
||||
|
||||
grpc_error* ParseRetryThrottling(const Json& json, intptr_t* max_milli_tokens, |
||||
intptr_t* milli_token_ratio) { |
||||
if (json.type() != Json::Type::OBJECT) { |
||||
return GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling error:Type should be object"); |
||||
} |
||||
std::vector<grpc_error*> error_list; |
||||
// Parse maxTokens.
|
||||
auto it = json.object_value().find("maxTokens"); |
||||
if (it == json.object_value().end()) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:maxTokens error:Not found")); |
||||
} else if (it->second.type() != Json::Type::NUMBER) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:maxTokens error:Type should be " |
||||
"number")); |
||||
} else { |
||||
*max_milli_tokens = |
||||
gpr_parse_nonnegative_int(it->second.string_value().c_str()) * 1000; |
||||
if (*max_milli_tokens <= 0) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:maxTokens error:should be " |
||||
"greater than zero")); |
||||
} |
||||
} |
||||
// Parse tokenRatio.
|
||||
it = json.object_value().find("tokenRatio"); |
||||
if (it == json.object_value().end()) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:tokenRatio error:Not found")); |
||||
} else if (it->second.type() != Json::Type::NUMBER) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:tokenRatio error:type should be " |
||||
"number")); |
||||
} else { |
||||
// We support up to 3 decimal digits.
|
||||
size_t whole_len = it->second.string_value().size(); |
||||
const char* value = it->second.string_value().c_str(); |
||||
uint32_t multiplier = 1; |
||||
uint32_t decimal_value = 0; |
||||
const char* decimal_point = strchr(value, '.'); |
||||
if (decimal_point != nullptr) { |
||||
whole_len = static_cast<size_t>(decimal_point - value); |
||||
multiplier = 1000; |
||||
size_t decimal_len = strlen(decimal_point + 1); |
||||
if (decimal_len > 3) decimal_len = 3; |
||||
if (!gpr_parse_bytes_to_uint32(decimal_point + 1, decimal_len, |
||||
&decimal_value)) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:tokenRatio error:Failed " |
||||
"parsing")); |
||||
return GRPC_ERROR_CREATE_FROM_VECTOR("retryThrottling", &error_list); |
||||
} |
||||
uint32_t decimal_multiplier = 1; |
||||
for (size_t i = 0; i < (3 - decimal_len); ++i) { |
||||
decimal_multiplier *= 10; |
||||
} |
||||
decimal_value *= decimal_multiplier; |
||||
} |
||||
uint32_t whole_value; |
||||
if (!gpr_parse_bytes_to_uint32(value, whole_len, &whole_value)) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:tokenRatio error:Failed " |
||||
"parsing")); |
||||
return GRPC_ERROR_CREATE_FROM_VECTOR("retryThrottling", &error_list); |
||||
} |
||||
*milli_token_ratio = |
||||
static_cast<int>((whole_value * multiplier) + decimal_value); |
||||
if (*milli_token_ratio <= 0) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryThrottling field:tokenRatio error:value should " |
||||
"be greater than 0")); |
||||
} |
||||
} |
||||
return GRPC_ERROR_CREATE_FROM_VECTOR("retryThrottling", &error_list); |
||||
} |
||||
|
||||
} // namespace
|
||||
|
||||
std::unique_ptr<ServiceConfigParser::ParsedConfig> |
||||
RetryServiceConfigParser::ParseGlobalParams(const grpc_channel_args* /*args*/, |
||||
const Json& json, |
||||
grpc_error** error) { |
||||
GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE); |
||||
auto it = json.object_value().find("retryThrottling"); |
||||
if (it == json.object_value().end()) return nullptr; |
||||
intptr_t max_milli_tokens = 0; |
||||
intptr_t milli_token_ratio = 0; |
||||
*error = |
||||
ParseRetryThrottling(it->second, &max_milli_tokens, &milli_token_ratio); |
||||
if (*error != GRPC_ERROR_NONE) return nullptr; |
||||
return absl::make_unique<RetryGlobalConfig>(max_milli_tokens, |
||||
milli_token_ratio); |
||||
} |
||||
|
||||
namespace { |
||||
|
||||
grpc_error* ParseRetryPolicy(const Json& json, int* max_attempts, |
||||
grpc_millis* initial_backoff, |
||||
grpc_millis* max_backoff, |
||||
float* backoff_multiplier, |
||||
StatusCodeSet* retryable_status_codes) { |
||||
if (json.type() != Json::Type::OBJECT) { |
||||
return GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryPolicy error:should be of type object"); |
||||
} |
||||
std::vector<grpc_error*> error_list; |
||||
// Parse maxAttempts.
|
||||
auto it = json.object_value().find("maxAttempts"); |
||||
if (it != json.object_value().end()) { |
||||
if (it->second.type() != Json::Type::NUMBER) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:maxAttempts error:should be of type number")); |
||||
} else { |
||||
*max_attempts = |
||||
gpr_parse_nonnegative_int(it->second.string_value().c_str()); |
||||
if (*max_attempts <= 1) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:maxAttempts error:should be at least 2")); |
||||
} else if (*max_attempts > MAX_MAX_RETRY_ATTEMPTS) { |
||||
gpr_log(GPR_ERROR, |
||||
"service config: clamped retryPolicy.maxAttempts at %d", |
||||
MAX_MAX_RETRY_ATTEMPTS); |
||||
*max_attempts = MAX_MAX_RETRY_ATTEMPTS; |
||||
} |
||||
} |
||||
} |
||||
// Parse initialBackoff.
|
||||
if (ParseJsonObjectFieldAsDuration(json.object_value(), "initialBackoff", |
||||
initial_backoff, &error_list) && |
||||
*initial_backoff == 0) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:initialBackoff error:must be greater than 0")); |
||||
} |
||||
// Parse maxBackoff.
|
||||
if (ParseJsonObjectFieldAsDuration(json.object_value(), "maxBackoff", |
||||
max_backoff, &error_list) && |
||||
*max_backoff == 0) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:maxBackoff error:should be greater than 0")); |
||||
} |
||||
// Parse backoffMultiplier.
|
||||
it = json.object_value().find("backoffMultiplier"); |
||||
if (it != json.object_value().end()) { |
||||
if (it->second.type() != Json::Type::NUMBER) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:backoffMultiplier error:should be of type number")); |
||||
} else { |
||||
if (sscanf(it->second.string_value().c_str(), "%f", backoff_multiplier) != |
||||
1) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:backoffMultiplier error:failed to parse")); |
||||
} else if (*backoff_multiplier <= 0) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:backoffMultiplier error:should be greater than 0")); |
||||
} |
||||
} |
||||
} |
||||
// Parse retryableStatusCodes.
|
||||
it = json.object_value().find("retryableStatusCodes"); |
||||
if (it != json.object_value().end()) { |
||||
if (it->second.type() != Json::Type::ARRAY) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryableStatusCodes error:should be of type array")); |
||||
} else { |
||||
for (const Json& element : it->second.array_value()) { |
||||
if (element.type() != Json::Type::STRING) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryableStatusCodes error:status codes should be of type " |
||||
"string")); |
||||
continue; |
||||
} |
||||
grpc_status_code status; |
||||
if (!grpc_status_code_from_string(element.string_value().c_str(), |
||||
&status)) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryableStatusCodes error:failed to parse status code")); |
||||
continue; |
||||
} |
||||
retryable_status_codes->Add(status); |
||||
} |
||||
if (retryable_status_codes->Empty()) { |
||||
error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryableStatusCodes error:should be non-empty")); |
||||
}; |
||||
} |
||||
} |
||||
// Make sure required fields are set.
|
||||
if (error_list.empty()) { |
||||
if (*max_attempts == 0 || *initial_backoff == 0 || *max_backoff == 0 || |
||||
*backoff_multiplier == 0 || retryable_status_codes->Empty()) { |
||||
return GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:retryPolicy error:Missing required field(s)"); |
||||
} |
||||
} |
||||
return GRPC_ERROR_CREATE_FROM_VECTOR("retryPolicy", &error_list); |
||||
} |
||||
|
||||
} // namespace
|
||||
|
||||
std::unique_ptr<ServiceConfigParser::ParsedConfig> |
||||
RetryServiceConfigParser::ParsePerMethodParams( |
||||
const grpc_channel_args* /*args*/, const Json& json, grpc_error** error) { |
||||
GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE); |
||||
// Parse retry policy.
|
||||
auto it = json.object_value().find("retryPolicy"); |
||||
if (it == json.object_value().end()) return nullptr; |
||||
int max_attempts = 0; |
||||
grpc_millis initial_backoff = 0; |
||||
grpc_millis max_backoff = 0; |
||||
float backoff_multiplier = 0; |
||||
StatusCodeSet retryable_status_codes; |
||||
*error = ParseRetryPolicy(it->second, &max_attempts, &initial_backoff, |
||||
&max_backoff, &backoff_multiplier, |
||||
&retryable_status_codes); |
||||
if (*error != GRPC_ERROR_NONE) return nullptr; |
||||
return absl::make_unique<RetryMethodConfig>(max_attempts, initial_backoff, |
||||
max_backoff, backoff_multiplier, |
||||
retryable_status_codes); |
||||
} |
||||
|
||||
} // namespace internal
|
||||
} // namespace grpc_core
|
@ -0,0 +1,90 @@ |
||||
//
|
||||
// Copyright 2018 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#ifndef GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_RETRY_SERVICE_CONFIG_H |
||||
#define GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_RETRY_SERVICE_CONFIG_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <memory> |
||||
|
||||
#include "src/core/ext/filters/client_channel/retry_throttle.h" |
||||
#include "src/core/ext/filters/client_channel/service_config_parser.h" |
||||
#include "src/core/lib/channel/status_util.h" |
||||
#include "src/core/lib/iomgr/exec_ctx.h" // for grpc_millis |
||||
|
||||
namespace grpc_core { |
||||
namespace internal { |
||||
|
||||
class RetryGlobalConfig : public ServiceConfigParser::ParsedConfig { |
||||
public: |
||||
RetryGlobalConfig(intptr_t max_milli_tokens, intptr_t milli_token_ratio) |
||||
: max_milli_tokens_(max_milli_tokens), |
||||
milli_token_ratio_(milli_token_ratio) {} |
||||
|
||||
intptr_t max_milli_tokens() const { return max_milli_tokens_; } |
||||
intptr_t milli_token_ratio() const { return milli_token_ratio_; } |
||||
|
||||
private: |
||||
intptr_t max_milli_tokens_ = 0; |
||||
intptr_t milli_token_ratio_ = 0; |
||||
}; |
||||
|
||||
class RetryMethodConfig : public ServiceConfigParser::ParsedConfig { |
||||
public: |
||||
RetryMethodConfig(int max_attempts, grpc_millis initial_backoff, |
||||
grpc_millis max_backoff, float backoff_multiplier, |
||||
StatusCodeSet retryable_status_codes) |
||||
: max_attempts_(max_attempts), |
||||
initial_backoff_(initial_backoff), |
||||
max_backoff_(max_backoff), |
||||
backoff_multiplier_(backoff_multiplier), |
||||
retryable_status_codes_(retryable_status_codes) {} |
||||
|
||||
int max_attempts() const { return max_attempts_; } |
||||
grpc_millis initial_backoff() const { return initial_backoff_; } |
||||
grpc_millis max_backoff() const { return max_backoff_; } |
||||
float backoff_multiplier() const { return backoff_multiplier_; } |
||||
StatusCodeSet retryable_status_codes() const { |
||||
return retryable_status_codes_; |
||||
} |
||||
|
||||
private: |
||||
int max_attempts_ = 0; |
||||
grpc_millis initial_backoff_ = 0; |
||||
grpc_millis max_backoff_ = 0; |
||||
float backoff_multiplier_ = 0; |
||||
StatusCodeSet retryable_status_codes_; |
||||
}; |
||||
|
||||
class RetryServiceConfigParser : public ServiceConfigParser::Parser { |
||||
public: |
||||
std::unique_ptr<ServiceConfigParser::ParsedConfig> ParseGlobalParams( |
||||
const grpc_channel_args* /*args*/, const Json& json, |
||||
grpc_error** error) override; |
||||
|
||||
std::unique_ptr<ServiceConfigParser::ParsedConfig> ParsePerMethodParams( |
||||
const grpc_channel_args* /*args*/, const Json& json, |
||||
grpc_error** error) override; |
||||
|
||||
static size_t ParserIndex(); |
||||
static void Register(); |
||||
}; |
||||
|
||||
} // namespace internal
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_RETRY_SERVICE_CONFIG_H
|
Loading…
Reference in new issue