From 4993962c9add0ed61e869e327e7c63f72ffbb9d4 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 16 Jun 2021 15:05:25 -0700 Subject: [PATCH 01/46] Introduced a unified mux implementation Signed-off-by: Dmitri Dolguikh --- envoy/config/grpc_mux.h | 58 +++ source/common/config/xds_mux/BUILD | 21 + .../config/xds_mux/delta_subscription_state.h | 15 + source/common/config/xds_mux/grpc_mux_impl.cc | 422 ++++++++++++++++++ source/common/config/xds_mux/grpc_mux_impl.h | 263 +++++++++++ .../config/xds_mux/sotw_subscription_state.cc | 2 +- .../config/xds_mux/sotw_subscription_state.h | 15 + .../config/xds_mux/subscription_state.h | 8 + 8 files changed, 803 insertions(+), 1 deletion(-) create mode 100644 source/common/config/xds_mux/grpc_mux_impl.cc create mode 100644 source/common/config/xds_mux/grpc_mux_impl.h diff --git a/envoy/config/grpc_mux.h b/envoy/config/grpc_mux.h index 0139cb3d9524..491154d69f31 100644 --- a/envoy/config/grpc_mux.h +++ b/envoy/config/grpc_mux.h @@ -31,6 +31,10 @@ struct ControlPlaneStats { GENERATE_TEXT_READOUT_STRUCT) }; +// TODO (dmitri-d) This forward declaration is needed in order for the unified mux to fulfill legacy mux interface. +// It should be removed together with the rest of legacy mux code when the switch to unified mux has been finalized. +struct Watch; + /** * Handle on a muxed gRPC subscription. The subscription is canceled on destruction. */ @@ -105,6 +109,60 @@ class GrpcMux { virtual void requestOnDemandUpdate(const std::string& type_url, const absl::flat_hash_set& for_update) PURE; + + // Unified mux interface starts here + /** + * Start a configuration subscription asynchronously for some API type and resources. + * @param type_url type URL corresponding to xDS API, e.g. + * type.googleapis.com/envoy.api.v2.Cluster. + * @param resources set of resource names to watch for. If this is empty, then all + * resources for type_url will result in callbacks. + * @param callbacks the callbacks to be notified of configuration updates. These must be valid + * until GrpcMuxWatch is destroyed. + * @param resource_decoder how incoming opaque resource objects are to be decoded. + * @param use_namespace_matching if namespace watch should be created. This is used for creating + * watches on collections of resources; individual members of a collection are identified by the + * namespace in resource name. + * @return Watch* an opaque watch token added or updated, to be used in future addOrUpdateWatch + * calls. + */ + virtual Watch* addWatch(const std::string& type_url, + const absl::flat_hash_set& resources, + SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, + std::chrono::milliseconds init_fetch_timeout, + const bool use_namespace_matching) PURE; + + // Updates the list of resource names watched by the given watch. If an added name is new across + // the whole subscription, or if a removed name has no other watch interested in it, then the + // subscription will enqueue and attempt to send an appropriate discovery request. + virtual void updateWatch(const std::string& type_url, Watch* watch, + const absl::flat_hash_set& resources, + const bool creating_namespace_watch) PURE; + + // Unified mux interface starts here + /** + * Cleanup of a Watch* added by addOrUpdateWatch(). Receiving a Watch* from addOrUpdateWatch() + * makes you responsible for eventually invoking this cleanup. + * @param type_url type URL corresponding to xDS API e.g. type.googleapis.com/envoy.api.v2.Cluster + * @param watch the watch to be cleaned up. + */ + virtual void removeWatch(const std::string& type_url, Watch* watch) PURE; + + /** + * Retrieves the current pause state as set by pause()/resume(). + * @param type_url type URL corresponding to xDS API, e.g. + * type.googleapis.com/envoy.api.v2.Cluster + * @return bool whether the API is paused. + */ + virtual bool paused(const std::string& type_url) const PURE; + + /** + * Passes through to all multiplexed SubscriptionStates. To be called when something + * definitive happens with the initial fetch: either an update is successfully received, + * or some sort of error happened.*/ + virtual void disableInitFetchTimeoutTimer() PURE; + + virtual bool isUnified() const { return false; } }; using GrpcMuxPtr = std::unique_ptr; diff --git a/source/common/config/xds_mux/BUILD b/source/common/config/xds_mux/BUILD index f934e46e19dc..5a6900d0e456 100644 --- a/source/common/config/xds_mux/BUILD +++ b/source/common/config/xds_mux/BUILD @@ -51,3 +51,24 @@ envoy_cc_library( "@envoy_api//envoy/service/discovery/v3:pkg_cc_proto", ], ) + +envoy_cc_library( + name = "grpc_mux_lib", + srcs = ["grpc_mux_impl.cc"], + hdrs = ["grpc_mux_impl.h"], + deps = [ + ":delta_subscription_state_lib", + ":sotw_subscription_state_lib", + "//envoy/event:dispatcher_interface", + "//envoy/grpc:async_client_interface", + "//source/common/config:api_version_lib", + "//source/common/config:decoded_resource_lib", + "//source/common/config:grpc_stream_lib", + "//source/common/config:pausable_ack_queue_lib", + "//source/common/config:watch_map_lib", + "//source/common/config:xds_context_params_lib", + "//source/common/config:xds_resource_lib", + "//source/common/memory:utils_lib", + "@envoy_api//envoy/service/discovery/v3:pkg_cc_proto", + ], +) \ No newline at end of file diff --git a/source/common/config/xds_mux/delta_subscription_state.h b/source/common/config/xds_mux/delta_subscription_state.h index 801bd5edc0c1..f292b1c6bf60 100644 --- a/source/common/config/xds_mux/delta_subscription_state.h +++ b/source/common/config/xds_mux/delta_subscription_state.h @@ -93,6 +93,21 @@ class DeltaSubscriptionState std::set names_removed_; }; +class DeltaSubscriptionStateFactory : public SubscriptionStateFactory { +public: + DeltaSubscriptionStateFactory(Event::Dispatcher& dispatcher) + : dispatcher_(dispatcher) {} + ~DeltaSubscriptionStateFactory() override = default; + std::unique_ptr makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, + std::chrono::milliseconds init_fetch_timeout, OpaqueResourceDecoder&, const bool wildcard) override { + return std::make_unique(type_url, callbacks, init_fetch_timeout, + dispatcher_, wildcard); + + } +private: + Event::Dispatcher& dispatcher_; +}; + } // namespace XdsMux } // namespace Config } // namespace Envoy diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc new file mode 100644 index 000000000000..396015c86106 --- /dev/null +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -0,0 +1,422 @@ +#include "source/common/config/unified_mux/grpc_mux_impl.h" + +#include "envoy/service/discovery/v3/discovery.pb.h" + +#include "source/common/common/assert.h" +#include "source/common/common/backoff_strategy.h" +#include "source/common/config/decoded_resource_impl.h" +#include "source/common/config/utility.h" +#include "source/common/config/version_converter.h" +#include "source/common/config/xds_context_params.h" +#include "source/common/config/xds_resource.h" +#include "source/common/memory/utils.h" +#include "source/common/protobuf/protobuf.h" +#include "source/common/protobuf/utility.h" + +namespace Envoy { +namespace Config { +namespace UnifiedMux { + +template +GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_factory, + bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, + envoy::config::core::v3::ApiVersion transport_api_version) + : subscription_state_factory_(std::move(subscription_state_factory)), + skip_subsequent_node_(skip_subsequent_node), local_info_(local_info), + dynamic_update_callback_handle_(local_info.contextProvider().addDynamicContextUpdateCallback( + [this](absl::string_view resource_type_url) { + onDynamicContextUpdate(resource_type_url); + })), + transport_api_version_(transport_api_version) { + Config::Utility::checkLocalInfo("ads", local_info); +} + +template +void GrpcMuxImpl::onDynamicContextUpdate(absl::string_view resource_type_url) { + ENVOY_LOG(debug, "GrpcMuxImpl::onDynamicContextUpdate for {}", resource_type_url); + auto sub = subscriptions_.find(resource_type_url); + if (sub == subscriptions_.end()) { + return; + } + sub->second->setDynamicContextChanged(); + trySendDiscoveryRequests(); +} + +template +Watch* GrpcMuxImpl::addWatch(const std::string& type_url, + const absl::flat_hash_set& resources, + SubscriptionCallbacks& callbacks, + OpaqueResourceDecoder& resource_decoder, + std::chrono::milliseconds init_fetch_timeout, + const bool use_namespace_matching) { + auto watch_map = watch_maps_.find(type_url); + if (watch_map == watch_maps_.end()) { + // We don't yet have a subscription for type_url! Make one! + watch_map = + watch_maps_.emplace(type_url, std::make_unique(use_namespace_matching)).first; + subscriptions_.emplace(type_url, subscription_state_factory_->makeSubscriptionState( + type_url, *watch_maps_[type_url], init_fetch_timeout, resource_decoder, use_namespace_matching)); + subscription_ordering_.emplace_back(type_url); + } + + Watch* watch = watch_map->second->addWatch(callbacks, resource_decoder); + // updateWatch() queues a discovery request if any of 'resources' are not yet subscribed. + updateWatch(type_url, watch, resources, use_namespace_matching); + return watch; +} + +// Updates the list of resource names watched by the given watch. If an added name is new across +// the whole subscription, or if a removed name has no other watch interested in it, then the +// subscription will enqueue and attempt to send an appropriate discovery request. +template +void GrpcMuxImpl::updateWatch(const std::string& type_url, Watch* watch, + const absl::flat_hash_set& resources, + const bool creating_namespace_watch) { + ENVOY_LOG(debug, "GrpcMuxImpl::updateWatch for {}", type_url); + ASSERT(watch != nullptr); + auto& sub = subscriptionStateFor(type_url); + WatchMap& watch_map = watchMapFor(type_url); + + // If this is a glob collection subscription, we need to compute actual context parameters. + absl::flat_hash_set xdstp_resources; + // TODO(htuch): add support for resources beyond glob collections, the constraints below around + // resource size and ID reflect the progress of the xdstp:// implementation. + if (!resources.empty() && XdsResourceIdentifier::hasXdsTpScheme(*resources.begin())) { + // Callers must be asking for a single resource, the collection. + ASSERT(resources.size() == 1); + auto resource = XdsResourceIdentifier::decodeUrn(*resources.begin()); + // We only know how to deal with glob collections and static context parameters right now. + // TODO(htuch): add support for dynamic context params and list collections in the future. + if (absl::EndsWith(resource.id(), "/*")) { + auto encoded_context = XdsContextParams::encodeResource( + local_info_.contextProvider().nodeContext(), resource.context(), {}, {}); + resource.mutable_context()->CopyFrom(encoded_context); + XdsResourceIdentifier::EncodeOptions encode_options; + encode_options.sort_context_params_ = true; + xdstp_resources.insert(XdsResourceIdentifier::encodeUrn(resource, encode_options)); + } else { + // TODO(htuch): We will handle list collections here in future work. + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + } + } + + auto added_removed = + watch_map.updateWatchInterest(watch, xdstp_resources.empty() ? resources : xdstp_resources); + if (creating_namespace_watch && xdstp_resources.empty()) { + // This is to prevent sending out of requests that contain prefixes instead of resource names + sub.updateSubscriptionInterest({}, {}); + } else { + sub.updateSubscriptionInterest(added_removed.added_, added_removed.removed_); + } + + // Tell the server about our change in interest, if any. + if (sub.subscriptionUpdatePending()) { + trySendDiscoveryRequests(); + } +} + +template +void GrpcMuxImpl::removeWatch(const std::string& type_url, Watch* watch) { + updateWatch(type_url, watch, {}); + watchMapFor(type_url).removeWatch(watch); +} + +template +ScopedResume GrpcMuxImpl::pause(const std::string& type_url) { + return pause(std::vector{type_url}); +} + +template +ScopedResume GrpcMuxImpl::pause(const std::vector type_urls) { + for (const auto& type_url : type_urls) { + pausable_ack_queue_.pause(type_url); + } + + return std::make_unique([this, type_urls]() { + for (const auto& type_url : type_urls) { + pausable_ack_queue_.resume(type_url); + trySendDiscoveryRequests(); + } + }); +} + +template +bool GrpcMuxImpl::paused(const std::string& type_url) const { + return pausable_ack_queue_.paused(type_url); +} + +template +void GrpcMuxImpl::genericHandleResponse(const std::string& type_url, + const RS& response_proto) { + auto sub = subscriptions_.find(type_url); + if (sub == subscriptions_.end()) { + ENVOY_LOG(warn, + "The server sent an xDS response proto with type_url {}, which we have " + "not subscribed to. Ignoring.", + type_url); + return; + } + pausable_ack_queue_.push(sub->second->handleResponse(response_proto)); + trySendDiscoveryRequests(); + Memory::Utils::tryShrinkHeap(); +} + +template +void GrpcMuxImpl::start() { + ENVOY_LOG(debug, "GrpcMuxImpl now trying to establish a stream"); + establishGrpcStream(); +} + +template +void GrpcMuxImpl::handleEstablishedStream() { + ENVOY_LOG(debug, "GrpcMuxImpl stream successfully established"); + for (auto& [type_url, subscription_state] : subscriptions_) { + subscription_state->markStreamFresh(); + } + setAnyRequestSentYetInCurrentStream(false); + maybeUpdateQueueSizeStat(0); + pausable_ack_queue_.clear(); + trySendDiscoveryRequests(); +} + +template +void GrpcMuxImpl::disableInitFetchTimeoutTimer() { + for (auto& [type_url, subscription_state] : subscriptions_) { + subscription_state->disableInitFetchTimeoutTimer(); + } +} + +template +void GrpcMuxImpl::handleStreamEstablishmentFailure() { + ENVOY_LOG(debug, "GrpcMuxImpl stream failed to establish"); + // If this happens while Envoy is still initializing, the onConfigUpdateFailed() we ultimately + // call on CDS will cause LDS to start up, which adds to subscriptions_ here. So, to avoid a + // crash, the iteration needs to dance around a little: collect pointers to all + // SubscriptionStates, call on all those pointers we haven't yet called on, repeat if there are + // now more SubscriptionStates. + absl::flat_hash_map all_subscribed; + absl::flat_hash_map already_called; + do { + for (auto& [type_url, subscription_state] : subscriptions_) { + all_subscribed[type_url] = subscription_state.get(); + } + for (auto& sub : all_subscribed) { + if (already_called.insert(sub).second) { // insert succeeded ==> not already called + sub.second->handleEstablishmentFailure(); + } + } + } while (all_subscribed.size() != subscriptions_.size()); +} + +template +S& GrpcMuxImpl::subscriptionStateFor(const std::string& type_url) { + auto sub = subscriptions_.find(type_url); + RELEASE_ASSERT(sub != subscriptions_.end(), + fmt::format("Tried to look up SubscriptionState for non-existent subscription {}.", + type_url)); + return *sub->second; +} + +template +WatchMap& GrpcMuxImpl::watchMapFor(const std::string& type_url) { + auto watch_map = watch_maps_.find(type_url); + RELEASE_ASSERT( + watch_map != watch_maps_.end(), + fmt::format("Tried to look up WatchMap for non-existent subscription {}.", type_url)); + return *watch_map->second; +} + +template +void GrpcMuxImpl::trySendDiscoveryRequests() { + while (true) { + // Do any of our subscriptions even want to send a request? + absl::optional request_type_if_any = whoWantsToSendDiscoveryRequest(); + if (!request_type_if_any.has_value()) { + break; + } + // If so, which one (by type_url)? + std::string next_request_type_url = request_type_if_any.value(); + auto& sub = subscriptionStateFor(next_request_type_url); + ENVOY_LOG(debug, "GrpcMuxImpl wants to send discovery request for {}", next_request_type_url); + // Try again later if paused/rate limited/stream down. + if (!canSendDiscoveryRequest(next_request_type_url)) { + break; + } + std::unique_ptr request; + // Get our subscription state to generate the appropriate discovery request, and send. + if (!pausable_ack_queue_.empty()) { + // Because ACKs take precedence over plain requests, if there is anything in the queue, it's + // safe to assume it's of the type_url that we're wanting to send. + // + // getNextRequestWithAck() returns a raw unowned pointer, which sendGrpcMessage deletes. + request = sub.getNextRequestWithAck(pausable_ack_queue_.popFront()); + ENVOY_LOG(debug, "GrpcMuxImpl sent ACK discovery request for {}", next_request_type_url); + } else { + // Returns a raw unowned pointer, which sendGrpcMessage deletes. + request = sub.getNextRequestAckless(); + ENVOY_LOG(debug, "GrpcMuxImpl sent non-ACK discovery request for {}", next_request_type_url); + } + ENVOY_LOG(debug, "GrpcMuxImpl skip_subsequent_node: {}", skipSubsequentNode()); + sendGrpcMessage(*request, sub); + } + maybeUpdateQueueSizeStat(pausable_ack_queue_.size()); +} + +// Checks whether external conditions allow sending a discovery request. (Does not check +// whether we *want* to send a discovery request). +template +bool GrpcMuxImpl::canSendDiscoveryRequest(const std::string& type_url) { + RELEASE_ASSERT( + !pausable_ack_queue_.paused(type_url), + fmt::format("canSendDiscoveryRequest() called on paused type_url {}. Pausedness is " + "supposed to be filtered out by whoWantsToSendDiscoveryRequest(). ", + type_url)); + + if (!grpcStreamAvailable()) { + ENVOY_LOG(trace, "No stream available to send a discovery request for {}.", type_url); + return false; + } else if (!rateLimitAllowsDrain()) { + ENVOY_LOG(trace, "{} discovery request hit rate limit; will try later.", type_url); + return false; + } + return true; +} + +// Checks whether we have something to say in a discovery request, which can be an ACK and/or +// a subscription update. (Does not check whether we *can* send that discovery request). +// Returns the type_url we should send the discovery request for (if any). +// First, prioritizes ACKs over non-ACK subscription interest updates. +// Then, prioritizes non-ACK updates in the order the various types +// of subscriptions were activated. +template +absl::optional GrpcMuxImpl::whoWantsToSendDiscoveryRequest() { + // All ACKs are sent before plain updates. trySendDiscoveryRequests() relies on this. So, choose + // type_url from pausable_ack_queue_ if possible, before looking at pending updates. + if (!pausable_ack_queue_.empty()) { + return pausable_ack_queue_.front().type_url_; + } + // If we're looking to send multiple non-ACK requests, send them in the order that their + // subscriptions were initiated. + for (const auto& sub_type : subscription_ordering_) { + auto& sub = subscriptionStateFor(sub_type); + if (sub.subscriptionUpdatePending() && !pausable_ack_queue_.paused(sub_type)) { + return sub_type; + } + } + return absl::nullopt; +} + +template class GrpcMuxImpl; +template class GrpcMuxImpl; + +// Delta- and SotW-specific concrete subclasses: +GrpcMuxDelta::GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, + const Protobuf::MethodDescriptor& service_method, + envoy::config::core::v3::ApiVersion transport_api_version, + Random::RandomGenerator& random, Stats::Scope& scope, + const RateLimitSettings& rate_limit_settings, + const LocalInfo::LocalInfo& local_info, bool skip_subsequent_node) + : GrpcMuxImpl(std::make_unique(dispatcher), skip_subsequent_node, + local_info, transport_api_version), + grpc_stream_(this, std::move(async_client), service_method, random, dispatcher, scope, + rate_limit_settings) {} + +// GrpcStreamCallbacks for GrpcMuxDelta +void GrpcMuxDelta::onStreamEstablished() { handleEstablishedStream(); } +void GrpcMuxDelta::onEstablishmentFailure() { handleStreamEstablishmentFailure(); } +void GrpcMuxDelta::onWriteable() { trySendDiscoveryRequests(); } +void GrpcMuxDelta::onDiscoveryResponse( + std::unique_ptr&& message, + ControlPlaneStats&) { + genericHandleResponse(message->type_url(), *message); +} + +void GrpcMuxDelta::establishGrpcStream() { grpc_stream_.establishNewStream(); } +void GrpcMuxDelta::sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, DeltaSubscriptionState& sub_state) { + if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || + !skipSubsequentNode()) { + msg_proto.mutable_node()->MergeFrom(localInfo().node()); + } + VersionConverter::prepareMessageForGrpcWire(msg_proto, transportApiVersion()); + grpc_stream_.sendMessage(msg_proto); + setAnyRequestSentYetInCurrentStream(true); + sub_state.clearDynamicContextChanged(); +} +void GrpcMuxDelta::maybeUpdateQueueSizeStat(uint64_t size) { + grpc_stream_.maybeUpdateQueueSizeStat(size); +} +bool GrpcMuxDelta::grpcStreamAvailable() const { return grpc_stream_.grpcStreamAvailable(); } +bool GrpcMuxDelta::rateLimitAllowsDrain() { return grpc_stream_.checkRateLimitAllowsDrain(); } + +void GrpcMuxDelta::requestOnDemandUpdate(const std::string& type_url, + const absl::flat_hash_set& for_update) { + auto& sub = subscriptionStateFor(type_url); + sub.updateSubscriptionInterest(for_update, {}); + // Tell the server about our change in interest, if any. + if (sub.subscriptionUpdatePending()) { + trySendDiscoveryRequests(); + } +} + +GrpcMuxSotw::GrpcMuxSotw(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, + const Protobuf::MethodDescriptor& service_method, + envoy::config::core::v3::ApiVersion transport_api_version, + Random::RandomGenerator& random, Stats::Scope& scope, + const RateLimitSettings& rate_limit_settings, + const LocalInfo::LocalInfo& local_info, bool skip_subsequent_node) + : GrpcMuxImpl(std::make_unique(dispatcher), skip_subsequent_node, + local_info, transport_api_version), + grpc_stream_(this, std::move(async_client), service_method, random, dispatcher, scope, + rate_limit_settings) {} + +// GrpcStreamCallbacks for GrpcMuxSotw +void GrpcMuxSotw::onStreamEstablished() { handleEstablishedStream(); } +void GrpcMuxSotw::onEstablishmentFailure() { handleStreamEstablishmentFailure(); } +void GrpcMuxSotw::onWriteable() { trySendDiscoveryRequests(); } +void GrpcMuxSotw::onDiscoveryResponse( + std::unique_ptr&& message, + ControlPlaneStats& control_plane_stats) { + if (message->has_control_plane()) { + control_plane_stats.identifier_.set(message->control_plane().identifier()); + } + genericHandleResponse(message->type_url(), *message); +} + +void GrpcMuxSotw::establishGrpcStream() { grpc_stream_.establishNewStream(); } + +void GrpcMuxSotw::sendGrpcMessage(envoy::service::discovery::v3::DiscoveryRequest& msg_proto, SotwSubscriptionState& sub_state) { + if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || + !skipSubsequentNode()) { + msg_proto.mutable_node()->MergeFrom(localInfo().node()); + } + VersionConverter::prepareMessageForGrpcWire(msg_proto, transportApiVersion()); + grpc_stream_.sendMessage(msg_proto); + setAnyRequestSentYetInCurrentStream(true); + sub_state.clearDynamicContextChanged(); +} + +void GrpcMuxSotw::maybeUpdateQueueSizeStat(uint64_t size) { + grpc_stream_.maybeUpdateQueueSizeStat(size); +} + +bool GrpcMuxSotw::grpcStreamAvailable() const { return grpc_stream_.grpcStreamAvailable(); } +bool GrpcMuxSotw::rateLimitAllowsDrain() { return grpc_stream_.checkRateLimitAllowsDrain(); } + +Watch* NullGrpcMuxImpl::addWatch(const std::string&, const absl::flat_hash_set&, + SubscriptionCallbacks&, OpaqueResourceDecoder&, + std::chrono::milliseconds, const bool) { + throw EnvoyException("ADS must be configured to support an ADS config source"); +} + +void NullGrpcMuxImpl::updateWatch(const std::string&, Watch*, + const absl::flat_hash_set&, const bool) { + throw EnvoyException("ADS must be configured to support an ADS config source"); +} + +void NullGrpcMuxImpl::removeWatch(const std::string&, Watch*) { + throw EnvoyException("ADS must be configured to support an ADS config source"); +} + +} // namespace UnifiedMux +} // namespace Config +} // namespace Envoy \ No newline at end of file diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h new file mode 100644 index 000000000000..bc0c8ccd73c3 --- /dev/null +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -0,0 +1,263 @@ +#pragma once + +#include +#include +#include + +#include "envoy/common/random_generator.h" +#include "envoy/common/time.h" +#include "envoy/common/token_bucket.h" +#include "envoy/config/grpc_mux.h" +#include "envoy/config/subscription.h" +#include "envoy/event/dispatcher.h" +#include "envoy/grpc/status.h" +#include "envoy/service/discovery/v3/discovery.pb.h" +#include "envoy/upstream/cluster_manager.h" + +#include "source/common/common/logger.h" +#include "source/common/common/utility.h" +#include "source/common/config/api_version.h" +#include "source/common/config/grpc_stream.h" +#include "source/common/config/pausable_ack_queue.h" +#include "source/common/config/unified_mux/delta_subscription_state.h" +#include "source/common/config/unified_mux/sotw_subscription_state.h" +#include "source/common/config/watch_map.h" +#include "source/common/grpc/common.h" + +#include "absl/container/node_hash_map.h" + +namespace Envoy { +namespace Config { +namespace UnifiedMux { + +// Manages subscriptions to one or more type of resource. The logical protocol +// state of those subscription(s) is handled by SubscriptionState. +// This class owns the GrpcStream used to talk to the server, maintains queuing +// logic to properly order the subscription(s)' various messages, and allows +// starting/stopping/pausing of the subscriptions. +template +class GrpcMuxImpl : public GrpcMux, Logger::Loggable { +public: + GrpcMuxImpl(std::unique_ptr subscription_state_factory, + bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, + envoy::config::core::v3::ApiVersion transport_api_version); + + Watch* addWatch(const std::string& type_url, const absl::flat_hash_set& resources, + SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, + std::chrono::milliseconds init_fetch_timeout, + const bool use_namespace_matching = false) override; + void updateWatch(const std::string& type_url, Watch* watch, + const absl::flat_hash_set& resources, + const bool creating_namespace_watch = false) override; + void removeWatch(const std::string& type_url, Watch* watch) override; + + ScopedResume pause(const std::string& type_url) override; + ScopedResume pause(const std::vector type_urls) override; + bool paused(const std::string& type_url) const override; + void start() override; + void disableInitFetchTimeoutTimer() override; + const absl::flat_hash_map>& + subscriptions() const { + return subscriptions_; + } + + // legacy mux interface not implemented by unified mux. + GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, + SubscriptionCallbacks&, OpaqueResourceDecoder&, const SubscriptionOptions&) override { + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + } + + void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + } + + bool isUnified() const override { return true; } + +protected: + // Everything related to GrpcStream must remain abstract. GrpcStream (and the gRPC-using classes + // that underlie it) are templated on protobufs. That means that a single implementation that + // supports different types of protobufs cannot use polymorphism to share code. The workaround: + // the GrpcStream will be owned by a derived class, and all code that would touch grpc_stream_ is + // seen here in the base class as calls to abstract functions, to be provided by those derived + // classes. + virtual void establishGrpcStream() PURE; + // Deletes msg_proto_ptr. + virtual void sendGrpcMessage(RQ& msg_proto, S& sub_state) PURE; + virtual void maybeUpdateQueueSizeStat(uint64_t size) PURE; + virtual bool grpcStreamAvailable() const PURE; + virtual bool rateLimitAllowsDrain() PURE; + + S& subscriptionStateFor(const std::string& type_url); + WatchMap& watchMapFor(const std::string& type_url); + void handleEstablishedStream(); + void handleStreamEstablishmentFailure(); + void genericHandleResponse(const std::string& type_url, const RS& response_proto); + void trySendDiscoveryRequests(); + bool skipSubsequentNode() const { return skip_subsequent_node_; } + bool anyRequestSentYetInCurrentStream() const { return any_request_sent_yet_in_current_stream_; } + void setAnyRequestSentYetInCurrentStream(bool value) { + any_request_sent_yet_in_current_stream_ = value; + } + const LocalInfo::LocalInfo& localInfo() const { return local_info_; } + const envoy::config::core::v3::ApiVersion& transportApiVersion() const { + return transport_api_version_; + } + +private: + // Checks whether external conditions allow sending a DeltaDiscoveryRequest. (Does not check + // whether we *want* to send a DeltaDiscoveryRequest). + bool canSendDiscoveryRequest(const std::string& type_url); + + // Checks whether we have something to say in a DeltaDiscoveryRequest, which can be an ACK and/or + // a subscription update. (Does not check whether we *can* send that DeltaDiscoveryRequest). + // Returns the type_url we should send the DeltaDiscoveryRequest for (if any). + // First, prioritizes ACKs over non-ACK subscription interest updates. + // Then, prioritizes non-ACK updates in the order the various types + // of subscriptions were activated (as tracked by subscription_ordering_). + absl::optional whoWantsToSendDiscoveryRequest(); + + // Invoked when dynamic context parameters change for a resource type. + void onDynamicContextUpdate(absl::string_view resource_type_url); + + // Resource (N)ACKs we're waiting to send, stored in the order that they should be sent in. All + // of our different resource types' ACKs are mixed together in this queue. See class for + // description of how it interacts with pause() and resume(). + PausableAckQueue pausable_ack_queue_; + + // Makes SubscriptionStates, to be held in the subscriptions_ map. Whether this GrpcMux is doing + // delta or state of the world xDS is determined by which concrete subclass this variable gets. + std::unique_ptr subscription_state_factory_; + + // Map key is type_url. + // Only addWatch() should insert into these maps. + absl::flat_hash_map> subscriptions_; + absl::flat_hash_map> watch_maps_; + + // Determines the order of initial discovery requests. (Assumes that subscriptions are added + // to this GrpcMux in the order of Envoy's dependency ordering). + std::list subscription_ordering_; + + // Whether to enable the optimization of only including the node field in the very first + // discovery request in an xDS gRPC stream (really just one: *not* per-type_url). + const bool skip_subsequent_node_; + + // State to help with skip_subsequent_node's logic. + bool any_request_sent_yet_in_current_stream_{}; + + // Used to populate the [Delta]DiscoveryRequest's node field. That field is the same across + // all type_urls, and moreover, the 'skip_subsequent_node' logic needs to operate across all + // the type_urls. So, while the SubscriptionStates populate every other field of these messages, + // this one is up to GrpcMux. + const LocalInfo::LocalInfo& local_info_; + Common::CallbackHandlePtr dynamic_update_callback_handle_; + + const envoy::config::core::v3::ApiVersion transport_api_version_; +}; + +class GrpcMuxDelta + : public GrpcMuxImpl, + public GrpcStreamCallbacks { +public: + GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, + const Protobuf::MethodDescriptor& service_method, + envoy::config::core::v3::ApiVersion transport_api_version, + Random::RandomGenerator& random, Stats::Scope& scope, + const RateLimitSettings& rate_limit_settings, const LocalInfo::LocalInfo& local_info, + bool skip_subsequent_node); + + // GrpcStreamCallbacks + void onStreamEstablished() override; + void onEstablishmentFailure() override; + void onWriteable() override; + void onDiscoveryResponse( + std::unique_ptr&& message, + ControlPlaneStats& control_plane_stats) override; + void requestOnDemandUpdate(const std::string& type_url, + const absl::flat_hash_set& for_update) override; + +protected: + void establishGrpcStream() override; + void sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, DeltaSubscriptionState& sub_state) override; + void maybeUpdateQueueSizeStat(uint64_t size) override; + bool grpcStreamAvailable() const override; + bool rateLimitAllowsDrain() override; + +private: + GrpcStream + grpc_stream_; +}; + +class GrpcMuxSotw : public GrpcMuxImpl, + public GrpcStreamCallbacks { +public: + GrpcMuxSotw(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, + const Protobuf::MethodDescriptor& service_method, + envoy::config::core::v3::ApiVersion transport_api_version, + Random::RandomGenerator& random, Stats::Scope& scope, + const RateLimitSettings& rate_limit_settings, const LocalInfo::LocalInfo& local_info, + bool skip_subsequent_node); + + // GrpcStreamCallbacks + void onStreamEstablished() override; + void onEstablishmentFailure() override; + void onWriteable() override; + void + onDiscoveryResponse(std::unique_ptr&& message, + ControlPlaneStats& control_plane_stats) override; + void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + }; + GrpcStream& + grpcStreamForTest() { + return grpc_stream_; + } + +protected: + void establishGrpcStream() override; + void sendGrpcMessage(envoy::service::discovery::v3::DiscoveryRequest& msg_proto, SotwSubscriptionState& sub_state) override; + void maybeUpdateQueueSizeStat(uint64_t size) override; + bool grpcStreamAvailable() const override; + bool rateLimitAllowsDrain() override; + +private: + GrpcStream + grpc_stream_; +}; + +class NullGrpcMuxImpl : public GrpcMux { +public: + void start() override {} + + ScopedResume pause(const std::string&) override { + return std::make_unique([]() {}); + } + ScopedResume pause(const std::vector) override { + return std::make_unique([]() {}); + } + bool paused(const std::string&) const override { return false; } + void disableInitFetchTimeoutTimer() override {} + + Watch* addWatch(const std::string&, const absl::flat_hash_set&, + SubscriptionCallbacks&, OpaqueResourceDecoder&, std::chrono::milliseconds, + const bool) override; + void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, + const bool) override; + void removeWatch(const std::string&, Watch*) override; + + // legacy mux interface not implemented by unified mux. + GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, + SubscriptionCallbacks&, OpaqueResourceDecoder&, const SubscriptionOptions&) override { + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + } + + void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + } +}; + +} // namespace UnifiedMux +} // namespace Config +} // namespace Envoy diff --git a/source/common/config/xds_mux/sotw_subscription_state.cc b/source/common/config/xds_mux/sotw_subscription_state.cc index bb7a9f4c3a9c..90be3a318a2a 100644 --- a/source/common/config/xds_mux/sotw_subscription_state.cc +++ b/source/common/config/xds_mux/sotw_subscription_state.cc @@ -70,7 +70,7 @@ void SotwSubscriptionState::handleGoodResponse( } // TODO (dmitri-d) to eliminate decoding of resources twice consider expanding the interface to - // support passing of decoded resources + // support passing of decoded resources. This would also avoid a resource copy above. callbacks().onConfigUpdate(non_heartbeat_resources, message.version_info()); // Now that we're passed onConfigUpdate() without an exception thrown, we know we're good. last_good_version_info_ = message.version_info(); diff --git a/source/common/config/xds_mux/sotw_subscription_state.h b/source/common/config/xds_mux/sotw_subscription_state.h index 4d191fb93a3c..663cb466f7f8 100644 --- a/source/common/config/xds_mux/sotw_subscription_state.h +++ b/source/common/config/xds_mux/sotw_subscription_state.h @@ -62,6 +62,21 @@ class SotwSubscriptionState absl::flat_hash_set names_tracked_; }; +class SotwSubscriptionStateFactory : public SubscriptionStateFactory { +public: + SotwSubscriptionStateFactory(Event::Dispatcher& dispatcher) + : dispatcher_(dispatcher) {} + ~SotwSubscriptionStateFactory() override = default; + std::unique_ptr makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, + std::chrono::milliseconds init_fetch_timeout, OpaqueResourceDecoder& resource_decoder, const bool) override { + return std::make_unique(type_url, callbacks, init_fetch_timeout, + dispatcher_, resource_decoder); + + } +private: + Event::Dispatcher& dispatcher_; +}; + } // namespace XdsMux } // namespace Config } // namespace Envoy diff --git a/source/common/config/xds_mux/subscription_state.h b/source/common/config/xds_mux/subscription_state.h index 660798974576..269d4c2e1304 100644 --- a/source/common/config/xds_mux/subscription_state.h +++ b/source/common/config/xds_mux/subscription_state.h @@ -113,6 +113,14 @@ class BaseSubscriptionState : public SubscriptionState, bool dynamic_context_changed_{}; }; +template class SubscriptionStateFactory { +public: + virtual ~SubscriptionStateFactory() = default; + // Note that, outside of tests, we expect callbacks to always be a WatchMap. + virtual std::unique_ptr makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, + std::chrono::milliseconds init_fetch_timeout, OpaqueResourceDecoder& resource_decoder, const bool wildcard) PURE; +}; + } // namespace XdsMux } // namespace Config } // namespace Envoy From c43f23c1c6170df5aacce80ad51db02b56e45b83 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Fri, 9 Jul 2021 15:35:50 -0700 Subject: [PATCH 02/46] Removed init fetch timeout from the mux Signed-off-by: Dmitri Dolguikh --- envoy/config/grpc_mux.h | 6 +- .../config/xds_mux/delta_subscription_state.h | 13 ++-- source/common/config/xds_mux/grpc_mux_impl.cc | 60 +++++++++---------- source/common/config/xds_mux/grpc_mux_impl.h | 40 +++++++------ .../config/xds_mux/sotw_subscription_state.h | 14 ++--- .../config/xds_mux/subscription_state.h | 6 +- 6 files changed, 71 insertions(+), 68 deletions(-) diff --git a/envoy/config/grpc_mux.h b/envoy/config/grpc_mux.h index 491154d69f31..6a7feba7c4b4 100644 --- a/envoy/config/grpc_mux.h +++ b/envoy/config/grpc_mux.h @@ -31,8 +31,9 @@ struct ControlPlaneStats { GENERATE_TEXT_READOUT_STRUCT) }; -// TODO (dmitri-d) This forward declaration is needed in order for the unified mux to fulfill legacy mux interface. -// It should be removed together with the rest of legacy mux code when the switch to unified mux has been finalized. +// TODO (dmitri-d) This forward declaration is needed in order for the unified mux to fulfill legacy +// mux interface. It should be removed together with the rest of legacy mux code when the switch to +// unified mux has been finalized. struct Watch; /** @@ -129,7 +130,6 @@ class GrpcMux { virtual Watch* addWatch(const std::string& type_url, const absl::flat_hash_set& resources, SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, - std::chrono::milliseconds init_fetch_timeout, const bool use_namespace_matching) PURE; // Updates the list of resource names watched by the given watch. If an added name is new across diff --git a/source/common/config/xds_mux/delta_subscription_state.h b/source/common/config/xds_mux/delta_subscription_state.h index f292b1c6bf60..ced0c9fd52f0 100644 --- a/source/common/config/xds_mux/delta_subscription_state.h +++ b/source/common/config/xds_mux/delta_subscription_state.h @@ -95,15 +95,14 @@ class DeltaSubscriptionState class DeltaSubscriptionStateFactory : public SubscriptionStateFactory { public: - DeltaSubscriptionStateFactory(Event::Dispatcher& dispatcher) - : dispatcher_(dispatcher) {} + DeltaSubscriptionStateFactory(Event::Dispatcher& dispatcher) : dispatcher_(dispatcher) {} ~DeltaSubscriptionStateFactory() override = default; - std::unique_ptr makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, - std::chrono::milliseconds init_fetch_timeout, OpaqueResourceDecoder&, const bool wildcard) override { - return std::make_unique(type_url, callbacks, init_fetch_timeout, - dispatcher_, wildcard); - + std::unique_ptr + makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, + OpaqueResourceDecoder&, const bool wildcard) override { + return std::make_unique(type_url, callbacks, dispatcher_, wildcard); } + private: Event::Dispatcher& dispatcher_; }; diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 396015c86106..02873ba2f207 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -1,4 +1,4 @@ -#include "source/common/config/unified_mux/grpc_mux_impl.h" +#include "source/common/config/xds_mux/grpc_mux_impl.h" #include "envoy/service/discovery/v3/discovery.pb.h" @@ -15,12 +15,13 @@ namespace Envoy { namespace Config { -namespace UnifiedMux { +namespace XdsMux { template GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_factory, - bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, - envoy::config::core::v3::ApiVersion transport_api_version) + bool skip_subsequent_node, + const LocalInfo::LocalInfo& local_info, + envoy::config::core::v3::ApiVersion transport_api_version) : subscription_state_factory_(std::move(subscription_state_factory)), skip_subsequent_node_(skip_subsequent_node), local_info_(local_info), dynamic_update_callback_handle_(local_info.contextProvider().addDynamicContextUpdateCallback( @@ -44,18 +45,18 @@ void GrpcMuxImpl::onDynamicContextUpdate(absl::string_view resourc template Watch* GrpcMuxImpl::addWatch(const std::string& type_url, - const absl::flat_hash_set& resources, - SubscriptionCallbacks& callbacks, - OpaqueResourceDecoder& resource_decoder, - std::chrono::milliseconds init_fetch_timeout, - const bool use_namespace_matching) { + const absl::flat_hash_set& resources, + SubscriptionCallbacks& callbacks, + OpaqueResourceDecoder& resource_decoder, + const bool use_namespace_matching) { auto watch_map = watch_maps_.find(type_url); if (watch_map == watch_maps_.end()) { // We don't yet have a subscription for type_url! Make one! watch_map = watch_maps_.emplace(type_url, std::make_unique(use_namespace_matching)).first; - subscriptions_.emplace(type_url, subscription_state_factory_->makeSubscriptionState( - type_url, *watch_maps_[type_url], init_fetch_timeout, resource_decoder, use_namespace_matching)); + subscriptions_.emplace( + type_url, subscription_state_factory_->makeSubscriptionState( + type_url, *watch_maps_[type_url], resource_decoder, use_namespace_matching)); subscription_ordering_.emplace_back(type_url); } @@ -70,8 +71,8 @@ Watch* GrpcMuxImpl::addWatch(const std::string& type_url, // subscription will enqueue and attempt to send an appropriate discovery request. template void GrpcMuxImpl::updateWatch(const std::string& type_url, Watch* watch, - const absl::flat_hash_set& resources, - const bool creating_namespace_watch) { + const absl::flat_hash_set& resources, + const bool creating_namespace_watch) { ENVOY_LOG(debug, "GrpcMuxImpl::updateWatch for {}", type_url); ASSERT(watch != nullptr); auto& sub = subscriptionStateFor(type_url); @@ -147,7 +148,7 @@ bool GrpcMuxImpl::paused(const std::string& type_url) const { template void GrpcMuxImpl::genericHandleResponse(const std::string& type_url, - const RS& response_proto) { + const RS& response_proto) { auto sub = subscriptions_.find(type_url); if (sub == subscriptions_.end()) { ENVOY_LOG(warn, @@ -161,8 +162,7 @@ void GrpcMuxImpl::genericHandleResponse(const std::string& type_ur Memory::Utils::tryShrinkHeap(); } -template -void GrpcMuxImpl::start() { +template void GrpcMuxImpl::start() { ENVOY_LOG(debug, "GrpcMuxImpl now trying to establish a stream"); establishGrpcStream(); } @@ -179,13 +179,6 @@ void GrpcMuxImpl::handleEstablishedStream() { trySendDiscoveryRequests(); } -template -void GrpcMuxImpl::disableInitFetchTimeoutTimer() { - for (auto& [type_url, subscription_state] : subscriptions_) { - subscription_state->disableInitFetchTimeoutTimer(); - } -} - template void GrpcMuxImpl::handleStreamEstablishmentFailure() { ENVOY_LOG(debug, "GrpcMuxImpl stream failed to establish"); @@ -306,8 +299,12 @@ absl::optional GrpcMuxImpl::whoWantsToSendDiscoveryRe return absl::nullopt; } -template class GrpcMuxImpl; -template class GrpcMuxImpl; +template class GrpcMuxImpl; +template class GrpcMuxImpl; // Delta- and SotW-specific concrete subclasses: GrpcMuxDelta::GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, @@ -332,10 +329,11 @@ void GrpcMuxDelta::onDiscoveryResponse( } void GrpcMuxDelta::establishGrpcStream() { grpc_stream_.establishNewStream(); } -void GrpcMuxDelta::sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, DeltaSubscriptionState& sub_state) { +void GrpcMuxDelta::sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, + DeltaSubscriptionState& sub_state) { if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || !skipSubsequentNode()) { - msg_proto.mutable_node()->MergeFrom(localInfo().node()); + msg_proto.mutable_node()->CopyFrom(localInfo().node()); } VersionConverter::prepareMessageForGrpcWire(msg_proto, transportApiVersion()); grpc_stream_.sendMessage(msg_proto); @@ -384,7 +382,8 @@ void GrpcMuxSotw::onDiscoveryResponse( void GrpcMuxSotw::establishGrpcStream() { grpc_stream_.establishNewStream(); } -void GrpcMuxSotw::sendGrpcMessage(envoy::service::discovery::v3::DiscoveryRequest& msg_proto, SotwSubscriptionState& sub_state) { +void GrpcMuxSotw::sendGrpcMessage(envoy::service::discovery::v3::DiscoveryRequest& msg_proto, + SotwSubscriptionState& sub_state) { if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || !skipSubsequentNode()) { msg_proto.mutable_node()->MergeFrom(localInfo().node()); @@ -403,8 +402,7 @@ bool GrpcMuxSotw::grpcStreamAvailable() const { return grpc_stream_.grpcStreamAv bool GrpcMuxSotw::rateLimitAllowsDrain() { return grpc_stream_.checkRateLimitAllowsDrain(); } Watch* NullGrpcMuxImpl::addWatch(const std::string&, const absl::flat_hash_set&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, - std::chrono::milliseconds, const bool) { + SubscriptionCallbacks&, OpaqueResourceDecoder&, const bool) { throw EnvoyException("ADS must be configured to support an ADS config source"); } @@ -417,6 +415,6 @@ void NullGrpcMuxImpl::removeWatch(const std::string&, Watch*) { throw EnvoyException("ADS must be configured to support an ADS config source"); } -} // namespace UnifiedMux +} // namespace XdsMux } // namespace Config } // namespace Envoy \ No newline at end of file diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index bc0c8ccd73c3..4bd6fd3e7159 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -19,16 +19,16 @@ #include "source/common/config/api_version.h" #include "source/common/config/grpc_stream.h" #include "source/common/config/pausable_ack_queue.h" -#include "source/common/config/unified_mux/delta_subscription_state.h" -#include "source/common/config/unified_mux/sotw_subscription_state.h" #include "source/common/config/watch_map.h" +#include "source/common/config/xds_mux/delta_subscription_state.h" +#include "source/common/config/xds_mux/sotw_subscription_state.h" #include "source/common/grpc/common.h" #include "absl/container/node_hash_map.h" namespace Envoy { namespace Config { -namespace UnifiedMux { +namespace XdsMux { // Manages subscriptions to one or more type of resource. The logical protocol // state of those subscription(s) is handled by SubscriptionState. @@ -38,13 +38,12 @@ namespace UnifiedMux { template class GrpcMuxImpl : public GrpcMux, Logger::Loggable { public: - GrpcMuxImpl(std::unique_ptr subscription_state_factory, - bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, + GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, + const LocalInfo::LocalInfo& local_info, envoy::config::core::v3::ApiVersion transport_api_version); Watch* addWatch(const std::string& type_url, const absl::flat_hash_set& resources, SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, - std::chrono::milliseconds init_fetch_timeout, const bool use_namespace_matching = false) override; void updateWatch(const std::string& type_url, Watch* watch, const absl::flat_hash_set& resources, @@ -55,15 +54,14 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { ScopedResume pause(const std::vector type_urls) override; bool paused(const std::string& type_url) const override; void start() override; - void disableInitFetchTimeoutTimer() override; - const absl::flat_hash_map>& - subscriptions() const { + const absl::flat_hash_map>& subscriptions() const { return subscriptions_; } // legacy mux interface not implemented by unified mux. GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, const SubscriptionOptions&) override { + SubscriptionCallbacks&, OpaqueResourceDecoder&, + const SubscriptionOptions&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } @@ -155,7 +153,9 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { }; class GrpcMuxDelta - : public GrpcMuxImpl, + : public GrpcMuxImpl, public GrpcStreamCallbacks { public: GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, @@ -177,7 +177,8 @@ class GrpcMuxDelta protected: void establishGrpcStream() override; - void sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, DeltaSubscriptionState& sub_state) override; + void sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, + DeltaSubscriptionState& sub_state) override; void maybeUpdateQueueSizeStat(uint64_t size) override; bool grpcStreamAvailable() const override; bool rateLimitAllowsDrain() override; @@ -188,7 +189,9 @@ class GrpcMuxDelta grpc_stream_; }; -class GrpcMuxSotw : public GrpcMuxImpl, +class GrpcMuxSotw : public GrpcMuxImpl, public GrpcStreamCallbacks { public: GrpcMuxSotw(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, @@ -216,7 +219,8 @@ class GrpcMuxSotw : public GrpcMuxImpl&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, std::chrono::milliseconds, - const bool) override; + SubscriptionCallbacks&, OpaqueResourceDecoder&, const bool) override; void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, const bool) override; void removeWatch(const std::string&, Watch*) override; // legacy mux interface not implemented by unified mux. GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, const SubscriptionOptions&) override { + SubscriptionCallbacks&, OpaqueResourceDecoder&, + const SubscriptionOptions&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } @@ -258,6 +262,6 @@ class NullGrpcMuxImpl : public GrpcMux { } }; -} // namespace UnifiedMux +} // namespace XdsMux } // namespace Config } // namespace Envoy diff --git a/source/common/config/xds_mux/sotw_subscription_state.h b/source/common/config/xds_mux/sotw_subscription_state.h index 663cb466f7f8..86063198f5a7 100644 --- a/source/common/config/xds_mux/sotw_subscription_state.h +++ b/source/common/config/xds_mux/sotw_subscription_state.h @@ -64,15 +64,15 @@ class SotwSubscriptionState class SotwSubscriptionStateFactory : public SubscriptionStateFactory { public: - SotwSubscriptionStateFactory(Event::Dispatcher& dispatcher) - : dispatcher_(dispatcher) {} + SotwSubscriptionStateFactory(Event::Dispatcher& dispatcher) : dispatcher_(dispatcher) {} ~SotwSubscriptionStateFactory() override = default; - std::unique_ptr makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, - std::chrono::milliseconds init_fetch_timeout, OpaqueResourceDecoder& resource_decoder, const bool) override { - return std::make_unique(type_url, callbacks, init_fetch_timeout, - dispatcher_, resource_decoder); - + std::unique_ptr + makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, + OpaqueResourceDecoder& resource_decoder, const bool) override { + return std::make_unique(type_url, callbacks, dispatcher_, + resource_decoder); } + private: Event::Dispatcher& dispatcher_; }; diff --git a/source/common/config/xds_mux/subscription_state.h b/source/common/config/xds_mux/subscription_state.h index 269d4c2e1304..034578eee3f8 100644 --- a/source/common/config/xds_mux/subscription_state.h +++ b/source/common/config/xds_mux/subscription_state.h @@ -117,8 +117,10 @@ template class SubscriptionStateFactory { public: virtual ~SubscriptionStateFactory() = default; // Note that, outside of tests, we expect callbacks to always be a WatchMap. - virtual std::unique_ptr makeSubscriptionState(const std::string& type_url, UntypedConfigUpdateCallbacks& callbacks, - std::chrono::milliseconds init_fetch_timeout, OpaqueResourceDecoder& resource_decoder, const bool wildcard) PURE; + virtual std::unique_ptr makeSubscriptionState(const std::string& type_url, + UntypedConfigUpdateCallbacks& callbacks, + OpaqueResourceDecoder& resource_decoder, + const bool wildcard) PURE; }; } // namespace XdsMux From 3308ced61fa9c91a285ed0fc7f8a6cfcbed2491e Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 13 Jul 2021 17:19:00 -0700 Subject: [PATCH 03/46] delta mux tests are passing now Signed-off-by: Dmitri Dolguikh --- envoy/config/grpc_mux.h | 29 +--- source/common/config/grpc_mux_impl.h | 8 ++ source/common/config/new_grpc_mux_impl.h | 8 +- source/common/config/xds_mux/grpc_mux_impl.cc | 91 ++++++------ source/common/config/xds_mux/grpc_mux_impl.h | 50 ++++--- .../config/xds_mux/subscription_state.h | 4 + test/common/config/BUILD | 1 + test/common/config/new_grpc_mux_impl_test.cc | 130 +++++++++++++----- 8 files changed, 197 insertions(+), 124 deletions(-) diff --git a/envoy/config/grpc_mux.h b/envoy/config/grpc_mux.h index 6a7feba7c4b4..c4fe26a6454e 100644 --- a/envoy/config/grpc_mux.h +++ b/envoy/config/grpc_mux.h @@ -112,34 +112,13 @@ class GrpcMux { const absl::flat_hash_set& for_update) PURE; // Unified mux interface starts here - /** - * Start a configuration subscription asynchronously for some API type and resources. - * @param type_url type URL corresponding to xDS API, e.g. - * type.googleapis.com/envoy.api.v2.Cluster. - * @param resources set of resource names to watch for. If this is empty, then all - * resources for type_url will result in callbacks. - * @param callbacks the callbacks to be notified of configuration updates. These must be valid - * until GrpcMuxWatch is destroyed. - * @param resource_decoder how incoming opaque resource objects are to be decoded. - * @param use_namespace_matching if namespace watch should be created. This is used for creating - * watches on collections of resources; individual members of a collection are identified by the - * namespace in resource name. - * @return Watch* an opaque watch token added or updated, to be used in future addOrUpdateWatch - * calls. - */ - virtual Watch* addWatch(const std::string& type_url, - const absl::flat_hash_set& resources, - SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, - const bool use_namespace_matching) PURE; - // Updates the list of resource names watched by the given watch. If an added name is new across // the whole subscription, or if a removed name has no other watch interested in it, then the // subscription will enqueue and attempt to send an appropriate discovery request. virtual void updateWatch(const std::string& type_url, Watch* watch, const absl::flat_hash_set& resources, - const bool creating_namespace_watch) PURE; + const SubscriptionOptions& options) PURE; - // Unified mux interface starts here /** * Cleanup of a Watch* added by addOrUpdateWatch(). Receiving a Watch* from addOrUpdateWatch() * makes you responsible for eventually invoking this cleanup. @@ -156,12 +135,6 @@ class GrpcMux { */ virtual bool paused(const std::string& type_url) const PURE; - /** - * Passes through to all multiplexed SubscriptionStates. To be called when something - * definitive happens with the initial fetch: either an update is successfully received, - * or some sort of error happened.*/ - virtual void disableInitFetchTimeoutTimer() PURE; - virtual bool isUnified() const { return false; } }; diff --git a/source/common/config/grpc_mux_impl.h b/source/common/config/grpc_mux_impl.h index 585d028fe2b6..c02b0c6a7134 100644 --- a/source/common/config/grpc_mux_impl.h +++ b/source/common/config/grpc_mux_impl.h @@ -77,6 +77,14 @@ class GrpcMuxImpl : public GrpcMux, void setRetryTimer(); void sendDiscoveryRequest(const std::string& type_url); + bool paused(const std::string&) const override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } + + void removeWatch(const std::string&, Watch*) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } + void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, + const SubscriptionOptions&) override { + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + } + struct GrpcMuxWatchImpl : public GrpcMuxWatch { GrpcMuxWatchImpl(const absl::flat_hash_set& resources, SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, diff --git a/source/common/config/new_grpc_mux_impl.h b/source/common/config/new_grpc_mux_impl.h index 4c2246fed813..46880b8eb2fe 100644 --- a/source/common/config/new_grpc_mux_impl.h +++ b/source/common/config/new_grpc_mux_impl.h @@ -65,6 +65,10 @@ class NewGrpcMuxImpl // TODO(fredlas) remove this from the GrpcMux interface. void start() override; + bool paused(const std::string& type_url) const override { + return !pausable_ack_queue_.paused(type_url); + } + GrpcStream& grpcStreamForTest() { @@ -120,14 +124,14 @@ class NewGrpcMuxImpl const SubscriptionOptions options_; }; - void removeWatch(const std::string& type_url, Watch* watch); + void removeWatch(const std::string& type_url, Watch* watch) override; // Updates the list of resource names watched by the given watch. If an added name is new across // the whole subscription, or if a removed name has no other watch interested in it, then the // subscription will enqueue and attempt to send an appropriate discovery request. void updateWatch(const std::string& type_url, Watch* watch, const absl::flat_hash_set& resources, - const SubscriptionOptions& options); + const SubscriptionOptions& options) override; // Adds a subscription for the type_url to the subscriptions map and order list. void addSubscription(const std::string& type_url, bool use_namespace_matching, diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 02873ba2f207..a26b09e0a77c 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -44,26 +44,28 @@ void GrpcMuxImpl::onDynamicContextUpdate(absl::string_view resourc } template -Watch* GrpcMuxImpl::addWatch(const std::string& type_url, - const absl::flat_hash_set& resources, - SubscriptionCallbacks& callbacks, - OpaqueResourceDecoder& resource_decoder, - const bool use_namespace_matching) { +Config::GrpcMuxWatchPtr GrpcMuxImpl::addWatch( + const std::string& type_url, const absl::flat_hash_set& resources, + SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, + const SubscriptionOptions& options) { auto watch_map = watch_maps_.find(type_url); if (watch_map == watch_maps_.end()) { // We don't yet have a subscription for type_url! Make one! watch_map = - watch_maps_.emplace(type_url, std::make_unique(use_namespace_matching)).first; + watch_maps_.emplace(type_url, std::make_unique(options.use_namespace_matching_)) + .first; subscriptions_.emplace( type_url, subscription_state_factory_->makeSubscriptionState( - type_url, *watch_maps_[type_url], resource_decoder, use_namespace_matching)); + type_url, *watch_maps_[type_url], resource_decoder, resources.empty())); subscription_ordering_.emplace_back(type_url); } Watch* watch = watch_map->second->addWatch(callbacks, resource_decoder); // updateWatch() queues a discovery request if any of 'resources' are not yet subscribed. - updateWatch(type_url, watch, resources, use_namespace_matching); - return watch; + updateWatch(type_url, watch, resources, options); + // TODO (dmitri-d) return naked pointer instead once legacy mux has been removed and mux interface + // can be updated + return std::make_unique(watch); } // Updates the list of resource names watched by the given watch. If an added name is new across @@ -72,38 +74,33 @@ Watch* GrpcMuxImpl::addWatch(const std::string& type_url, template void GrpcMuxImpl::updateWatch(const std::string& type_url, Watch* watch, const absl::flat_hash_set& resources, - const bool creating_namespace_watch) { + const SubscriptionOptions& options) { ENVOY_LOG(debug, "GrpcMuxImpl::updateWatch for {}", type_url); ASSERT(watch != nullptr); auto& sub = subscriptionStateFor(type_url); WatchMap& watch_map = watchMapFor(type_url); - // If this is a glob collection subscription, we need to compute actual context parameters. - absl::flat_hash_set xdstp_resources; - // TODO(htuch): add support for resources beyond glob collections, the constraints below around - // resource size and ID reflect the progress of the xdstp:// implementation. - if (!resources.empty() && XdsResourceIdentifier::hasXdsTpScheme(*resources.begin())) { - // Callers must be asking for a single resource, the collection. - ASSERT(resources.size() == 1); - auto resource = XdsResourceIdentifier::decodeUrn(*resources.begin()); - // We only know how to deal with glob collections and static context parameters right now. - // TODO(htuch): add support for dynamic context params and list collections in the future. - if (absl::EndsWith(resource.id(), "/*")) { - auto encoded_context = XdsContextParams::encodeResource( - local_info_.contextProvider().nodeContext(), resource.context(), {}, {}); - resource.mutable_context()->CopyFrom(encoded_context); + // We need to prepare xdstp:// resources for the transport, by normalizing and adding any extra + // context parameters. + absl::flat_hash_set effective_resources; + for (const auto& resource : resources) { + if (XdsResourceIdentifier::hasXdsTpScheme(resource)) { + auto xdstp_resource = XdsResourceIdentifier::decodeUrn(resource); + if (options.add_xdstp_node_context_params_) { + const auto context = XdsContextParams::encodeResource( + local_info_.contextProvider().nodeContext(), xdstp_resource.context(), {}, {}); + xdstp_resource.mutable_context()->CopyFrom(context); + } XdsResourceIdentifier::EncodeOptions encode_options; encode_options.sort_context_params_ = true; - xdstp_resources.insert(XdsResourceIdentifier::encodeUrn(resource, encode_options)); + effective_resources.insert(XdsResourceIdentifier::encodeUrn(xdstp_resource, encode_options)); } else { - // TODO(htuch): We will handle list collections here in future work. - NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + effective_resources.insert(resource); } } - auto added_removed = - watch_map.updateWatchInterest(watch, xdstp_resources.empty() ? resources : xdstp_resources); - if (creating_namespace_watch && xdstp_resources.empty()) { + auto added_removed = watch_map.updateWatchInterest(watch, effective_resources); + if (options.use_namespace_matching_) { // This is to prevent sending out of requests that contain prefixes instead of resource names sub.updateSubscriptionInterest({}, {}); } else { @@ -118,7 +115,7 @@ void GrpcMuxImpl::updateWatch(const std::string& type_url, Watch* template void GrpcMuxImpl::removeWatch(const std::string& type_url, Watch* watch) { - updateWatch(type_url, watch, {}); + updateWatch(type_url, watch, {}, {}); watchMapFor(type_url).removeWatch(watch); } @@ -148,7 +145,8 @@ bool GrpcMuxImpl::paused(const std::string& type_url) const { template void GrpcMuxImpl::genericHandleResponse(const std::string& type_url, - const RS& response_proto) { + const RS& response_proto, + ControlPlaneStats& control_plane_stats) { auto sub = subscriptions_.find(type_url); if (sub == subscriptions_.end()) { ENVOY_LOG(warn, @@ -157,6 +155,17 @@ void GrpcMuxImpl::genericHandleResponse(const std::string& type_ur type_url); return; } + + if (response_proto.has_control_plane()) { + control_plane_stats.identifier_.set(response_proto.control_plane().identifier()); + } + + if (response_proto.control_plane().identifier() != sub->second->controlPlaneIdentifier()) { + sub->second->setControlPlaneIdentifier(response_proto.control_plane().identifier()); + ENVOY_LOG(debug, "Receiving gRPC updates for {} from {}", response_proto.type_url(), + sub->second->controlPlaneIdentifier()); + } + pausable_ack_queue_.push(sub->second->handleResponse(response_proto)); trySendDiscoveryRequests(); Memory::Utils::tryShrinkHeap(); @@ -324,8 +333,8 @@ void GrpcMuxDelta::onEstablishmentFailure() { handleStreamEstablishmentFailure() void GrpcMuxDelta::onWriteable() { trySendDiscoveryRequests(); } void GrpcMuxDelta::onDiscoveryResponse( std::unique_ptr&& message, - ControlPlaneStats&) { - genericHandleResponse(message->type_url(), *message); + ControlPlaneStats& control_plane_stats) { + genericHandleResponse(message->type_url(), *message, control_plane_stats); } void GrpcMuxDelta::establishGrpcStream() { grpc_stream_.establishNewStream(); } @@ -374,10 +383,7 @@ void GrpcMuxSotw::onWriteable() { trySendDiscoveryRequests(); } void GrpcMuxSotw::onDiscoveryResponse( std::unique_ptr&& message, ControlPlaneStats& control_plane_stats) { - if (message->has_control_plane()) { - control_plane_stats.identifier_.set(message->control_plane().identifier()); - } - genericHandleResponse(message->type_url(), *message); + genericHandleResponse(message->type_url(), *message, control_plane_stats); } void GrpcMuxSotw::establishGrpcStream() { grpc_stream_.establishNewStream(); } @@ -401,13 +407,16 @@ void GrpcMuxSotw::maybeUpdateQueueSizeStat(uint64_t size) { bool GrpcMuxSotw::grpcStreamAvailable() const { return grpc_stream_.grpcStreamAvailable(); } bool GrpcMuxSotw::rateLimitAllowsDrain() { return grpc_stream_.checkRateLimitAllowsDrain(); } -Watch* NullGrpcMuxImpl::addWatch(const std::string&, const absl::flat_hash_set&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, const bool) { +Config::GrpcMuxWatchPtr NullGrpcMuxImpl::addWatch(const std::string&, + const absl::flat_hash_set&, + SubscriptionCallbacks&, OpaqueResourceDecoder&, + const SubscriptionOptions&) { throw EnvoyException("ADS must be configured to support an ADS config source"); } void NullGrpcMuxImpl::updateWatch(const std::string&, Watch*, - const absl::flat_hash_set&, const bool) { + const absl::flat_hash_set&, + const SubscriptionOptions&) { throw EnvoyException("ADS must be configured to support an ADS config source"); } diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 4bd6fd3e7159..ed35bd5dc6f2 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -30,6 +30,14 @@ namespace Envoy { namespace Config { namespace XdsMux { +class WatchCompatibilityWrapper : public Envoy::Config::GrpcMuxWatch { +public: + WatchCompatibilityWrapper(Watch* watch) : watch_(watch) {} + void update(const absl::flat_hash_set&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } + + Watch* watch_; +}; + // Manages subscriptions to one or more type of resource. The logical protocol // state of those subscription(s) is handled by SubscriptionState. // This class owns the GrpcStream used to talk to the server, maintains queuing @@ -42,12 +50,16 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { const LocalInfo::LocalInfo& local_info, envoy::config::core::v3::ApiVersion transport_api_version); - Watch* addWatch(const std::string& type_url, const absl::flat_hash_set& resources, - SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, - const bool use_namespace_matching = false) override; + // TODO (dmitri-d) return a naked pointer instead of the wrapper once the legacy mux has been + // removed and the mux interface can be changed + Config::GrpcMuxWatchPtr addWatch(const std::string& type_url, + const absl::flat_hash_set& resources, + SubscriptionCallbacks& callbacks, + OpaqueResourceDecoder& resource_decoder, + const SubscriptionOptions& options) override; void updateWatch(const std::string& type_url, Watch* watch, const absl::flat_hash_set& resources, - const bool creating_namespace_watch = false) override; + const SubscriptionOptions& options) override; void removeWatch(const std::string& type_url, Watch* watch) override; ScopedResume pause(const std::string& type_url) override; @@ -58,13 +70,6 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { return subscriptions_; } - // legacy mux interface not implemented by unified mux. - GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, - const SubscriptionOptions&) override { - NOT_IMPLEMENTED_GCOVR_EXCL_LINE; - } - void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } @@ -89,7 +94,8 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { WatchMap& watchMapFor(const std::string& type_url); void handleEstablishedStream(); void handleStreamEstablishmentFailure(); - void genericHandleResponse(const std::string& type_url, const RS& response_proto); + void genericHandleResponse(const std::string& type_url, const RS& response_proto, + ControlPlaneStats& control_plane_stats); void trySendDiscoveryRequests(); bool skipSubsequentNode() const { return skip_subsequent_node_; } bool anyRequestSentYetInCurrentStream() const { return any_request_sent_yet_in_current_stream_; } @@ -174,6 +180,11 @@ class GrpcMuxDelta ControlPlaneStats& control_plane_stats) override; void requestOnDemandUpdate(const std::string& type_url, const absl::flat_hash_set& for_update) override; + GrpcStream& + grpcStreamForTest() { + return grpc_stream_; + } protected: void establishGrpcStream() override; @@ -242,21 +253,16 @@ class NullGrpcMuxImpl : public GrpcMux { return std::make_unique([]() {}); } bool paused(const std::string&) const override { return false; } - void disableInitFetchTimeoutTimer() override {} - Watch* addWatch(const std::string&, const absl::flat_hash_set&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, const bool) override; void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, - const bool) override; + const SubscriptionOptions&) override; void removeWatch(const std::string&, Watch*) override; - // legacy mux interface not implemented by unified mux. - GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, - SubscriptionCallbacks&, OpaqueResourceDecoder&, - const SubscriptionOptions&) override { - NOT_IMPLEMENTED_GCOVR_EXCL_LINE; - } + Config::GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, + SubscriptionCallbacks&, OpaqueResourceDecoder&, + const SubscriptionOptions&) override; + // legacy mux interface not implemented by unified mux. void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } diff --git a/source/common/config/xds_mux/subscription_state.h b/source/common/config/xds_mux/subscription_state.h index 034578eee3f8..9f9b48cd7723 100644 --- a/source/common/config/xds_mux/subscription_state.h +++ b/source/common/config/xds_mux/subscription_state.h @@ -47,6 +47,9 @@ class BaseSubscriptionState : public SubscriptionState, void clearDynamicContextChanged() { dynamic_context_changed_ = false; } bool dynamicContextChanged() const { return dynamic_context_changed_; } + void setControlPlaneIdentifier(const std::string& id) { control_plane_identifier_ = id; } + std::string& controlPlaneIdentifier() { return control_plane_identifier_; } + // Whether there was a change in our subscription interest we have yet to inform the server of. virtual bool subscriptionUpdatePending() const PURE; @@ -111,6 +114,7 @@ class BaseSubscriptionState : public SubscriptionState, UntypedConfigUpdateCallbacks& callbacks_; Event::Dispatcher& dispatcher_; bool dynamic_context_changed_{}; + std::string control_plane_identifier_{}; }; template class SubscriptionStateFactory { diff --git a/test/common/config/BUILD b/test/common/config/BUILD index f8cd436f4946..4e84024f2ee2 100644 --- a/test/common/config/BUILD +++ b/test/common/config/BUILD @@ -179,6 +179,7 @@ envoy_cc_test( "//source/common/config:new_grpc_mux_lib", "//source/common/config:protobuf_link_hacks", "//source/common/config:version_converter_lib", + "//source/common/config/xds_mux:grpc_mux_lib", "//source/common/protobuf", "//test/common/stats:stat_test_utility_lib", "//test/mocks:common_lib", diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index ce917b39a0fc..1929ca53bb27 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -10,6 +10,7 @@ #include "source/common/config/protobuf_link_hacks.h" #include "source/common/config/utility.h" #include "source/common/config/version_converter.h" +#include "source/common/config/xds_mux/grpc_mux_impl.h" #include "source/common/protobuf/protobuf.h" #include "test/common/stats/stat_test_utility.h" @@ -41,17 +42,29 @@ namespace Envoy { namespace Config { namespace { +enum class LegacyOrUnified { Legacy, Unified }; + // We test some mux specific stuff below, other unit test coverage for singleton use of // NewGrpcMuxImpl is provided in [grpc_]subscription_impl_test.cc. -class NewGrpcMuxImplTestBase : public testing::Test { +class NewGrpcMuxImplTestBase : public testing::TestWithParam { public: - NewGrpcMuxImplTestBase() + NewGrpcMuxImplTestBase(LegacyOrUnified legacy_or_unified) : async_client_(new Grpc::MockAsyncClient()), control_plane_stats_(Utility::generateControlPlaneStats(stats_)), control_plane_connected_state_( - stats_.gauge("control_plane.connected_state", Stats::Gauge::ImportMode::NeverImport)) {} + stats_.gauge("control_plane.connected_state", Stats::Gauge::ImportMode::NeverImport)), + should_use_unified_(legacy_or_unified == LegacyOrUnified::Unified) {} void setup() { + if (isUnifiedMuxTest()) { + grpc_mux_ = std::make_unique( + std::unique_ptr(async_client_), dispatcher_, + *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( + "envoy.service.discovery.v2.AggregatedDiscoveryService.StreamAggregatedResources"), + envoy::config::core::v3::ApiVersion::AUTO, random_, stats_, rate_limit_settings_, + local_info_, false); + return; + } grpc_mux_ = std::make_unique( std::unique_ptr(async_client_), dispatcher_, *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( @@ -88,12 +101,52 @@ class NewGrpcMuxImplTestBase : public testing::Test { EXPECT_CALL(async_stream_, sendMessageRaw_(Grpc::ProtoBufferEq(expected_request), false)); } + void remoteClose() { + if (isUnifiedMuxTest()) { + dynamic_cast(grpc_mux_.get()) + ->grpcStreamForTest() + .onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); + return; + } + dynamic_cast(grpc_mux_.get()) + ->grpcStreamForTest() + .onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); + } + + void onDiscoveryResponse( + std::unique_ptr&& response) { + if (isUnifiedMuxTest()) { + dynamic_cast(grpc_mux_.get()) + ->onDiscoveryResponse(std::move(response), control_plane_stats_); + return; + } + dynamic_cast(grpc_mux_.get()) + ->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + // the code is duplicated here, but all calls other than the check in return statement, return + // different types. + bool subscriptionExists(const std::string& type_url) const { + if (isUnifiedMuxTest()) { + auto* mux = dynamic_cast(grpc_mux_.get()); + auto& subscriptions = mux->subscriptions(); + auto sub = subscriptions.find(type_url); + return sub != subscriptions.end(); + } + auto* mux = dynamic_cast(grpc_mux_.get()); + auto& subscriptions = mux->subscriptions(); + auto sub = subscriptions.find(type_url); + return sub != subscriptions.end(); + } + + bool isUnifiedMuxTest() const { return should_use_unified_; } + NiceMock dispatcher_; NiceMock random_; Grpc::MockAsyncClient* async_client_; NiceMock async_stream_; NiceMock local_info_; - NewGrpcMuxImplPtr grpc_mux_; + std::unique_ptr grpc_mux_; NiceMock callbacks_; TestUtility::TestOpaqueResourceDecoderImpl resource_decoder_{"cluster_name"}; @@ -101,15 +154,20 @@ class NewGrpcMuxImplTestBase : public testing::Test { Envoy::Config::RateLimitSettings rate_limit_settings_; ControlPlaneStats control_plane_stats_; Stats::Gauge& control_plane_connected_state_; + bool should_use_unified_; }; class NewGrpcMuxImplTest : public NewGrpcMuxImplTestBase { public: + NewGrpcMuxImplTest() : NewGrpcMuxImplTestBase(GetParam()) {} Event::SimulatedTimeSystem time_system_; }; +INSTANTIATE_TEST_SUITE_P(NewGrpcMuxImplTest, NewGrpcMuxImplTest, + testing::ValuesIn({LegacyOrUnified::Legacy, LegacyOrUnified::Unified})); + // Validate behavior when dynamic context parameters are updated. -TEST_F(NewGrpcMuxImplTest, DynamicContextParameters) { +TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { setup(); InSequence s; auto foo_sub = grpc_mux_->addWatch("foo", {"x", "y"}, callbacks_, resource_decoder_, {}); @@ -126,11 +184,15 @@ TEST_F(NewGrpcMuxImplTest, DynamicContextParameters) { // Update to bar type should resend Node. expectSendMessage("bar", {}, {}); local_info_.context_provider_.update_cb_handler_.runCallbacks("bar"); - expectSendMessage("foo", {}, {"x", "y"}); + if (!isUnifiedMuxTest()) { + // in "legacy" delta mux implementation destruction of "foo_sub" + // results in an unsubscribe message. + expectSendMessage("foo", {}, {"x", "y"}); + } } // Validate cached nonces are cleared on reconnection. -TEST_F(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { +TEST_P(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { Event::MockTimer* grpc_stream_retry_timer{new Event::MockTimer()}; Event::MockTimer* ttl_mgr_timer{new NiceMock()}; Event::TimerCb grpc_stream_retry_timer_cb; @@ -164,7 +226,7 @@ TEST_F(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { add_response_resource("y", "3000", *response); // Pause EDS to allow the ACK to be cached. auto resume_eds = grpc_mux_->pause(type_url); - grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + onDiscoveryResponse(std::move(response)); // Now disconnect. // Grpc stream retry timer will kick in and reconnection will happen. EXPECT_CALL(*grpc_stream_retry_timer, enableTimer(_, _)) @@ -173,14 +235,17 @@ TEST_F(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { // initial_resource_versions should contain client side all resource:version info. expectSendMessage(type_url, {"x", "y"}, {}, "", Grpc::Status::WellKnownGrpcStatus::Ok, "", {{"x", "2000"}, {"y", "3000"}}); - grpc_mux_->grpcStreamForTest().onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); - // Destruction of the EDS subscription will issue an "unsubscribe" request. - expectSendMessage(type_url, {}, {"x", "y"}); + remoteClose(); + if (!isUnifiedMuxTest()) { + // in legacy mux implmentation, destruction of the EDS subscription will issue an "unsubscribe" + // request. + expectSendMessage(type_url, {}, {"x", "y"}); + } } // Validate resources are not sent on wildcard watch reconnection. // Regression test of https://github.com/envoyproxy/envoy/issues/16063. -TEST_F(NewGrpcMuxImplTest, ReconnectionResetsWildcardSubscription) { +TEST_P(NewGrpcMuxImplTest, ReconnectionResetsWildcardSubscription) { Event::MockTimer* grpc_stream_retry_timer{new Event::MockTimer()}; Event::MockTimer* ttl_mgr_timer{new NiceMock()}; Event::TimerCb grpc_stream_retry_timer_cb; @@ -231,7 +296,7 @@ TEST_F(NewGrpcMuxImplTest, ReconnectionResetsWildcardSubscription) { })); // Expect an ack with the nonce. expectSendMessage(type_url, {}, {}, "111"); - grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + onDiscoveryResponse(std::move(response)); } // Send another response with a different resource, but where EDS is paused. auto resume_eds = grpc_mux_->pause(type_url); @@ -246,7 +311,7 @@ TEST_F(NewGrpcMuxImplTest, ReconnectionResetsWildcardSubscription) { TestUtility::protoEqual(added_resources[0].get().resource(), load_assignment)); })); // No ack reply is expected in this case, as EDS is suspended. - grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + onDiscoveryResponse(std::move(response)); } // Now disconnect. @@ -258,12 +323,12 @@ TEST_F(NewGrpcMuxImplTest, ReconnectionResetsWildcardSubscription) { // added resources because this is a wildcard request. expectSendMessage(type_url, {}, {}, "", Grpc::Status::WellKnownGrpcStatus::Ok, "", {{"x", "1000"}, {"y", "2000"}}); - grpc_mux_->grpcStreamForTest().onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); + remoteClose(); // Destruction of wildcard will not issue unsubscribe requests for the resources. } // Test that we simply ignore a message for an unknown type_url, with no ill effects. -TEST_F(NewGrpcMuxImplTest, DiscoveryResponseNonexistentSub) { +TEST_P(NewGrpcMuxImplTest, DiscoveryResponseNonexistentSub) { setup(); const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; @@ -279,7 +344,7 @@ TEST_F(NewGrpcMuxImplTest, DiscoveryResponseNonexistentSub) { unexpected_response->set_system_version_info("0"); // empty response should call onConfigUpdate on wildcard watch EXPECT_CALL(callbacks_, onConfigUpdate(_, _, "0")); - grpc_mux_->onDiscoveryResponse(std::move(unexpected_response), control_plane_stats_); + onDiscoveryResponse(std::move(unexpected_response)); } { auto response = std::make_unique(); @@ -296,13 +361,13 @@ TEST_F(NewGrpcMuxImplTest, DiscoveryResponseNonexistentSub) { EXPECT_TRUE( TestUtility::protoEqual(added_resources[0].get().resource(), load_assignment)); })); - grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + onDiscoveryResponse(std::move(response)); } } // DeltaDiscoveryResponse that comes in response to an on-demand request updates the watch with // resource's name. The watch is initially created with an alias used in the on-demand request. -TEST_F(NewGrpcMuxImplTest, ConfigUpdateWithAliases) { +TEST_P(NewGrpcMuxImplTest, ConfigUpdateWithAliases) { setup(); const std::string& type_url = Config::TypeUrl::get().VirtualHost; @@ -329,20 +394,23 @@ TEST_F(NewGrpcMuxImplTest, ConfigUpdateWithAliases) { response->mutable_resources()->at(0).add_aliases("prefix/domain2.test"); EXPECT_LOG_CONTAINS("debug", "for " + type_url + " from HAL 9000", - grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_)); - - const auto& subscriptions = grpc_mux_->subscriptions(); - auto sub = subscriptions.find(type_url); - - EXPECT_TRUE(sub != subscriptions.end()); - watch->update({}); + onDiscoveryResponse(std::move(response))); + EXPECT_TRUE(subscriptionExists(type_url)); + if (isUnifiedMuxTest()) { + dynamic_cast(grpc_mux_.get()) + ->updateWatch(type_url, + dynamic_cast(watch.get())->watch_, {}, + options); + } else { + watch->update({}); + } EXPECT_EQ("HAL 9000", stats_.textReadout("control_plane.identifier").value()); } // DeltaDiscoveryResponse that comes in response to an on-demand request that couldn't be resolved // will contain an empty Resource. The Resource's aliases field will be populated with the alias // originally used in the request. -TEST_F(NewGrpcMuxImplTest, ConfigUpdateWithNotFoundResponse) { +TEST_P(NewGrpcMuxImplTest, ConfigUpdateWithNotFoundResponse) { setup(); const std::string& type_url = Config::TypeUrl::get().VirtualHost; @@ -363,7 +431,7 @@ TEST_F(NewGrpcMuxImplTest, ConfigUpdateWithNotFoundResponse) { } // Validate basic gRPC mux subscriptions to xdstp:// glob collections. -TEST_F(NewGrpcMuxImplTest, XdsTpGlobCollection) { +TEST_P(NewGrpcMuxImplTest, XdsTpGlobCollection) { setup(); const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; @@ -398,11 +466,11 @@ TEST_F(NewGrpcMuxImplTest, XdsTpGlobCollection) { EXPECT_EQ(1, added_resources.size()); EXPECT_TRUE(TestUtility::protoEqual(added_resources[0].get().resource(), load_assignment)); })); - grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + onDiscoveryResponse(std::move(response)); } // Validate basic gRPC mux subscriptions to xdstp:// singletons. -TEST_F(NewGrpcMuxImplTest, XdsTpSingleton) { +TEST_P(NewGrpcMuxImplTest, XdsTpSingleton) { setup(); const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; @@ -455,7 +523,7 @@ TEST_F(NewGrpcMuxImplTest, XdsTpSingleton) { EXPECT_TRUE(TestUtility::protoEqual(added_resources[1].get().resource(), load_assignment)); EXPECT_TRUE(TestUtility::protoEqual(added_resources[2].get().resource(), load_assignment)); })); - grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + onDiscoveryResponse(std::move(response)); } } // namespace From 4a66a3113eb9a59b38011eac97bb48ffd2b5164d Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 14 Jul 2021 13:10:15 -0700 Subject: [PATCH 04/46] Moved most of mux functionality into the base class Signed-off-by: Dmitri Dolguikh --- source/common/config/grpc_stream.h | 20 +++-- source/common/config/xds_mux/grpc_mux_impl.cc | 70 +++--------------- source/common/config/xds_mux/grpc_mux_impl.h | 74 +++++++++---------- 3 files changed, 60 insertions(+), 104 deletions(-) diff --git a/source/common/config/grpc_stream.h b/source/common/config/grpc_stream.h index a682fef77d0c..6ebad5e1d879 100644 --- a/source/common/config/grpc_stream.h +++ b/source/common/config/grpc_stream.h @@ -17,11 +17,21 @@ namespace Config { template using ResponseProtoPtr = std::unique_ptr; +class GrpcStreamBase { +public: + virtual ~GrpcStreamBase() = default; + virtual void establishNewStream() PURE; + virtual void maybeUpdateQueueSizeStat(uint64_t size) PURE; + virtual bool grpcStreamAvailable() const PURE; + virtual bool checkRateLimitAllowsDrain() PURE; +}; + // Oversees communication for gRPC xDS implementations (parent to both regular xDS and delta // xDS variants). Reestablishes the gRPC channel when necessary, and provides rate limiting of // requests. template -class GrpcStream : public Grpc::AsyncStreamCallbacks, +class GrpcStream : public GrpcStreamBase, + public Grpc::AsyncStreamCallbacks, public Logger::Loggable { public: GrpcStream(GrpcStreamCallbacks* callbacks, Grpc::RawAsyncClientPtr async_client, @@ -52,7 +62,7 @@ class GrpcStream : public Grpc::AsyncStreamCallbacks, RetryInitialDelayMs, RetryMaxDelayMs, random_); } - void establishNewStream() { + void establishNewStream() override { ENVOY_LOG(debug, "Establishing new gRPC bidi stream for {}", service_method_.DebugString()); if (stream_ != nullptr) { ENVOY_LOG(warn, "gRPC bidi stream for {} already exists!", service_method_.DebugString()); @@ -69,7 +79,7 @@ class GrpcStream : public Grpc::AsyncStreamCallbacks, callbacks_->onStreamEstablished(); } - bool grpcStreamAvailable() const { return stream_ != nullptr; } + bool grpcStreamAvailable() const override { return stream_ != nullptr; } void sendMessage(const RequestProto& request) { stream_->sendMessage(request, false); } @@ -105,7 +115,7 @@ class GrpcStream : public Grpc::AsyncStreamCallbacks, setRetryTimer(); } - void maybeUpdateQueueSizeStat(uint64_t size) { + void maybeUpdateQueueSizeStat(uint64_t size) override { // Although request_queue_.push() happens elsewhere, the only time the queue is non-transiently // non-empty is when it remains non-empty after a drain attempt. (The push() doesn't matter // because we always attempt this drain immediately after the push). Basically, a change in @@ -118,7 +128,7 @@ class GrpcStream : public Grpc::AsyncStreamCallbacks, } } - bool checkRateLimitAllowsDrain() { + bool checkRateLimitAllowsDrain() override { if (!rate_limiting_enabled_ || limit_request_->consume(1, false)) { return true; } diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index a26b09e0a77c..e7f083c4fa6f 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -143,6 +143,18 @@ bool GrpcMuxImpl::paused(const std::string& type_url) const { return pausable_ack_queue_.paused(type_url); } +template +void GrpcMuxImpl::sendGrpcMessage(RQ& msg_proto, S& sub_state) { + if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || + !skipSubsequentNode()) { + msg_proto.mutable_node()->CopyFrom(localInfo().node()); + } + VersionConverter::prepareMessageForGrpcWire(msg_proto, transportApiVersion()); + sendMessage(msg_proto); + setAnyRequestSentYetInCurrentStream(true); + sub_state.clearDynamicContextChanged(); +} + template void GrpcMuxImpl::genericHandleResponse(const std::string& type_url, const RS& response_proto, @@ -328,33 +340,6 @@ GrpcMuxDelta::GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispat rate_limit_settings) {} // GrpcStreamCallbacks for GrpcMuxDelta -void GrpcMuxDelta::onStreamEstablished() { handleEstablishedStream(); } -void GrpcMuxDelta::onEstablishmentFailure() { handleStreamEstablishmentFailure(); } -void GrpcMuxDelta::onWriteable() { trySendDiscoveryRequests(); } -void GrpcMuxDelta::onDiscoveryResponse( - std::unique_ptr&& message, - ControlPlaneStats& control_plane_stats) { - genericHandleResponse(message->type_url(), *message, control_plane_stats); -} - -void GrpcMuxDelta::establishGrpcStream() { grpc_stream_.establishNewStream(); } -void GrpcMuxDelta::sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, - DeltaSubscriptionState& sub_state) { - if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || - !skipSubsequentNode()) { - msg_proto.mutable_node()->CopyFrom(localInfo().node()); - } - VersionConverter::prepareMessageForGrpcWire(msg_proto, transportApiVersion()); - grpc_stream_.sendMessage(msg_proto); - setAnyRequestSentYetInCurrentStream(true); - sub_state.clearDynamicContextChanged(); -} -void GrpcMuxDelta::maybeUpdateQueueSizeStat(uint64_t size) { - grpc_stream_.maybeUpdateQueueSizeStat(size); -} -bool GrpcMuxDelta::grpcStreamAvailable() const { return grpc_stream_.grpcStreamAvailable(); } -bool GrpcMuxDelta::rateLimitAllowsDrain() { return grpc_stream_.checkRateLimitAllowsDrain(); } - void GrpcMuxDelta::requestOnDemandUpdate(const std::string& type_url, const absl::flat_hash_set& for_update) { auto& sub = subscriptionStateFor(type_url); @@ -376,37 +361,6 @@ GrpcMuxSotw::GrpcMuxSotw(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatch grpc_stream_(this, std::move(async_client), service_method, random, dispatcher, scope, rate_limit_settings) {} -// GrpcStreamCallbacks for GrpcMuxSotw -void GrpcMuxSotw::onStreamEstablished() { handleEstablishedStream(); } -void GrpcMuxSotw::onEstablishmentFailure() { handleStreamEstablishmentFailure(); } -void GrpcMuxSotw::onWriteable() { trySendDiscoveryRequests(); } -void GrpcMuxSotw::onDiscoveryResponse( - std::unique_ptr&& message, - ControlPlaneStats& control_plane_stats) { - genericHandleResponse(message->type_url(), *message, control_plane_stats); -} - -void GrpcMuxSotw::establishGrpcStream() { grpc_stream_.establishNewStream(); } - -void GrpcMuxSotw::sendGrpcMessage(envoy::service::discovery::v3::DiscoveryRequest& msg_proto, - SotwSubscriptionState& sub_state) { - if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || - !skipSubsequentNode()) { - msg_proto.mutable_node()->MergeFrom(localInfo().node()); - } - VersionConverter::prepareMessageForGrpcWire(msg_proto, transportApiVersion()); - grpc_stream_.sendMessage(msg_proto); - setAnyRequestSentYetInCurrentStream(true); - sub_state.clearDynamicContextChanged(); -} - -void GrpcMuxSotw::maybeUpdateQueueSizeStat(uint64_t size) { - grpc_stream_.maybeUpdateQueueSizeStat(size); -} - -bool GrpcMuxSotw::grpcStreamAvailable() const { return grpc_stream_.grpcStreamAvailable(); } -bool GrpcMuxSotw::rateLimitAllowsDrain() { return grpc_stream_.checkRateLimitAllowsDrain(); } - Config::GrpcMuxWatchPtr NullGrpcMuxImpl::addWatch(const std::string&, const absl::flat_hash_set&, SubscriptionCallbacks&, OpaqueResourceDecoder&, diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index ed35bd5dc6f2..6e26c2d0d5c1 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -44,7 +44,8 @@ class WatchCompatibilityWrapper : public Envoy::Config::GrpcMuxWatch { // logic to properly order the subscription(s)' various messages, and allows // starting/stopping/pausing of the subscriptions. template -class GrpcMuxImpl : public GrpcMux, Logger::Loggable { +class GrpcMuxImpl : public GrpcStreamCallbacks, + public GrpcMux, Logger::Loggable { public: GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, @@ -69,13 +70,20 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { const absl::flat_hash_map>& subscriptions() const { return subscriptions_; } + bool isUnified() const override { return true; } + // GrpcStreamCallbacks + void onStreamEstablished() override { handleEstablishedStream(); } + void onEstablishmentFailure() override { handleStreamEstablishmentFailure(); } + void onWriteable() override { trySendDiscoveryRequests(); } + void onDiscoveryResponse( + std::unique_ptr&& message, ControlPlaneStats& control_plane_stats) override { + genericHandleResponse(message->type_url(), *message, control_plane_stats); + } void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } - bool isUnified() const override { return true; } - protected: // Everything related to GrpcStream must remain abstract. GrpcStream (and the gRPC-using classes // that underlie it) are templated on protobufs. That means that a single implementation that @@ -83,12 +91,13 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { // the GrpcStream will be owned by a derived class, and all code that would touch grpc_stream_ is // seen here in the base class as calls to abstract functions, to be provided by those derived // classes. - virtual void establishGrpcStream() PURE; - // Deletes msg_proto_ptr. - virtual void sendGrpcMessage(RQ& msg_proto, S& sub_state) PURE; - virtual void maybeUpdateQueueSizeStat(uint64_t size) PURE; - virtual bool grpcStreamAvailable() const PURE; - virtual bool rateLimitAllowsDrain() PURE; + virtual GrpcStreamBase& grpcStream() PURE; + void establishGrpcStream() { grpcStream().establishNewStream(); } + void sendGrpcMessage(RQ& msg_proto, S& sub_state); + void maybeUpdateQueueSizeStat(uint64_t size) { grpcStream().maybeUpdateQueueSizeStat(size); } + bool grpcStreamAvailable() { return grpcStream().grpcStreamAvailable(); } + bool rateLimitAllowsDrain() { return grpcStream().checkRateLimitAllowsDrain(); } + virtual void sendMessage(RQ& msg_proto) PURE; S& subscriptionStateFor(const std::string& type_url); WatchMap& watchMapFor(const std::string& type_url); @@ -161,8 +170,7 @@ class GrpcMuxImpl : public GrpcMux, Logger::Loggable { class GrpcMuxDelta : public GrpcMuxImpl, - public GrpcStreamCallbacks { + envoy::service::discovery::v3::DeltaDiscoveryResponse> { public: GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, const Protobuf::MethodDescriptor& service_method, @@ -172,14 +180,9 @@ class GrpcMuxDelta bool skip_subsequent_node); // GrpcStreamCallbacks - void onStreamEstablished() override; - void onEstablishmentFailure() override; - void onWriteable() override; - void onDiscoveryResponse( - std::unique_ptr&& message, - ControlPlaneStats& control_plane_stats) override; void requestOnDemandUpdate(const std::string& type_url, const absl::flat_hash_set& for_update) override; + GrpcStream& grpcStreamForTest() { @@ -187,12 +190,12 @@ class GrpcMuxDelta } protected: - void establishGrpcStream() override; - void sendGrpcMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto, - DeltaSubscriptionState& sub_state) override; - void maybeUpdateQueueSizeStat(uint64_t size) override; - bool grpcStreamAvailable() const override; - bool rateLimitAllowsDrain() override; + GrpcStreamBase& grpcStream() override { + return grpc_stream_; + } + void sendMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto) override { + grpc_stream_.sendMessage(msg_proto); + } private: GrpcStream, - public GrpcStreamCallbacks { + envoy::service::discovery::v3::DiscoveryResponse> { public: GrpcMuxSotw(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, const Protobuf::MethodDescriptor& service_method, @@ -212,16 +214,6 @@ class GrpcMuxSotw : public GrpcMuxImpl&& message, - ControlPlaneStats& control_plane_stats) override; - void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { - NOT_IMPLEMENTED_GCOVR_EXCL_LINE; - }; GrpcStream& grpcStreamForTest() { @@ -229,12 +221,12 @@ class GrpcMuxSotw : public GrpcMuxImpl Date: Wed, 14 Jul 2021 14:07:20 -0700 Subject: [PATCH 05/46] Added more mux tests Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.h | 24 +- test/common/config/BUILD | 27 + test/common/config/new_grpc_mux_impl_test.cc | 2 + test/common/config/xds_grpc_mux_impl_test.cc | 959 +++++++++++++++++++ 4 files changed, 998 insertions(+), 14 deletions(-) create mode 100644 test/common/config/xds_grpc_mux_impl_test.cc diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 6e26c2d0d5c1..d6470186b820 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -45,7 +45,8 @@ class WatchCompatibilityWrapper : public Envoy::Config::GrpcMuxWatch { // starting/stopping/pausing of the subscriptions. template class GrpcMuxImpl : public GrpcStreamCallbacks, - public GrpcMux, Logger::Loggable { + public GrpcMux, + Logger::Loggable { public: GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, @@ -76,8 +77,8 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, void onStreamEstablished() override { handleEstablishedStream(); } void onEstablishmentFailure() override { handleStreamEstablishmentFailure(); } void onWriteable() override { trySendDiscoveryRequests(); } - void onDiscoveryResponse( - std::unique_ptr&& message, ControlPlaneStats& control_plane_stats) override { + void onDiscoveryResponse(std::unique_ptr&& message, + ControlPlaneStats& control_plane_stats) override { genericHandleResponse(message->type_url(), *message, control_plane_stats); } void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { @@ -167,10 +168,9 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, const envoy::config::core::v3::ApiVersion transport_api_version_; }; -class GrpcMuxDelta - : public GrpcMuxImpl { +class GrpcMuxDelta : public GrpcMuxImpl { public: GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, const Protobuf::MethodDescriptor& service_method, @@ -182,7 +182,7 @@ class GrpcMuxDelta // GrpcStreamCallbacks void requestOnDemandUpdate(const std::string& type_url, const absl::flat_hash_set& for_update) override; - + GrpcStream& grpcStreamForTest() { @@ -190,9 +190,7 @@ class GrpcMuxDelta } protected: - GrpcStreamBase& grpcStream() override { - return grpc_stream_; - } + GrpcStreamBase& grpcStream() override { return grpc_stream_; } void sendMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto) override { grpc_stream_.sendMessage(msg_proto); } @@ -221,9 +219,7 @@ class GrpcMuxSotw : public GrpcMuxImpl()}; diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc new file mode 100644 index 000000000000..12e96dc53614 --- /dev/null +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -0,0 +1,959 @@ +#include + +#include "envoy/api/v2/discovery.pb.h" +#include "envoy/config/endpoint/v3/endpoint.pb.h" +#include "envoy/config/endpoint/v3/endpoint.pb.validate.h" +#include "envoy/service/discovery/v3/discovery.pb.h" + +#include "source/common/common/empty_string.h" +#include "source/common/config/api_version.h" +#include "source/common/config/protobuf_link_hacks.h" +#include "source/common/config/utility.h" +#include "source/common/config/version_converter.h" +#include "source/common/config/xds_mux/grpc_mux_impl.h" +#include "source/common/protobuf/protobuf.h" +#include "source/common/stats/isolated_store_impl.h" + +#include "test/common/stats/stat_test_utility.h" +#include "test/mocks/common.h" +#include "test/mocks/config/mocks.h" +#include "test/mocks/event/mocks.h" +#include "test/mocks/grpc/mocks.h" +#include "test/mocks/local_info/mocks.h" +#include "test/mocks/runtime/mocks.h" +#include "test/test_common/logging.h" +#include "test/test_common/resources.h" +#include "test/test_common/simulated_time_system.h" +#include "test/test_common/test_runtime.h" +#include "test/test_common/test_time.h" +#include "test/test_common/utility.h" + +#include "gmock/gmock.h" +#include "gtest/gtest.h" + +using testing::_; +using testing::AtLeast; +using testing::InSequence; +using testing::Invoke; +using testing::IsSubstring; +using testing::NiceMock; +using testing::Return; +using testing::ReturnRef; + +namespace Envoy { +namespace Config { +namespace XdsMux { +namespace { + +// We test some mux specific stuff below, other unit test coverage for singleton use of GrpcMuxImpl +// is provided in [grpc_]subscription_impl_test.cc. +class GrpcMuxImplTestBase : public testing::Test { +public: + GrpcMuxImplTestBase() + : async_client_(new Grpc::MockAsyncClient()), + control_plane_stats_(Utility::generateControlPlaneStats(stats_)), + control_plane_connected_state_( + stats_.gauge("control_plane.connected_state", Stats::Gauge::ImportMode::NeverImport)), + control_plane_pending_requests_( + stats_.gauge("control_plane.pending_requests", Stats::Gauge::ImportMode::NeverImport)), + resource_decoder_(TestUtility::TestOpaqueResourceDecoderImpl< + envoy::config::endpoint::v3::ClusterLoadAssignment>("cluster_name")) {} + + void setup() { + grpc_mux_ = std::make_unique( + std::unique_ptr(async_client_), dispatcher_, + *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( + "envoy.service.discovery.v2.AggregatedDiscoveryService.StreamAggregatedResources"), + envoy::config::core::v3::ApiVersion::AUTO, random_, stats_, rate_limit_settings_, + local_info_, true); + } + + void setup(const RateLimitSettings& custom_rate_limit_settings) { + grpc_mux_ = std::make_unique( + std::unique_ptr(async_client_), dispatcher_, + *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( + "envoy.service.discovery.v2.AggregatedDiscoveryService.StreamAggregatedResources"), + envoy::config::core::v3::ApiVersion::AUTO, random_, stats_, custom_rate_limit_settings, + local_info_, true); + } + + void expectSendMessage(const std::string& type_url, + const std::vector& resource_names, const std::string& version, + bool first = false, const std::string& nonce = "", + const Protobuf::int32 error_code = Grpc::Status::WellKnownGrpcStatus::Ok, + const std::string& error_message = "") { + API_NO_BOOST(envoy::api::v2::DiscoveryRequest) expected_request; + if (first) { + expected_request.mutable_node()->CopyFrom(API_DOWNGRADE(local_info_.node())); + } + for (const auto& resource : resource_names) { + expected_request.add_resource_names(resource); + } + if (!version.empty()) { + expected_request.set_version_info(version); + } + expected_request.set_response_nonce(nonce); + expected_request.set_type_url(type_url); + if (error_code != Grpc::Status::WellKnownGrpcStatus::Ok) { + ::google::rpc::Status* error_detail = expected_request.mutable_error_detail(); + error_detail->set_code(error_code); + error_detail->set_message(error_message); + } + EXPECT_CALL( + async_stream_, + sendMessageRaw_(Grpc::ProtoBufferEqIgnoreRepeatedFieldOrdering(expected_request), false)); + } + + // These tests were written around GrpcMuxWatch, an RAII type returned by the old subscribe(). + // To preserve these tests for the new code, we need an RAII watch handler. That is + // GrpcSubscriptionImpl, but to keep things simple, we'll fake it. (What we really care about + // is the destructor, which is identical to the real one). + class FakeGrpcSubscription { + public: + FakeGrpcSubscription(GrpcMux* grpc_mux, std::string type_url, Config::GrpcMuxWatchPtr watch) + : grpc_mux_(grpc_mux), type_url_(std::move(type_url)), + watch_(dynamic_cast(watch.get())->watch_) {} + ~FakeGrpcSubscription() { grpc_mux_->removeWatch(type_url_, watch_); } + + private: + GrpcMux* const grpc_mux_; + std::string type_url_; + Watch* const watch_; + }; + + FakeGrpcSubscription makeWatch(const std::string& type_url, + const absl::flat_hash_set& resources) { + return FakeGrpcSubscription( + grpc_mux_.get(), type_url, + grpc_mux_->addWatch(type_url, resources, callbacks_, resource_decoder_, {})); + } + + FakeGrpcSubscription makeWatch(const std::string& type_url, + const absl::flat_hash_set& resources, + NiceMock& callbacks, + Config::OpaqueResourceDecoder& resource_decoder) { + return FakeGrpcSubscription( + grpc_mux_.get(), type_url, + grpc_mux_->addWatch(type_url, resources, callbacks, resource_decoder, {})); + } + + NiceMock dispatcher_; + NiceMock random_; + Grpc::MockAsyncClient* async_client_; + Grpc::MockAsyncStream async_stream_; + NiceMock local_info_; + std::unique_ptr grpc_mux_; + NiceMock callbacks_; + Stats::TestUtil::TestStore stats_; + ControlPlaneStats control_plane_stats_; + Envoy::Config::RateLimitSettings rate_limit_settings_; + Stats::Gauge& control_plane_connected_state_; + Stats::Gauge& control_plane_pending_requests_; + TestUtility::TestOpaqueResourceDecoderImpl + resource_decoder_; +}; + +class GrpcMuxImplTest : public GrpcMuxImplTestBase { +public: + Event::SimulatedTimeSystem time_system_; +}; + +// Validate behavior when multiple type URL watches are maintained, watches are created/destroyed. +TEST_F(GrpcMuxImplTest, MultipleTypeUrlStreams) { + setup(); + InSequence s; + + FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); + FakeGrpcSubscription bar_sub = makeWatch("type_url_bar", {}); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("type_url_foo", {"x", "y"}, "", true); + expectSendMessage("type_url_bar", {}, ""); + grpc_mux_->start(); + EXPECT_EQ(1, control_plane_connected_state_.value()); + expectSendMessage("type_url_bar", {"z"}, ""); + FakeGrpcSubscription bar_z_sub = makeWatch("type_url_bar", {"z"}); + expectSendMessage("type_url_bar", {"zz", "z"}, ""); + FakeGrpcSubscription bar_zz_sub = makeWatch("type_url_bar", {"zz"}); + expectSendMessage("type_url_bar", {"z"}, ""); + expectSendMessage("type_url_bar", {}, ""); + expectSendMessage("type_url_foo", {}, ""); +} + +// Validate behavior when multiple type URL watches are maintained and the stream is reset. +TEST_F(GrpcMuxImplTest, ResetStream) { + InSequence s; + + auto* timer = new Event::MockTimer(&dispatcher_); + // TTL timers. + new Event::MockTimer(&dispatcher_); + new Event::MockTimer(&dispatcher_); + new Event::MockTimer(&dispatcher_); + + setup(); + FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); + FakeGrpcSubscription bar_sub = makeWatch("type_url_bar", {}); + FakeGrpcSubscription baz_sub = makeWatch("type_url_baz", {"z"}); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("type_url_foo", {"x", "y"}, "", true); + expectSendMessage("type_url_bar", {}, ""); + expectSendMessage("type_url_baz", {"z"}, ""); + grpc_mux_->start(); + + // Send another message for foo so that the node is cleared in the cached request. + // This is to test that the the node is set again in the first message below. + expectSendMessage("type_url_foo", {"z", "x", "y"}, ""); + FakeGrpcSubscription foo_z_sub = makeWatch("type_url_foo", {"z"}); + + EXPECT_CALL(callbacks_, + onConfigUpdateFailed(Envoy::Config::ConfigUpdateFailureReason::ConnectionFailure, _)) + .Times(4); + EXPECT_CALL(random_, random()); + EXPECT_CALL(*timer, enableTimer(_, _)); + grpc_mux_->grpcStreamForTest().onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); + EXPECT_EQ(0, control_plane_connected_state_.value()); + EXPECT_EQ(0, control_plane_pending_requests_.value()); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("type_url_foo", {"z", "x", "y"}, "", true); + expectSendMessage("type_url_bar", {}, ""); + expectSendMessage("type_url_baz", {"z"}, ""); + expectSendMessage("type_url_foo", {"x", "y"}, ""); + timer->invokeCallback(); + + expectSendMessage("type_url_baz", {}, ""); + expectSendMessage("type_url_foo", {}, ""); +} + +// Validate pause-resume behavior. +TEST_F(GrpcMuxImplTest, PauseResume) { + setup(); + InSequence s; + grpc_mux_->addWatch("type_url_foo", {"x", "y"}, callbacks_, resource_decoder_, {}); + { + ScopedResume a = grpc_mux_->pause("type_url_foo"); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + grpc_mux_->start(); + expectSendMessage("type_url_foo", {"x", "y"}, "", true); + } + { + ScopedResume a = grpc_mux_->pause("type_url_bar"); + expectSendMessage("type_url_foo", {"z", "x", "y"}, ""); + grpc_mux_->addWatch("type_url_foo", {"z"}, callbacks_, resource_decoder_, {}); + } + { + ScopedResume a = grpc_mux_->pause("type_url_foo"); + grpc_mux_->addWatch("type_url_foo", {"zz"}, callbacks_, resource_decoder_, {}); + expectSendMessage("type_url_foo", {"zz", "z", "x", "y"}, ""); + } + // When nesting, we only have a single resumption. + { + ScopedResume a = grpc_mux_->pause("type_url_foo"); + ScopedResume b = grpc_mux_->pause("type_url_foo"); + grpc_mux_->addWatch("type_url_foo", {"zzz"}, callbacks_, resource_decoder_, {}); + expectSendMessage("type_url_foo", {"zzz", "zz", "z", "x", "y"}, ""); + } + grpc_mux_->pause("type_url_foo")->cancel(); +} + +// Validate behavior when type URL mismatches occur. +TEST_F(GrpcMuxImplTest, TypeUrlMismatch) { + setup(); + + auto invalid_response = std::make_unique(); + InSequence s; + FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); + + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("type_url_foo", {"x", "y"}, "", true); + grpc_mux_->start(); + + { + auto response = std::make_unique(); + response->set_type_url("type_url_bar"); + response->set_version_info("bar-version"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + { + invalid_response->set_type_url("type_url_foo"); + invalid_response->set_version_info("foo-version"); + invalid_response->mutable_resources()->Add()->set_type_url("type_url_bar"); + EXPECT_CALL(callbacks_, onConfigUpdateFailed(_, _)) + .WillOnce(Invoke([](Envoy::Config::ConfigUpdateFailureReason, const EnvoyException* e) { + EXPECT_TRUE( + IsSubstring("", "", + "type URL type_url_bar embedded in an individual Any does not match the " + "message-wide type URL type_url_foo in DiscoveryResponse", + e->what())); + })); + + expectSendMessage( + "type_url_foo", {"x", "y"}, "", false, "", Grpc::Status::WellKnownGrpcStatus::Internal, + fmt::format("type URL type_url_bar embedded in an individual Any does not match the " + "message-wide type URL type_url_foo in DiscoveryResponse {}", + invalid_response->DebugString())); + grpc_mux_->onDiscoveryResponse(std::move(invalid_response), control_plane_stats_); + } + expectSendMessage("type_url_foo", {}, ""); +} + +TEST_F(GrpcMuxImplTest, RpcErrorMessageTruncated) { + setup(); + auto invalid_response = std::make_unique(); + InSequence s; + FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); + + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("type_url_foo", {"x", "y"}, "", true); + grpc_mux_->start(); + + { // Large error message sent back to management server is truncated. + const std::string very_large_type_url(1 << 20, 'A'); + invalid_response->set_type_url("type_url_foo"); + invalid_response->set_version_info("invalid"); + invalid_response->mutable_resources()->Add()->set_type_url(very_large_type_url); + EXPECT_CALL(callbacks_, onConfigUpdateFailed(_, _)) + .WillOnce(Invoke([&very_large_type_url](Envoy::Config::ConfigUpdateFailureReason, + const EnvoyException* e) { + EXPECT_TRUE( + IsSubstring("", "", + fmt::format("type URL {} embedded in an individual Any does not match " + "the message-wide type URL type_url_foo in DiscoveryResponse", + very_large_type_url), // Local error message is not truncated. + e->what())); + })); + expectSendMessage("type_url_foo", {"x", "y"}, "", false, "", + Grpc::Status::WellKnownGrpcStatus::Internal, + fmt::format("type URL {}...(truncated)", std::string(4087, 'A'))); + grpc_mux_->onDiscoveryResponse(std::move(invalid_response), control_plane_stats_); + } + expectSendMessage("type_url_foo", {}, ""); +} + +envoy::service::discovery::v3::Resource heartbeatResource(std::chrono::milliseconds ttl, + const std::string& name) { + envoy::service::discovery::v3::Resource resource; + + resource.mutable_ttl()->CopyFrom(Protobuf::util::TimeUtil::MillisecondsToDuration(ttl.count())); + resource.set_name(name); + + return resource; +} + +envoy::service::discovery::v3::Resource +resourceWithTtl(std::chrono::milliseconds ttl, + envoy::config::endpoint::v3::ClusterLoadAssignment& cla) { + envoy::service::discovery::v3::Resource resource; + resource.mutable_resource()->PackFrom(cla); + resource.mutable_ttl()->CopyFrom(Protobuf::util::TimeUtil::MillisecondsToDuration(ttl.count())); + + resource.set_name(cla.cluster_name()); + + return resource; +} +envoy::service::discovery::v3::Resource +resourceWithEmptyTtl(envoy::config::endpoint::v3::ClusterLoadAssignment& cla) { + envoy::service::discovery::v3::Resource resource; + resource.mutable_resource()->PackFrom(cla); + resource.set_name(cla.cluster_name()); + return resource; +} +// Validates the behavior when the TTL timer expires. +TEST_F(GrpcMuxImplTest, ResourceTTL) { + setup(); + + time_system_.setSystemTime(std::chrono::seconds(0)); + + TestUtility::TestOpaqueResourceDecoderImpl + resource_decoder("cluster_name"); + const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; + InSequence s; + auto* ttl_timer = new Event::MockTimer(&dispatcher_); + FakeGrpcSubscription eds_sub = makeWatch(type_url, {"x"}, callbacks_, resource_decoder); + + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage(type_url, {"x"}, "", true); + grpc_mux_->start(); + + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment; + load_assignment.set_cluster_name("x"); + + auto wrapped_resource = resourceWithTtl(std::chrono::milliseconds(1000), load_assignment); + response->add_resources()->PackFrom(wrapped_resource); + + EXPECT_CALL(*ttl_timer, enabled()); + EXPECT_CALL(*ttl_timer, enableTimer(std::chrono::milliseconds(1000), _)); + EXPECT_CALL(callbacks_, onConfigUpdate(_, "1")) + .WillOnce(Invoke([](const std::vector& resources, const std::string&) { + EXPECT_EQ(1, resources.size()); + })); + expectSendMessage(type_url, {"x"}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + // Increase the TTL. + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment; + load_assignment.set_cluster_name("x"); + auto wrapped_resource = resourceWithTtl(std::chrono::milliseconds(10000), load_assignment); + response->add_resources()->PackFrom(wrapped_resource); + + EXPECT_CALL(*ttl_timer, enabled()); + EXPECT_CALL(*ttl_timer, enableTimer(std::chrono::milliseconds(10000), _)); + EXPECT_CALL(callbacks_, onConfigUpdate(_, "1")) + .WillOnce(Invoke([](const std::vector& resources, const std::string&) { + EXPECT_EQ(1, resources.size()); + })); + // No update, just a change in TTL. + expectSendMessage(type_url, {"x"}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + // Refresh the TTL with a heartbeat response. + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + auto wrapped_resource = heartbeatResource(std::chrono::milliseconds(10000), "x"); + response->add_resources()->PackFrom(wrapped_resource); + + EXPECT_CALL(*ttl_timer, enabled()); + EXPECT_CALL(callbacks_, onConfigUpdate(_, "1")) + .WillOnce(Invoke([](const std::vector& resources, const std::string&) { + EXPECT_TRUE(resources.empty()); + })); + + // No update, just a change in TTL. + expectSendMessage(type_url, {"x"}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + // Remove the TTL. + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment; + load_assignment.set_cluster_name("x"); + response->add_resources()->PackFrom(resourceWithEmptyTtl(load_assignment)); + + EXPECT_CALL(*ttl_timer, disableTimer()); + EXPECT_CALL(callbacks_, onConfigUpdate(_, "1")) + .WillOnce(Invoke([](const std::vector& resources, const std::string&) { + EXPECT_EQ(1, resources.size()); + })); + expectSendMessage(type_url, {"x"}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + // Put the TTL back. + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment; + load_assignment.set_cluster_name("x"); + auto wrapped_resource = resourceWithTtl(std::chrono::milliseconds(10000), load_assignment); + response->add_resources()->PackFrom(wrapped_resource); + + EXPECT_CALL(*ttl_timer, enabled()); + EXPECT_CALL(*ttl_timer, enableTimer(std::chrono::milliseconds(10000), _)); + EXPECT_CALL(callbacks_, onConfigUpdate(_, "1")) + .WillOnce(Invoke([](const std::vector& resources, const std::string&) { + EXPECT_EQ(1, resources.size()); + })); + // No update, just a change in TTL. + expectSendMessage(type_url, {"x"}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + time_system_.setSystemTime(std::chrono::seconds(11)); + EXPECT_CALL(callbacks_, onConfigUpdate(_, _, "")) + .WillOnce(Invoke([](auto, const auto& removed, auto) { + EXPECT_EQ(1, removed.size()); + EXPECT_EQ("x", removed.Get(0)); + })); + // Fire the TTL timer. + EXPECT_CALL(*ttl_timer, disableTimer()); + ttl_timer->invokeCallback(); + + expectSendMessage(type_url, {}, "1"); +} + +// Checks that the control plane identifier is logged +TEST_F(GrpcMuxImplTest, LogsControlPlaneIndentifier) { + setup(); + + std::string type_url = "foo"; + auto foo_sub = makeWatch(type_url, {}, callbacks_, resource_decoder_); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage(type_url, {}, "", true); + grpc_mux_->start(); + + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + response->mutable_control_plane()->set_identifier("control_plane_ID"); + + EXPECT_CALL(callbacks_, onConfigUpdate(_, _)); + expectSendMessage(type_url, {}, "1"); + EXPECT_LOG_CONTAINS("debug", "for foo from control_plane_ID", + grpc_mux_->grpcStreamForTest().onReceiveMessage(std::move(response))); + } + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("2"); + response->mutable_control_plane()->set_identifier("different_ID"); + + EXPECT_CALL(callbacks_, onConfigUpdate(_, _)); + expectSendMessage(type_url, {}, "2"); + EXPECT_LOG_CONTAINS("debug", "for foo from different_ID", + grpc_mux_->grpcStreamForTest().onReceiveMessage(std::move(response))); + } +} + +// Validate behavior when watches has an unknown resource name. +TEST_F(GrpcMuxImplTest, WildcardWatch) { + setup(); + + const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; + FakeGrpcSubscription foo_sub = makeWatch(type_url, {}, callbacks_, resource_decoder_); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage(type_url, {}, "", true); + grpc_mux_->start(); + + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment; + load_assignment.set_cluster_name("x"); + response->add_resources()->PackFrom(load_assignment); + EXPECT_CALL(callbacks_, onConfigUpdate(_, "1")) + .WillOnce(Invoke([&load_assignment](const std::vector& resources, + const std::string&) { + EXPECT_EQ(1, resources.size()); + const auto& expected_assignment = + dynamic_cast( + resources[0].get().resource()); + EXPECT_TRUE(TestUtility::protoEqual(expected_assignment, load_assignment)); + })); + expectSendMessage(type_url, {}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } +} + +// Validate behavior when watches specify resources (potentially overlapping). +TEST_F(GrpcMuxImplTest, WatchDemux) { + setup(); + // We will not require InSequence here: an update that causes multiple onConfigUpdates + // causes them in an indeterminate order, based on the whims of the hash map. + const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; + + NiceMock foo_callbacks; + FakeGrpcSubscription foo_sub = makeWatch(type_url, {"x", "y"}, foo_callbacks, resource_decoder_); + NiceMock bar_callbacks; + FakeGrpcSubscription bar_sub = makeWatch(type_url, {"y", "z"}, bar_callbacks, resource_decoder_); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + // Should dedupe the "x" resource. + expectSendMessage(type_url, {"y", "z", "x"}, "", true); + grpc_mux_->start(); + + // Send just x; only foo_callbacks should receive an onConfigUpdate(). + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment; + load_assignment.set_cluster_name("x"); + response->add_resources()->PackFrom(load_assignment); + EXPECT_CALL(bar_callbacks, onConfigUpdate(_, "1")).Times(0); + EXPECT_CALL(foo_callbacks, onConfigUpdate(_, "1")) + .WillOnce(Invoke([&load_assignment](const std::vector& resources, + const std::string&) { + EXPECT_EQ(1, resources.size()); + const auto& expected_assignment = + dynamic_cast( + resources[0].get().resource()); + EXPECT_TRUE(TestUtility::protoEqual(expected_assignment, load_assignment)); + })); + expectSendMessage(type_url, {"y", "z", "x"}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + // Send x y and z; foo_ and bar_callbacks should both receive onConfigUpdate()s, carrying {x,y} + // and {y,z} respectively. + { + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("2"); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment_x; + load_assignment_x.set_cluster_name("x"); + response->add_resources()->PackFrom(load_assignment_x); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment_y; + load_assignment_y.set_cluster_name("y"); + response->add_resources()->PackFrom(load_assignment_y); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment_z; + load_assignment_z.set_cluster_name("z"); + response->add_resources()->PackFrom(load_assignment_z); + EXPECT_CALL(bar_callbacks, onConfigUpdate(_, "2")) + .WillOnce(Invoke([&load_assignment_y, &load_assignment_z]( + const std::vector& resources, const std::string&) { + EXPECT_EQ(2, resources.size()); + const auto& expected_assignment = + dynamic_cast( + resources[0].get().resource()); + EXPECT_TRUE(TestUtility::protoEqual(expected_assignment, load_assignment_y)); + const auto& expected_assignment_1 = + dynamic_cast( + resources[1].get().resource()); + EXPECT_TRUE(TestUtility::protoEqual(expected_assignment_1, load_assignment_z)); + })); + EXPECT_CALL(foo_callbacks, onConfigUpdate(_, "2")) + .WillOnce(Invoke([&load_assignment_x, &load_assignment_y]( + const std::vector& resources, const std::string&) { + EXPECT_EQ(2, resources.size()); + const auto& expected_assignment = + dynamic_cast( + resources[0].get().resource()); + EXPECT_TRUE(TestUtility::protoEqual(expected_assignment, load_assignment_x)); + const auto& expected_assignment_1 = + dynamic_cast( + resources[1].get().resource()); + EXPECT_TRUE(TestUtility::protoEqual(expected_assignment_1, load_assignment_y)); + })); + expectSendMessage(type_url, {"y", "z", "x"}, "2"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + + expectSendMessage(type_url, {"x", "y"}, "2"); + expectSendMessage(type_url, {}, "2"); +} + +// Validate behavior when we have multiple watchers that send empty updates. +TEST_F(GrpcMuxImplTest, MultipleWatcherWithEmptyUpdates) { + setup(); + InSequence s; + const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; + NiceMock foo_callbacks; + FakeGrpcSubscription foo_sub = makeWatch(type_url, {"x", "y"}, foo_callbacks, resource_decoder_); + + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage(type_url, {"x", "y"}, "", true); + grpc_mux_->start(); + + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + + EXPECT_CALL(foo_callbacks, onConfigUpdate(_, "1")).Times(0); + expectSendMessage(type_url, {"x", "y"}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + + expectSendMessage(type_url, {}, "1"); +} + +// Validate behavior when we have Single Watcher that sends Empty updates. +TEST_F(GrpcMuxImplTest, SingleWatcherWithEmptyUpdates) { + setup(); + const std::string& type_url = Config::TypeUrl::get().Cluster; + NiceMock foo_callbacks; + FakeGrpcSubscription foo_sub = makeWatch(type_url, {}, foo_callbacks, resource_decoder_); + + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage(type_url, {}, "", true); + grpc_mux_->start(); + + auto response = std::make_unique(); + response->set_type_url(type_url); + response->set_version_info("1"); + // Validate that onConfigUpdate is called with empty resources. + EXPECT_CALL(foo_callbacks, onConfigUpdate(_, "1")) + .WillOnce(Invoke([](const std::vector& resources, const std::string&) { + EXPECT_TRUE(resources.empty()); + })); + expectSendMessage(type_url, {}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); +} + +// Exactly one test requires a mock time system to provoke behavior that cannot +// easily be achieved with a SimulatedTimeSystem. +class GrpcMuxImplTestWithMockTimeSystem : public GrpcMuxImplTestBase { +public: + Event::DelegatingTestTimeSystem mock_time_system_; +}; + +// Verifies that rate limiting is not enforced with defaults. +TEST_F(GrpcMuxImplTestWithMockTimeSystem, TooManyRequestsWithDefaultSettings) { + + auto ttl_timer = new Event::MockTimer(&dispatcher_); + // Retry timer, + new Event::MockTimer(&dispatcher_); + + // Validate that rate limiter is not created. + EXPECT_CALL(*mock_time_system_, monotonicTime()).Times(0); + + setup(); + + EXPECT_CALL(async_stream_, sendMessageRaw_(_, false)).Times(AtLeast(99)); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + + const auto onReceiveMessage = [&](uint64_t burst) { + for (uint64_t i = 0; i < burst; i++) { + auto response = std::make_unique(); + response->set_version_info("type_url_baz"); + response->set_nonce("type_url_bar"); + response->set_type_url("type_url_foo"); + EXPECT_CALL(*ttl_timer, disableTimer()); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + }; + + FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x"}); + expectSendMessage("type_url_foo", {"x"}, "", true); + grpc_mux_->start(); + + // Exhausts the limit. + onReceiveMessage(99); + + // API calls go over the limit but we do not see the stat incremented. + onReceiveMessage(1); + EXPECT_EQ(0, stats_.counter("control_plane.rate_limit_enforced").value()); +} + +// Verifies that default rate limiting is enforced with empty RateLimitSettings. +TEST_F(GrpcMuxImplTest, TooManyRequestsWithEmptyRateLimitSettings) { + // Validate that request drain timer is created. + + auto ttl_timer = new Event::MockTimer(&dispatcher_); + Event::MockTimer* drain_request_timer = new Event::MockTimer(&dispatcher_); + Event::MockTimer* retry_timer = new Event::MockTimer(&dispatcher_); + + RateLimitSettings custom_rate_limit_settings; + custom_rate_limit_settings.enabled_ = true; + setup(custom_rate_limit_settings); + + // Attempt to send 99 messages. One of them is rate limited (and we never drain). + EXPECT_CALL(async_stream_, sendMessageRaw_(_, false)).Times(99); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + + const auto onReceiveMessage = [&](uint64_t burst) { + for (uint64_t i = 0; i < burst; i++) { + auto response = std::make_unique(); + response->set_version_info("type_url_baz"); + response->set_nonce("type_url_bar"); + response->set_type_url("type_url_foo"); + EXPECT_CALL(*ttl_timer, disableTimer()); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + }; + + FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x"}); + expectSendMessage("type_url_foo", {"x"}, "", true); + grpc_mux_->start(); + + // Validate that drain_request_timer is enabled when there are no tokens. + EXPECT_CALL(*drain_request_timer, enableTimer(std::chrono::milliseconds(100), _)); + // The drain timer enable is checked twice, once when we limit, again when the watch is destroyed. + EXPECT_CALL(*drain_request_timer, enabled()).Times(11); + onReceiveMessage(110); + EXPECT_EQ(11, stats_.counter("control_plane.rate_limit_enforced").value()); + EXPECT_EQ(11, control_plane_pending_requests_.value()); + + // Validate that when we reset a stream with pending requests, it reverts back to the initial + // query (i.e. the queue is discarded). + EXPECT_CALL(callbacks_, + onConfigUpdateFailed(Envoy::Config::ConfigUpdateFailureReason::ConnectionFailure, _)); + EXPECT_CALL(random_, random()); + EXPECT_CALL(*retry_timer, enableTimer(_, _)); + grpc_mux_->grpcStreamForTest().onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); + EXPECT_EQ(11, control_plane_pending_requests_.value()); + EXPECT_EQ(0, control_plane_connected_state_.value()); + EXPECT_CALL(async_stream_, sendMessageRaw_(_, false)); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + time_system_.setMonotonicTime(std::chrono::seconds(30)); + retry_timer->invokeCallback(); + EXPECT_EQ(0, control_plane_pending_requests_.value()); + // One more message on the way out when the watch is destroyed. + EXPECT_CALL(async_stream_, sendMessageRaw_(_, false)); +} + +// Verifies that rate limiting is enforced with custom RateLimitSettings. +TEST_F(GrpcMuxImplTest, TooManyRequestsWithCustomRateLimitSettings) { + // Validate that request drain timer is created. + + // TTL timer. + auto ttl_timer = new Event::MockTimer(&dispatcher_); + Event::MockTimer* drain_request_timer = new Event::MockTimer(&dispatcher_); + // Retry timer. + new Event::MockTimer(&dispatcher_); + + RateLimitSettings custom_rate_limit_settings; + custom_rate_limit_settings.enabled_ = true; + custom_rate_limit_settings.max_tokens_ = 250; + custom_rate_limit_settings.fill_rate_ = 2; + setup(custom_rate_limit_settings); + + EXPECT_CALL(async_stream_, sendMessageRaw_(_, false)).Times(AtLeast(260)); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + + const auto onReceiveMessage = [&](uint64_t burst) { + for (uint64_t i = 0; i < burst; i++) { + auto response = std::make_unique(); + response->set_version_info("type_url_baz"); + response->set_nonce("type_url_bar"); + response->set_type_url("type_url_foo"); + EXPECT_CALL(*ttl_timer, disableTimer()); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + } + }; + + FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x"}); + expectSendMessage("type_url_foo", {"x"}, "", true); + grpc_mux_->start(); + + // Validate that rate limit is not enforced for 100 requests. + onReceiveMessage(100); + EXPECT_EQ(0, stats_.counter("control_plane.rate_limit_enforced").value()); + + // Validate that drain_request_timer is enabled when there are no tokens. + EXPECT_CALL(*drain_request_timer, enableTimer(std::chrono::milliseconds(500), _)); + EXPECT_CALL(*drain_request_timer, enabled()).Times(11); + onReceiveMessage(160); + EXPECT_EQ(11, stats_.counter("control_plane.rate_limit_enforced").value()); + EXPECT_EQ(11, control_plane_pending_requests_.value()); + + // Validate that drain requests call when there are multiple requests in queue. + time_system_.setMonotonicTime(std::chrono::seconds(10)); + drain_request_timer->invokeCallback(); + + // Check that the pending_requests stat is updated with the queue drain. + EXPECT_EQ(0, control_plane_pending_requests_.value()); +} + +// Verifies that a message with no resources is accepted. +TEST_F(GrpcMuxImplTest, UnwatchedTypeAcceptsEmptyResources) { + setup(); + + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + + const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; + + grpc_mux_->start(); + { + // subscribe and unsubscribe to simulate a cluster added and removed + expectSendMessage(type_url, {"y"}, "", true); + FakeGrpcSubscription temp_sub = makeWatch(type_url, {"y"}); + expectSendMessage(type_url, {}, ""); + } + + // simulate the server sending empty CLA message to notify envoy that the CLA was removed. + auto response = std::make_unique(); + response->set_nonce("bar"); + response->set_version_info("1"); + response->set_type_url(type_url); + + // Although the update will change nothing for us, we will "accept" it, and so according + // to the spec we should ACK it. + expectSendMessage(type_url, {}, "1", false, "bar"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); + + // When we become interested in "x", we should send a request indicating that interest. + expectSendMessage(type_url, {"x"}, "1", false, "bar"); + FakeGrpcSubscription sub = makeWatch(type_url, {"x"}); + + // Watch destroyed -> interest gone -> unsubscribe request. + expectSendMessage(type_url, {}, "1", false, "bar"); +} + +// Verifies that a message with some resources is accepted even when there are no watches. +// Rationale: SotW gRPC xDS has always been willing to accept updates that include +// uninteresting resources. It should not matter whether those uninteresting resources +// are accompanied by interesting ones. +// Note: this was previously "rejects", not "accepts". See +// https://github.com/envoyproxy/envoy/pull/8350#discussion_r328218220 for discussion. +TEST_F(GrpcMuxImplTest, UnwatchedTypeAcceptsResources) { + setup(); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + const std::string& type_url = + Config::getTypeUrl( + envoy::config::core::v3::ApiVersion::V3); + grpc_mux_->start(); + + // subscribe and unsubscribe so that the type is known to envoy + { + expectSendMessage(type_url, {"y"}, "", true); + expectSendMessage(type_url, {}, ""); + FakeGrpcSubscription delete_immediately = makeWatch(type_url, {"y"}); + } + auto response = std::make_unique(); + response->set_type_url(type_url); + envoy::config::endpoint::v3::ClusterLoadAssignment load_assignment; + load_assignment.set_cluster_name("x"); + response->add_resources()->PackFrom(load_assignment); + response->set_version_info("1"); + + expectSendMessage(type_url, {}, "1"); + grpc_mux_->onDiscoveryResponse(std::move(response), control_plane_stats_); +} + +TEST_F(GrpcMuxImplTest, BadLocalInfoEmptyClusterName) { + EXPECT_CALL(local_info_, clusterName()).WillOnce(ReturnRef(EMPTY_STRING)); + EXPECT_THROW_WITH_MESSAGE( + XdsMux::GrpcMuxSotw( + std::unique_ptr(async_client_), dispatcher_, + *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( + "envoy.service.discovery.v2.AggregatedDiscoveryService.StreamAggregatedResources"), + envoy::config::core::v3::ApiVersion::AUTO, random_, stats_, rate_limit_settings_, + local_info_, true), + EnvoyException, + "ads: node 'id' and 'cluster' are required. Set it either in 'node' config or via " + "--service-node and --service-cluster options."); +} + +TEST_F(GrpcMuxImplTest, BadLocalInfoEmptyNodeName) { + EXPECT_CALL(local_info_, nodeName()).WillOnce(ReturnRef(EMPTY_STRING)); + EXPECT_THROW_WITH_MESSAGE( + XdsMux::GrpcMuxSotw( + std::unique_ptr(async_client_), dispatcher_, + *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( + "envoy.service.discovery.v2.AggregatedDiscoveryService.StreamAggregatedResources"), + envoy::config::core::v3::ApiVersion::AUTO, random_, stats_, rate_limit_settings_, + local_info_, true), + EnvoyException, + "ads: node 'id' and 'cluster' are required. Set it either in 'node' config or via " + "--service-node and --service-cluster options."); +} + +// Validate behavior when dynamic context parameters are updated. +TEST_F(GrpcMuxImplTest, DynamicContextParameters) { + setup(); + InSequence s; + grpc_mux_->addWatch("foo", {"x", "y"}, callbacks_, resource_decoder_, {}); + grpc_mux_->addWatch("bar", {}, callbacks_, resource_decoder_, {}); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("foo", {"x", "y"}, "", true); + expectSendMessage("bar", {}, ""); + grpc_mux_->start(); + // Unknown type, shouldn't do anything. + local_info_.context_provider_.update_cb_handler_.runCallbacks("baz"); + // Update to foo type should resend Node. + expectSendMessage("foo", {"x", "y"}, "", true); + local_info_.context_provider_.update_cb_handler_.runCallbacks("foo"); + // Update to bar type should resend Node. + expectSendMessage("bar", {}, "", true); + local_info_.context_provider_.update_cb_handler_.runCallbacks("bar"); +} + +} // namespace +} // namespace XdsMux +} // namespace Config +} // namespace Envoy From 916d8ba698eb310ec68e9860ec4cb383646ed1cd Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 14 Jul 2021 15:01:13 -0700 Subject: [PATCH 06/46] Fixed vuild failure Signed-off-by: Dmitri Dolguikh --- source/common/config/grpc_mux_impl.cc | 10 ++++++++++ source/common/config/grpc_mux_impl.h | 4 ++++ test/mocks/config/mocks.h | 9 +++++++++ 3 files changed, 23 insertions(+) diff --git a/source/common/config/grpc_mux_impl.cc b/source/common/config/grpc_mux_impl.cc index 58807df00d2d..755da6efcf71 100644 --- a/source/common/config/grpc_mux_impl.cc +++ b/source/common/config/grpc_mux_impl.cc @@ -343,5 +343,15 @@ void GrpcMuxImpl::drainRequests() { grpc_stream_.maybeUpdateQueueSizeStat(request_queue_->size()); } +void NullGrpcMuxImpl::updateWatch(const std::string&, Watch*, + const absl::flat_hash_set&, + const SubscriptionOptions&) { + throw EnvoyException("ADS must be configured to support an ADS config source"); +} + +void NullGrpcMuxImpl::removeWatch(const std::string&, Watch*) { + throw EnvoyException("ADS must be configured to support an ADS config source"); +} + } // namespace Config } // namespace Envoy diff --git a/source/common/config/grpc_mux_impl.h b/source/common/config/grpc_mux_impl.h index c02b0c6a7134..2b4ddc167cac 100644 --- a/source/common/config/grpc_mux_impl.h +++ b/source/common/config/grpc_mux_impl.h @@ -209,6 +209,10 @@ class NullGrpcMuxImpl : public GrpcMux, ExceptionUtil::throwEnvoyException("ADS must be configured to support an ADS config source"); } + bool paused(const std::string&) const override { return false; } + void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, + const SubscriptionOptions&) override; + void removeWatch(const std::string&, Watch*) override; void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } diff --git a/test/mocks/config/mocks.h b/test/mocks/config/mocks.h index ae12fa665c6d..cbf69387a929 100644 --- a/test/mocks/config/mocks.h +++ b/test/mocks/config/mocks.h @@ -122,6 +122,15 @@ class MockGrpcMux : public GrpcMux { MOCK_METHOD(void, requestOnDemandUpdate, (const std::string& type_url, const absl::flat_hash_set& add_these_names)); + + MOCK_METHOD(void, updateWatch, + (const std::string& type_url, Watch* watch, + const absl::flat_hash_set& resources, + const SubscriptionOptions& options)); + + MOCK_METHOD(void, removeWatch, (const std::string& type_url, Watch* watch)); + + MOCK_METHOD(bool, paused, (const std::string& type_url), (const)); }; class MockGrpcStreamCallbacks From 6bed7945076b44b3b772a40907b9d12cdfaccb96 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 14 Jul 2021 15:37:13 -0700 Subject: [PATCH 07/46] Fixed formatting Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.h | 6 ------ test/common/config/new_grpc_mux_impl_test.cc | 4 ++-- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index d6470186b820..fdb8004036ab 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -86,12 +86,6 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, } protected: - // Everything related to GrpcStream must remain abstract. GrpcStream (and the gRPC-using classes - // that underlie it) are templated on protobufs. That means that a single implementation that - // supports different types of protobufs cannot use polymorphism to share code. The workaround: - // the GrpcStream will be owned by a derived class, and all code that would touch grpc_stream_ is - // seen here in the base class as calls to abstract functions, to be provided by those derived - // classes. virtual GrpcStreamBase& grpcStream() PURE; void establishGrpcStream() { grpcStream().establishNewStream(); } void sendGrpcMessage(RQ& msg_proto, S& sub_state); diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index ef10d0a49cbb..2892d0e9c2b6 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -192,7 +192,7 @@ TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { } // Validate cached nonces are cleared on reconnection. -// TODO (dmitri-d) remove this test when legacy muxes have been removed +// TODO (dmitri-d) remove this test when legacy implementations have been removed // common mux functionality is tested in xds_grpc_mux_impl_test.cc TEST_P(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { Event::MockTimer* grpc_stream_retry_timer{new Event::MockTimer()}; @@ -239,7 +239,7 @@ TEST_P(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { {{"x", "2000"}, {"y", "3000"}}); remoteClose(); if (!isUnifiedMuxTest()) { - // in legacy mux implmentation, destruction of the EDS subscription will issue an "unsubscribe" + // in legacy implmentation, destruction of the EDS subscription will issue an "unsubscribe" // request. expectSendMessage(type_url, {}, {"x", "y"}); } From 7289b6ea67a8a70ebfd7c4b1d620f19959899d6d Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 14 Jul 2021 16:09:50 -0700 Subject: [PATCH 08/46] Fixing formatting Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index 2892d0e9c2b6..86c0343b41ad 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -239,7 +239,7 @@ TEST_P(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { {{"x", "2000"}, {"y", "3000"}}); remoteClose(); if (!isUnifiedMuxTest()) { - // in legacy implmentation, destruction of the EDS subscription will issue an "unsubscribe" + // in legacy implementation, destruction of the EDS subscription will issue an "unsubscribe" // request. expectSendMessage(type_url, {}, {"x", "y"}); } From 968b68c50b1f04e85bc300c6e180aac057d05015 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 14 Jul 2021 16:22:53 -0700 Subject: [PATCH 09/46] Fixing formatting Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/BUILD | 2 +- source/common/config/xds_mux/grpc_mux_impl.cc | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/config/xds_mux/BUILD b/source/common/config/xds_mux/BUILD index 5a6900d0e456..dfb9f6786c7b 100644 --- a/source/common/config/xds_mux/BUILD +++ b/source/common/config/xds_mux/BUILD @@ -71,4 +71,4 @@ envoy_cc_library( "//source/common/memory:utils_lib", "@envoy_api//envoy/service/discovery/v3:pkg_cc_proto", ], -) \ No newline at end of file +) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index e7f083c4fa6f..0c9539153db6 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -380,4 +380,4 @@ void NullGrpcMuxImpl::removeWatch(const std::string&, Watch*) { } // namespace XdsMux } // namespace Config -} // namespace Envoy \ No newline at end of file +} // namespace Envoy From 25d713494c07f375a62ec2af763b527b7dab376c Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Fri, 16 Jul 2021 11:45:51 -0700 Subject: [PATCH 10/46] Added tests to improve test coverage Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 18 ++++++++++++++++++ test/common/config/xds_grpc_mux_impl_test.cc | 7 +++++++ 2 files changed, 25 insertions(+) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index 86c0343b41ad..76222bcce274 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -528,6 +528,24 @@ TEST_P(NewGrpcMuxImplTest, XdsTpSingleton) { onDiscoveryResponse(std::move(response)); } +TEST_P(NewGrpcMuxImplTest, RequestOnDemandUpdate) { + setup(); + + auto foo_sub = grpc_mux_->addWatch("foo", {"x", "y"}, callbacks_, resource_decoder_, {}); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("foo", {"x", "y"}, {}); + grpc_mux_->start(); + + expectSendMessage("foo", {"z"}, {}); + grpc_mux_->requestOnDemandUpdate("foo", {"z"}); + + if (!isUnifiedMuxTest()) { + // in legacy implementation, destruction of the EDS subscription will issue an "unsubscribe" + // request. + expectSendMessage("foo", {}, {"x", "y"}); + } +} + } // namespace } // namespace Config } // namespace Envoy diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc index 12e96dc53614..978306b9a6a9 100644 --- a/test/common/config/xds_grpc_mux_impl_test.cc +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -223,6 +223,13 @@ TEST_F(GrpcMuxImplTest, ResetStream) { expectSendMessage("type_url_foo", {}, ""); } +// Validate paused() behaviour. +TEST_F(GrpcMuxImplTest, Paused) { + setup(); + ScopedResume a = grpc_mux_->pause("type_url_foo"); + EXPECT_TRUE(grpc_mux_->paused("type_url_foo")); +} + // Validate pause-resume behavior. TEST_F(GrpcMuxImplTest, PauseResume) { setup(); From 8aa55b83443b356890a44e45cf1eb8390e50fb57 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Thu, 22 Jul 2021 10:48:16 -0700 Subject: [PATCH 11/46] small refactoring Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.h | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index fdb8004036ab..35413dee61ae 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -81,9 +81,6 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, ControlPlaneStats& control_plane_stats) override { genericHandleResponse(message->type_url(), *message, control_plane_stats); } - void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { - NOT_IMPLEMENTED_GCOVR_EXCL_LINE; - } protected: virtual GrpcStreamBase& grpcStream() PURE; @@ -206,6 +203,11 @@ class GrpcMuxSotw : public GrpcMuxImpl&) override { + NOT_IMPLEMENTED_GCOVR_EXCL_LINE; + } + GrpcStream& grpcStreamForTest() { From d9e39665e0a8c7a28bd8812262e88e0fd7b53309 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Thu, 22 Jul 2021 14:40:52 -0700 Subject: [PATCH 12/46] Added grpc_mux_impl.h to code coverage exceptions with 69.8 coverage Signed-off-by: Dmitri Dolguikh --- test/per_file_coverage.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/test/per_file_coverage.sh b/test/per_file_coverage.sh index 967c1f81a94e..ede11ffb58f9 100755 --- a/test/per_file_coverage.sh +++ b/test/per_file_coverage.sh @@ -21,6 +21,7 @@ declare -a KNOWN_LOW_COVERAGE=( "source/common/quic:90.6" "source/common/tracing:96.1" "source/common/watchdog:42.9" # Death tests don't report LCOV +"source/common/config/xds_mux/grpc_mux_impl.h:69.8" "source/exe:94.3" "source/extensions/common/crypto:91.5" "source/extensions/common/tap:95.9" From 421b56efd099644409be0b849ae46f4f25c6e343 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Thu, 22 Jul 2021 17:37:53 -0700 Subject: [PATCH 13/46] Fixed per-extension coverage for xds_mux Signed-off-by: Dmitri Dolguikh --- test/per_file_coverage.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/per_file_coverage.sh b/test/per_file_coverage.sh index ede11ffb58f9..ac98612b9002 100755 --- a/test/per_file_coverage.sh +++ b/test/per_file_coverage.sh @@ -21,7 +21,7 @@ declare -a KNOWN_LOW_COVERAGE=( "source/common/quic:90.6" "source/common/tracing:96.1" "source/common/watchdog:42.9" # Death tests don't report LCOV -"source/common/config/xds_mux/grpc_mux_impl.h:69.8" +"source/common/config/xds_mux:95.7" "source/exe:94.3" "source/extensions/common/crypto:91.5" "source/extensions/common/tap:95.9" From 284cdcabb95ed9d337eb23f6b1c6b9a262e30fe9 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 27 Jul 2021 17:05:56 -0700 Subject: [PATCH 14/46] Moved grpc_stream_ from concrete classes into the base Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 21 +++++--- source/common/config/xds_mux/grpc_mux_impl.h | 54 +++++-------------- 2 files changed, 26 insertions(+), 49 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 0c9539153db6..3055106926d1 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -21,8 +21,15 @@ template GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, - envoy::config::core::v3::ApiVersion transport_api_version) - : subscription_state_factory_(std::move(subscription_state_factory)), + envoy::config::core::v3::ApiVersion transport_api_version, + Grpc::RawAsyncClientPtr&& async_client, + Event::Dispatcher& dispatcher, + const Protobuf::MethodDescriptor& service_method, + Random::RandomGenerator& random, Stats::Scope& scope, + const RateLimitSettings& rate_limit_settings) + : grpc_stream_(this, std::move(async_client), service_method, random, dispatcher, scope, + rate_limit_settings), + subscription_state_factory_(std::move(subscription_state_factory)), skip_subsequent_node_(skip_subsequent_node), local_info_(local_info), dynamic_update_callback_handle_(local_info.contextProvider().addDynamicContextUpdateCallback( [this](absl::string_view resource_type_url) { @@ -335,9 +342,8 @@ GrpcMuxDelta::GrpcMuxDelta(Grpc::RawAsyncClientPtr&& async_client, Event::Dispat const RateLimitSettings& rate_limit_settings, const LocalInfo::LocalInfo& local_info, bool skip_subsequent_node) : GrpcMuxImpl(std::make_unique(dispatcher), skip_subsequent_node, - local_info, transport_api_version), - grpc_stream_(this, std::move(async_client), service_method, random, dispatcher, scope, - rate_limit_settings) {} + local_info, transport_api_version, std::move(async_client), dispatcher, + service_method, random, scope, rate_limit_settings) {} // GrpcStreamCallbacks for GrpcMuxDelta void GrpcMuxDelta::requestOnDemandUpdate(const std::string& type_url, @@ -357,9 +363,8 @@ GrpcMuxSotw::GrpcMuxSotw(Grpc::RawAsyncClientPtr&& async_client, Event::Dispatch const RateLimitSettings& rate_limit_settings, const LocalInfo::LocalInfo& local_info, bool skip_subsequent_node) : GrpcMuxImpl(std::make_unique(dispatcher), skip_subsequent_node, - local_info, transport_api_version), - grpc_stream_(this, std::move(async_client), service_method, random, dispatcher, scope, - rate_limit_settings) {} + local_info, transport_api_version, std::move(async_client), dispatcher, + service_method, random, scope, rate_limit_settings) {} Config::GrpcMuxWatchPtr NullGrpcMuxImpl::addWatch(const std::string&, const absl::flat_hash_set&, diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 35413dee61ae..ab2f2d759329 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -50,7 +50,10 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, public: GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, - envoy::config::core::v3::ApiVersion transport_api_version); + envoy::config::core::v3::ApiVersion transport_api_version, + Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, + const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, + Stats::Scope& scope, const RateLimitSettings& rate_limit_settings); // TODO (dmitri-d) return a naked pointer instead of the wrapper once the legacy mux has been // removed and the mux interface can be changed @@ -82,14 +85,15 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, genericHandleResponse(message->type_url(), *message, control_plane_stats); } + GrpcStream& grpcStreamForTest() { return grpc_stream_; } + protected: - virtual GrpcStreamBase& grpcStream() PURE; - void establishGrpcStream() { grpcStream().establishNewStream(); } + void establishGrpcStream() { grpc_stream_.establishNewStream(); } void sendGrpcMessage(RQ& msg_proto, S& sub_state); - void maybeUpdateQueueSizeStat(uint64_t size) { grpcStream().maybeUpdateQueueSizeStat(size); } - bool grpcStreamAvailable() { return grpcStream().grpcStreamAvailable(); } - bool rateLimitAllowsDrain() { return grpcStream().checkRateLimitAllowsDrain(); } - virtual void sendMessage(RQ& msg_proto) PURE; + void maybeUpdateQueueSizeStat(uint64_t size) { grpc_stream_.maybeUpdateQueueSizeStat(size); } + bool grpcStreamAvailable() { return grpc_stream_.grpcStreamAvailable(); } + bool rateLimitAllowsDrain() { return grpc_stream_.checkRateLimitAllowsDrain(); } + void sendMessage(RQ& msg_proto) { grpc_stream_.sendMessage(msg_proto); } S& subscriptionStateFor(const std::string& type_url); WatchMap& watchMapFor(const std::string& type_url); @@ -108,6 +112,8 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, return transport_api_version_; } + GrpcStream grpc_stream_; + private: // Checks whether external conditions allow sending a DeltaDiscoveryRequest. (Does not check // whether we *want* to send a DeltaDiscoveryRequest). @@ -173,23 +179,6 @@ class GrpcMuxDelta : public GrpcMuxImpl& for_update) override; - - GrpcStream& - grpcStreamForTest() { - return grpc_stream_; - } - -protected: - GrpcStreamBase& grpcStream() override { return grpc_stream_; } - void sendMessage(envoy::service::discovery::v3::DeltaDiscoveryRequest& msg_proto) override { - grpc_stream_.sendMessage(msg_proto); - } - -private: - GrpcStream - grpc_stream_; }; class GrpcMuxSotw : public GrpcMuxImpl&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } - - GrpcStream& - grpcStreamForTest() { - return grpc_stream_; - } - -protected: - GrpcStreamBase& grpcStream() override { return grpc_stream_; } - void sendMessage(envoy::service::discovery::v3::DiscoveryRequest& msg_proto) override { - grpc_stream_.sendMessage(msg_proto); - } - -private: - GrpcStream - grpc_stream_; }; class NullGrpcMuxImpl : public GrpcMux { From 1fd1c368806b48ab0b8b0d81993ebe26a8108e67 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 27 Jul 2021 17:23:11 -0700 Subject: [PATCH 15/46] removed paused() method from mux interface Signed-off-by: Dmitri Dolguikh --- envoy/config/grpc_mux.h | 8 -------- source/common/config/grpc_mux_impl.h | 3 --- source/common/config/new_grpc_mux_impl.h | 4 ---- source/common/config/xds_mux/grpc_mux_impl.cc | 5 ----- source/common/config/xds_mux/grpc_mux_impl.h | 2 -- test/common/config/xds_grpc_mux_impl_test.cc | 7 ------- 6 files changed, 29 deletions(-) diff --git a/envoy/config/grpc_mux.h b/envoy/config/grpc_mux.h index c4fe26a6454e..e976e43df3b8 100644 --- a/envoy/config/grpc_mux.h +++ b/envoy/config/grpc_mux.h @@ -127,14 +127,6 @@ class GrpcMux { */ virtual void removeWatch(const std::string& type_url, Watch* watch) PURE; - /** - * Retrieves the current pause state as set by pause()/resume(). - * @param type_url type URL corresponding to xDS API, e.g. - * type.googleapis.com/envoy.api.v2.Cluster - * @return bool whether the API is paused. - */ - virtual bool paused(const std::string& type_url) const PURE; - virtual bool isUnified() const { return false; } }; diff --git a/source/common/config/grpc_mux_impl.h b/source/common/config/grpc_mux_impl.h index cf3a50c4f6ca..c8cd80e67804 100644 --- a/source/common/config/grpc_mux_impl.h +++ b/source/common/config/grpc_mux_impl.h @@ -88,8 +88,6 @@ class GrpcMuxImpl : public GrpcMux, void setRetryTimer(); void sendDiscoveryRequest(const std::string& type_url); - bool paused(const std::string&) const override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } - void removeWatch(const std::string&, Watch*) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, const SubscriptionOptions&) override { @@ -224,7 +222,6 @@ class NullGrpcMuxImpl : public GrpcMux, ExceptionUtil::throwEnvoyException("ADS must be configured to support an ADS config source"); } - bool paused(const std::string&) const override { return false; } void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, const SubscriptionOptions&) override; void removeWatch(const std::string&, Watch*) override; diff --git a/source/common/config/new_grpc_mux_impl.h b/source/common/config/new_grpc_mux_impl.h index 404c86449d59..ba03137b5957 100644 --- a/source/common/config/new_grpc_mux_impl.h +++ b/source/common/config/new_grpc_mux_impl.h @@ -76,10 +76,6 @@ class NewGrpcMuxImpl // TODO(fredlas) remove this from the GrpcMux interface. void start() override; - bool paused(const std::string& type_url) const override { - return !pausable_ack_queue_.paused(type_url); - } - GrpcStream& grpcStreamForTest() { diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 3055106926d1..8d3da670d42f 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -145,11 +145,6 @@ ScopedResume GrpcMuxImpl::pause(const std::vector typ }); } -template -bool GrpcMuxImpl::paused(const std::string& type_url) const { - return pausable_ack_queue_.paused(type_url); -} - template void GrpcMuxImpl::sendGrpcMessage(RQ& msg_proto, S& sub_state) { if (sub_state.dynamicContextChanged() || !anyRequestSentYetInCurrentStream() || diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index ab2f2d759329..effcfac5f07e 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -69,7 +69,6 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, ScopedResume pause(const std::string& type_url) override; ScopedResume pause(const std::vector type_urls) override; - bool paused(const std::string& type_url) const override; void start() override; const absl::flat_hash_map>& subscriptions() const { return subscriptions_; @@ -208,7 +207,6 @@ class NullGrpcMuxImpl : public GrpcMux { ScopedResume pause(const std::vector) override { return std::make_unique([]() {}); } - bool paused(const std::string&) const override { return false; } void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, const SubscriptionOptions&) override; diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc index 978306b9a6a9..12e96dc53614 100644 --- a/test/common/config/xds_grpc_mux_impl_test.cc +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -223,13 +223,6 @@ TEST_F(GrpcMuxImplTest, ResetStream) { expectSendMessage("type_url_foo", {}, ""); } -// Validate paused() behaviour. -TEST_F(GrpcMuxImplTest, Paused) { - setup(); - ScopedResume a = grpc_mux_->pause("type_url_foo"); - EXPECT_TRUE(grpc_mux_->paused("type_url_foo")); -} - // Validate pause-resume behavior. TEST_F(GrpcMuxImplTest, PauseResume) { setup(); From a11a2dc8f269d7b649ec8a2440c82388bf52633f Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 27 Jul 2021 17:29:37 -0700 Subject: [PATCH 16/46] Removed BaseGrpcStream interface Signed-off-by: Dmitri Dolguikh --- source/common/config/grpc_stream.h | 20 +++++--------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/source/common/config/grpc_stream.h b/source/common/config/grpc_stream.h index d549fda3de96..23ee9a0d287b 100644 --- a/source/common/config/grpc_stream.h +++ b/source/common/config/grpc_stream.h @@ -25,21 +25,11 @@ constexpr uint32_t RetryMaxDelayMs = 30000; // Do not cross more than 30s template using ResponseProtoPtr = std::unique_ptr; -class GrpcStreamBase { -public: - virtual ~GrpcStreamBase() = default; - virtual void establishNewStream() PURE; - virtual void maybeUpdateQueueSizeStat(uint64_t size) PURE; - virtual bool grpcStreamAvailable() const PURE; - virtual bool checkRateLimitAllowsDrain() PURE; -}; - // Oversees communication for gRPC xDS implementations (parent to both regular xDS and delta // xDS variants). Reestablishes the gRPC channel when necessary, and provides rate limiting of // requests. template -class GrpcStream : public GrpcStreamBase, - public Grpc::AsyncStreamCallbacks, +class GrpcStream : public Grpc::AsyncStreamCallbacks, public Logger::Loggable { public: GrpcStream(GrpcStreamCallbacks* callbacks, Grpc::RawAsyncClientPtr async_client, @@ -67,7 +57,7 @@ class GrpcStream : public GrpcStreamBase, RetryInitialDelayMs, RetryMaxDelayMs, random_); } - void establishNewStream() override { + void establishNewStream() { ENVOY_LOG(debug, "Establishing new gRPC bidi stream for {}", service_method_.DebugString()); if (stream_ != nullptr) { ENVOY_LOG(warn, "gRPC bidi stream for {} already exists!", service_method_.DebugString()); @@ -84,7 +74,7 @@ class GrpcStream : public GrpcStreamBase, callbacks_->onStreamEstablished(); } - bool grpcStreamAvailable() const override { return stream_ != nullptr; } + bool grpcStreamAvailable() const { return stream_ != nullptr; } void sendMessage(const RequestProto& request) { stream_->sendMessage(request, false); } @@ -122,7 +112,7 @@ class GrpcStream : public GrpcStreamBase, setRetryTimer(); } - void maybeUpdateQueueSizeStat(uint64_t size) override { + void maybeUpdateQueueSizeStat(uint64_t size) { // Although request_queue_.push() happens elsewhere, the only time the queue is non-transiently // non-empty is when it remains non-empty after a drain attempt. (The push() doesn't matter // because we always attempt this drain immediately after the push). Basically, a change in @@ -135,7 +125,7 @@ class GrpcStream : public GrpcStreamBase, } } - bool checkRateLimitAllowsDrain() override { + bool checkRateLimitAllowsDrain() { if (!rate_limiting_enabled_ || limit_request_->consume(1, false)) { return true; } From 52a2b1a7924dcb124687ce9cf301d0d851f6e840 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 28 Jul 2021 12:28:11 -0700 Subject: [PATCH 17/46] Ported support for mux shutdown Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 42 +++++++++++++++++++ source/common/config/xds_mux/grpc_mux_impl.h | 23 +++++++++- test/common/config/new_grpc_mux_impl_test.cc | 22 ++++++++++ 3 files changed, 86 insertions(+), 1 deletion(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 8d3da670d42f..efeeb5e3b31c 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -17,6 +17,35 @@ namespace Envoy { namespace Config { namespace XdsMux { +namespace { +class AllMuxesState { +public: + void insert(ShutdownableMux* mux) { + absl::WriterMutexLock locker(&lock_); + muxes_.insert(mux); + } + + void erase(ShutdownableMux* mux) { + absl::WriterMutexLock locker(&lock_); + muxes_.erase(mux); + } + + void shutdownAll() { + absl::WriterMutexLock locker(&lock_); + for (auto& mux : muxes_) { + mux->shutdown(); + } + } + +private: + absl::flat_hash_set muxes_ ABSL_GUARDED_BY(lock_); + + // TODO(ggreenway): can this lock be removed? Is this code only run on the main thread? + absl::Mutex lock_; +}; +using AllMuxes = ThreadSafeSingleton; +} // namespace + template GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, @@ -37,6 +66,15 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac })), transport_api_version_(transport_api_version) { Config::Utility::checkLocalInfo("ads", local_info); + AllMuxes::get().insert(this); +} + +template GrpcMuxImpl::~GrpcMuxImpl() { + AllMuxes::get().erase(this); +} + +template void GrpcMuxImpl::shutdownAll() { + AllMuxes::get().shutdownAll(); } template @@ -244,6 +282,10 @@ WatchMap& GrpcMuxImpl::watchMapFor(const std::string& type_url) { template void GrpcMuxImpl::trySendDiscoveryRequests() { + if (shutdown_) { + return; + } + while (true) { // Do any of our subscriptions even want to send a request? absl::optional request_type_if_any = whoWantsToSendDiscoveryRequest(); diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index effcfac5f07e..c591746d5981 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -30,6 +30,12 @@ namespace Envoy { namespace Config { namespace XdsMux { +class ShutdownableMux { +public: + virtual ~ShutdownableMux() = default; + virtual void shutdown() PURE; +}; + class WatchCompatibilityWrapper : public Envoy::Config::GrpcMuxWatch { public: WatchCompatibilityWrapper(Watch* watch) : watch_(watch) {} @@ -46,6 +52,7 @@ class WatchCompatibilityWrapper : public Envoy::Config::GrpcMuxWatch { template class GrpcMuxImpl : public GrpcStreamCallbacks, public GrpcMux, + public ShutdownableMux, Logger::Loggable { public: GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, @@ -55,6 +62,17 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, Stats::Scope& scope, const RateLimitSettings& rate_limit_settings); + ~GrpcMuxImpl() override; + + // Causes all NewGrpcMuxImpl objects to stop sending any messages on `grpc_stream_` to fix a crash + // on Envoy shutdown due to dangling pointers. This may not be the ideal fix; it is probably + // preferable for the `ServerImpl` to cause all configuration subscriptions to be shutdown, which + // would then cause all `NewGrpcMuxImpl` to be destructed. + // TODO: figure out the correct fix: https://github.com/envoyproxy/envoy/issues/15072. + static void shutdownAll(); + + void shutdown() override { shutdown_ = true; } + // TODO (dmitri-d) return a naked pointer instead of the wrapper once the legacy mux has been // removed and the mux interface can be changed Config::GrpcMuxWatchPtr addWatch(const std::string& type_url, @@ -160,8 +178,11 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, // this one is up to GrpcMux. const LocalInfo::LocalInfo& local_info_; Common::CallbackHandlePtr dynamic_update_callback_handle_; - const envoy::config::core::v3::ApiVersion transport_api_version_; + + // True iff Envoy is shutting down; no messages should be sent on the `grpc_stream_` when this is + // true because it may contain dangling pointers. + std::atomic shutdown_{false}; }; class GrpcMuxDelta : public GrpcMuxImpl { ->onDiscoveryResponse(std::move(response), control_plane_stats_); } + void shutdownMux() { + if (isUnifiedMuxTest()) { + dynamic_cast(grpc_mux_.get())->shutdown(); + return; + } + dynamic_cast(grpc_mux_.get())->shutdown(); + } + // the code is duplicated here, but all calls other than the check in return statement, return // different types. bool subscriptionExists(const std::string& type_url) const { @@ -546,6 +554,20 @@ TEST_P(NewGrpcMuxImplTest, RequestOnDemandUpdate) { } } +TEST_P(NewGrpcMuxImplTest, Shutdown) { + setup(); + InSequence s; + auto foo_sub = grpc_mux_->addWatch("foo", {"x", "y"}, callbacks_, resource_decoder_, {}); + EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); + expectSendMessage("foo", {"x", "y"}, {}); + grpc_mux_->start(); + + shutdownMux(); + auto bar_sub = grpc_mux_->addWatch("bar", {"z"}, callbacks_, resource_decoder_, {}); + // We do not expect any messages to be sent here as the mux has been shutdown + // There won't be any unsubscribe messages for the legacy mux either for the same reason +} + } // namespace } // namespace Config } // namespace Envoy From d1562732e8785623e3428a5eb302324aaaae2dbe Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 28 Jul 2021 13:00:07 -0700 Subject: [PATCH 18/46] Added comments re: GrpcMuxImpl template parameters Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.h | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index c591746d5981..48677b5071f3 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -49,6 +49,15 @@ class WatchCompatibilityWrapper : public Envoy::Config::GrpcMuxWatch { // This class owns the GrpcStream used to talk to the server, maintains queuing // logic to properly order the subscription(s)' various messages, and allows // starting/stopping/pausing of the subscriptions. +// +// @tparam S SubscriptionState state type, either SotwSubscriptionState or DeltaSubscriptionState +// @tparam F SubscriptionStateFactory type, either SotwSubscriptionStateFactory or +// DeltaSubscriptionStateFactory +// @tparam RQ Xds request type, either envoy::service::discovery::v3::DiscoveryRequest or +// envoy::service::discovery::v3::DeltaDiscoveryRequest +// @tparam RS Xds response type, either envoy::service::discovery::v3::DiscoveryResponse or +// envoy::service::discovery::v3::DeltaDiscoveryResponse +// template class GrpcMuxImpl : public GrpcStreamCallbacks, public GrpcMux, From 4fabd097f170b9d6d3ec4e2fc78d3907f3adc1f4 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Thu, 29 Jul 2021 14:29:49 -0700 Subject: [PATCH 19/46] Adjusted test coverage for xds_mux Signed-off-by: Dmitri Dolguikh --- test/per_file_coverage.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/per_file_coverage.sh b/test/per_file_coverage.sh index 36680439db55..f151447899ea 100755 --- a/test/per_file_coverage.sh +++ b/test/per_file_coverage.sh @@ -21,7 +21,7 @@ declare -a KNOWN_LOW_COVERAGE=( "source/common/quic:90.6" "source/common/tracing:96.1" "source/common/watchdog:42.9" # Death tests don't report LCOV -"source/common/config/xds_mux:95.7" +"source/common/config/xds_mux:94.5" "source/exe:94.3" "source/extensions/common/crypto:91.5" "source/extensions/common/tap:95.9" From 026d42d871f86d1ca520ed2226f6f8f49e47b08d Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 3 Aug 2021 15:03:48 -0700 Subject: [PATCH 20/46] Reponded to feedback Signed-off-by: Dmitri Dolguikh --- envoy/config/grpc_mux.h | 22 +--- source/common/config/grpc_mux_impl.cc | 10 -- source/common/config/grpc_mux_impl.h | 9 -- source/common/config/new_grpc_mux_impl.h | 4 +- source/common/config/xds_mux/grpc_mux_impl.cc | 14 +-- source/common/config/xds_mux/grpc_mux_impl.h | 62 ++++++----- test/common/config/new_grpc_mux_impl_test.cc | 30 ++---- test/common/config/xds_grpc_mux_impl_test.cc | 100 ++++++++---------- test/mocks/config/mocks.h | 7 -- 9 files changed, 92 insertions(+), 166 deletions(-) diff --git a/envoy/config/grpc_mux.h b/envoy/config/grpc_mux.h index e976e43df3b8..f3c5d7c00808 100644 --- a/envoy/config/grpc_mux.h +++ b/envoy/config/grpc_mux.h @@ -31,11 +31,6 @@ struct ControlPlaneStats { GENERATE_TEXT_READOUT_STRUCT) }; -// TODO (dmitri-d) This forward declaration is needed in order for the unified mux to fulfill legacy -// mux interface. It should be removed together with the rest of legacy mux code when the switch to -// unified mux has been finalized. -struct Watch; - /** * Handle on a muxed gRPC subscription. The subscription is canceled on destruction. */ @@ -111,22 +106,7 @@ class GrpcMux { virtual void requestOnDemandUpdate(const std::string& type_url, const absl::flat_hash_set& for_update) PURE; - // Unified mux interface starts here - // Updates the list of resource names watched by the given watch. If an added name is new across - // the whole subscription, or if a removed name has no other watch interested in it, then the - // subscription will enqueue and attempt to send an appropriate discovery request. - virtual void updateWatch(const std::string& type_url, Watch* watch, - const absl::flat_hash_set& resources, - const SubscriptionOptions& options) PURE; - - /** - * Cleanup of a Watch* added by addOrUpdateWatch(). Receiving a Watch* from addOrUpdateWatch() - * makes you responsible for eventually invoking this cleanup. - * @param type_url type URL corresponding to xDS API e.g. type.googleapis.com/envoy.api.v2.Cluster - * @param watch the watch to be cleaned up. - */ - virtual void removeWatch(const std::string& type_url, Watch* watch) PURE; - + // TODO (dmitri-d) remove this when legacy muxes have been removed virtual bool isUnified() const { return false; } }; diff --git a/source/common/config/grpc_mux_impl.cc b/source/common/config/grpc_mux_impl.cc index 6f2d0d131956..060f0c845796 100644 --- a/source/common/config/grpc_mux_impl.cc +++ b/source/common/config/grpc_mux_impl.cc @@ -381,15 +381,5 @@ void GrpcMuxImpl::drainRequests() { grpc_stream_.maybeUpdateQueueSizeStat(request_queue_->size()); } -void NullGrpcMuxImpl::updateWatch(const std::string&, Watch*, - const absl::flat_hash_set&, - const SubscriptionOptions&) { - throw EnvoyException("ADS must be configured to support an ADS config source"); -} - -void NullGrpcMuxImpl::removeWatch(const std::string&, Watch*) { - throw EnvoyException("ADS must be configured to support an ADS config source"); -} - } // namespace Config } // namespace Envoy diff --git a/source/common/config/grpc_mux_impl.h b/source/common/config/grpc_mux_impl.h index c8cd80e67804..57b494609937 100644 --- a/source/common/config/grpc_mux_impl.h +++ b/source/common/config/grpc_mux_impl.h @@ -88,12 +88,6 @@ class GrpcMuxImpl : public GrpcMux, void setRetryTimer(); void sendDiscoveryRequest(const std::string& type_url); - void removeWatch(const std::string&, Watch*) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } - void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, - const SubscriptionOptions&) override { - NOT_IMPLEMENTED_GCOVR_EXCL_LINE; - } - struct GrpcMuxWatchImpl : public GrpcMuxWatch { GrpcMuxWatchImpl(const absl::flat_hash_set& resources, SubscriptionCallbacks& callbacks, OpaqueResourceDecoder& resource_decoder, @@ -222,9 +216,6 @@ class NullGrpcMuxImpl : public GrpcMux, ExceptionUtil::throwEnvoyException("ADS must be configured to support an ADS config source"); } - void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, - const SubscriptionOptions&) override; - void removeWatch(const std::string&, Watch*) override; void requestOnDemandUpdate(const std::string&, const absl::flat_hash_set&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } diff --git a/source/common/config/new_grpc_mux_impl.h b/source/common/config/new_grpc_mux_impl.h index ba03137b5957..98ded0dec357 100644 --- a/source/common/config/new_grpc_mux_impl.h +++ b/source/common/config/new_grpc_mux_impl.h @@ -131,14 +131,14 @@ class NewGrpcMuxImpl const SubscriptionOptions options_; }; - void removeWatch(const std::string& type_url, Watch* watch) override; + void removeWatch(const std::string& type_url, Watch* watch); // Updates the list of resource names watched by the given watch. If an added name is new across // the whole subscription, or if a removed name has no other watch interested in it, then the // subscription will enqueue and attempt to send an appropriate discovery request. void updateWatch(const std::string& type_url, Watch* watch, const absl::flat_hash_set& resources, - const SubscriptionOptions& options) override; + const SubscriptionOptions& options); // Adds a subscription for the type_url to the subscriptions map and order list. void addSubscription(const std::string& type_url, bool use_namespace_matching, diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index efeeb5e3b31c..5fd071837ca6 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -108,9 +108,7 @@ Config::GrpcMuxWatchPtr GrpcMuxImpl::addWatch( Watch* watch = watch_map->second->addWatch(callbacks, resource_decoder); // updateWatch() queues a discovery request if any of 'resources' are not yet subscribed. updateWatch(type_url, watch, resources, options); - // TODO (dmitri-d) return naked pointer instead once legacy mux has been removed and mux interface - // can be updated - return std::make_unique(watch); + return std::make_unique(type_url, watch, *this, options); } // Updates the list of resource names watched by the given watch. If an added name is new across @@ -410,16 +408,6 @@ Config::GrpcMuxWatchPtr NullGrpcMuxImpl::addWatch(const std::string&, throw EnvoyException("ADS must be configured to support an ADS config source"); } -void NullGrpcMuxImpl::updateWatch(const std::string&, Watch*, - const absl::flat_hash_set&, - const SubscriptionOptions&) { - throw EnvoyException("ADS must be configured to support an ADS config source"); -} - -void NullGrpcMuxImpl::removeWatch(const std::string&, Watch*) { - throw EnvoyException("ADS must be configured to support an ADS config source"); -} - } // namespace XdsMux } // namespace Config } // namespace Envoy diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 48677b5071f3..d439caf698c9 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -36,14 +36,6 @@ class ShutdownableMux { virtual void shutdown() PURE; }; -class WatchCompatibilityWrapper : public Envoy::Config::GrpcMuxWatch { -public: - WatchCompatibilityWrapper(Watch* watch) : watch_(watch) {} - void update(const absl::flat_hash_set&) override { NOT_IMPLEMENTED_GCOVR_EXCL_LINE; } - - Watch* watch_; -}; - // Manages subscriptions to one or more type of resource. The logical protocol // state of those subscription(s) is handled by SubscriptionState. // This class owns the GrpcStream used to talk to the server, maintains queuing @@ -73,10 +65,10 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, ~GrpcMuxImpl() override; - // Causes all NewGrpcMuxImpl objects to stop sending any messages on `grpc_stream_` to fix a crash + // Causes all GrpcMuxImpl objects to stop sending any messages on `grpc_stream_` to fix a crash // on Envoy shutdown due to dangling pointers. This may not be the ideal fix; it is probably // preferable for the `ServerImpl` to cause all configuration subscriptions to be shutdown, which - // would then cause all `NewGrpcMuxImpl` to be destructed. + // would then cause all `GrpcMuxImpl` to be destructed. // TODO: figure out the correct fix: https://github.com/envoyproxy/envoy/issues/15072. static void shutdownAll(); @@ -91,8 +83,8 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, const SubscriptionOptions& options) override; void updateWatch(const std::string& type_url, Watch* watch, const absl::flat_hash_set& resources, - const SubscriptionOptions& options) override; - void removeWatch(const std::string& type_url, Watch* watch) override; + const SubscriptionOptions& options); + void removeWatch(const std::string& type_url, Watch* watch); ScopedResume pause(const std::string& type_url) override; ScopedResume pause(const std::vector type_urls) override; @@ -114,6 +106,32 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, GrpcStream& grpcStreamForTest() { return grpc_stream_; } protected: + class WatchImpl : public Envoy::Config::GrpcMuxWatch { + public: + WatchImpl(const std::string& type_url, Watch* watch, GrpcMuxImpl& parent, + const SubscriptionOptions& options) + : type_url_(type_url), watch_(watch), parent_(parent), options_(options) {} + + ~WatchImpl() override { remove(); } + + void remove() { + if (watch_) { + parent_.removeWatch(type_url_, watch_); + watch_ = nullptr; + } + } + + void update(const absl::flat_hash_set& resources) override { + parent_.updateWatch(type_url_, watch_, resources, options_); + } + + private: + const std::string type_url_; + Watch* watch_; + GrpcMuxImpl& parent_; + const SubscriptionOptions options_; + }; + void establishGrpcStream() { grpc_stream_.establishNewStream(); } void sendGrpcMessage(RQ& msg_proto, S& sub_state); void maybeUpdateQueueSizeStat(uint64_t size) { grpc_stream_.maybeUpdateQueueSizeStat(size); } @@ -142,15 +160,15 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, private: // Checks whether external conditions allow sending a DeltaDiscoveryRequest. (Does not check - // whether we *want* to send a DeltaDiscoveryRequest). + // whether we *want* to send a (Delta)DiscoveryRequest). bool canSendDiscoveryRequest(const std::string& type_url); - // Checks whether we have something to say in a DeltaDiscoveryRequest, which can be an ACK and/or - // a subscription update. (Does not check whether we *can* send that DeltaDiscoveryRequest). - // Returns the type_url we should send the DeltaDiscoveryRequest for (if any). - // First, prioritizes ACKs over non-ACK subscription interest updates. - // Then, prioritizes non-ACK updates in the order the various types - // of subscriptions were activated (as tracked by subscription_ordering_). + // Checks whether we have something to say in a (Delta)DiscoveryRequest, which can be an ACK + // and/or a subscription update. (Does not check whether we *can* send that + // (Delta)DiscoveryRequest). Returns the type_url we should send the DeltaDiscoveryRequest for (if + // any). First, prioritizes ACKs over non-ACK subscription interest updates. Then, prioritizes + // non-ACK updates in the order the various types of subscriptions were activated (as tracked by + // subscription_ordering_). absl::optional whoWantsToSendDiscoveryRequest(); // Invoked when dynamic context parameters change for a resource type. @@ -181,7 +199,7 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, // State to help with skip_subsequent_node's logic. bool any_request_sent_yet_in_current_stream_{}; - // Used to populate the [Delta]DiscoveryRequest's node field. That field is the same across + // Used to populate the (Delta)DiscoveryRequest's node field. That field is the same across // all type_urls, and moreover, the 'skip_subsequent_node' logic needs to operate across all // the type_urls. So, while the SubscriptionStates populate every other field of these messages, // this one is up to GrpcMux. @@ -238,10 +256,6 @@ class NullGrpcMuxImpl : public GrpcMux { return std::make_unique([]() {}); } - void updateWatch(const std::string&, Watch*, const absl::flat_hash_set&, - const SubscriptionOptions&) override; - void removeWatch(const std::string&, Watch*) override; - Config::GrpcMuxWatchPtr addWatch(const std::string&, const absl::flat_hash_set&, SubscriptionCallbacks&, OpaqueResourceDecoder&, const SubscriptionOptions&) override; diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index c3faf33d62c0..f9a8747b0ce6 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -192,11 +192,8 @@ TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { // Update to bar type should resend Node. expectSendMessage("bar", {}, {}); local_info_.context_provider_.update_cb_handler_.runCallbacks("bar"); - if (!isUnifiedMuxTest()) { - // in "legacy" delta mux implementation destruction of "foo_sub" - // results in an unsubscribe message. - expectSendMessage("foo", {}, {"x", "y"}); - } + + expectSendMessage("foo", {}, {"x", "y"}); } // Validate cached nonces are cleared on reconnection. @@ -246,11 +243,8 @@ TEST_P(NewGrpcMuxImplTest, ReconnectionResetsNonceAndAcks) { expectSendMessage(type_url, {"x", "y"}, {}, "", Grpc::Status::WellKnownGrpcStatus::Ok, "", {{"x", "2000"}, {"y", "3000"}}); remoteClose(); - if (!isUnifiedMuxTest()) { - // in legacy implementation, destruction of the EDS subscription will issue an "unsubscribe" - // request. - expectSendMessage(type_url, {}, {"x", "y"}); - } + + expectSendMessage(type_url, {}, {"x", "y"}); } // Validate resources are not sent on wildcard watch reconnection. @@ -406,14 +400,8 @@ TEST_P(NewGrpcMuxImplTest, ConfigUpdateWithAliases) { EXPECT_LOG_CONTAINS("debug", "for " + type_url + " from HAL 9000", onDiscoveryResponse(std::move(response))); EXPECT_TRUE(subscriptionExists(type_url)); - if (isUnifiedMuxTest()) { - dynamic_cast(grpc_mux_.get()) - ->updateWatch(type_url, - dynamic_cast(watch.get())->watch_, {}, - options); - } else { - watch->update({}); - } + watch->update({}); + EXPECT_EQ("HAL 9000", stats_.textReadout("control_plane.identifier").value()); } @@ -547,11 +535,7 @@ TEST_P(NewGrpcMuxImplTest, RequestOnDemandUpdate) { expectSendMessage("foo", {"z"}, {}); grpc_mux_->requestOnDemandUpdate("foo", {"z"}); - if (!isUnifiedMuxTest()) { - // in legacy implementation, destruction of the EDS subscription will issue an "unsubscribe" - // request. - expectSendMessage("foo", {}, {"x", "y"}); - } + expectSendMessage("foo", {}, {"x", "y"}); } TEST_P(NewGrpcMuxImplTest, Shutdown) { diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc index 12e96dc53614..aab7f47c1439 100644 --- a/test/common/config/xds_grpc_mux_impl_test.cc +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -104,37 +104,16 @@ class GrpcMuxImplTestBase : public testing::Test { sendMessageRaw_(Grpc::ProtoBufferEqIgnoreRepeatedFieldOrdering(expected_request), false)); } - // These tests were written around GrpcMuxWatch, an RAII type returned by the old subscribe(). - // To preserve these tests for the new code, we need an RAII watch handler. That is - // GrpcSubscriptionImpl, but to keep things simple, we'll fake it. (What we really care about - // is the destructor, which is identical to the real one). - class FakeGrpcSubscription { - public: - FakeGrpcSubscription(GrpcMux* grpc_mux, std::string type_url, Config::GrpcMuxWatchPtr watch) - : grpc_mux_(grpc_mux), type_url_(std::move(type_url)), - watch_(dynamic_cast(watch.get())->watch_) {} - ~FakeGrpcSubscription() { grpc_mux_->removeWatch(type_url_, watch_); } - - private: - GrpcMux* const grpc_mux_; - std::string type_url_; - Watch* const watch_; - }; - - FakeGrpcSubscription makeWatch(const std::string& type_url, - const absl::flat_hash_set& resources) { - return FakeGrpcSubscription( - grpc_mux_.get(), type_url, - grpc_mux_->addWatch(type_url, resources, callbacks_, resource_decoder_, {})); + Config::GrpcMuxWatchPtr makeWatch(const std::string& type_url, + const absl::flat_hash_set& resources) { + return grpc_mux_->addWatch(type_url, resources, callbacks_, resource_decoder_, {}); } - FakeGrpcSubscription makeWatch(const std::string& type_url, - const absl::flat_hash_set& resources, - NiceMock& callbacks, - Config::OpaqueResourceDecoder& resource_decoder) { - return FakeGrpcSubscription( - grpc_mux_.get(), type_url, - grpc_mux_->addWatch(type_url, resources, callbacks, resource_decoder, {})); + Config::GrpcMuxWatchPtr makeWatch(const std::string& type_url, + const absl::flat_hash_set& resources, + NiceMock& callbacks, + Config::OpaqueResourceDecoder& resource_decoder) { + return grpc_mux_->addWatch(type_url, resources, callbacks, resource_decoder, {}); } NiceMock dispatcher_; @@ -163,17 +142,17 @@ TEST_F(GrpcMuxImplTest, MultipleTypeUrlStreams) { setup(); InSequence s; - FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); - FakeGrpcSubscription bar_sub = makeWatch("type_url_bar", {}); + auto foo_sub = makeWatch("type_url_foo", {"x", "y"}); + auto bar_sub = makeWatch("type_url_bar", {}); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage("type_url_foo", {"x", "y"}, "", true); expectSendMessage("type_url_bar", {}, ""); grpc_mux_->start(); EXPECT_EQ(1, control_plane_connected_state_.value()); expectSendMessage("type_url_bar", {"z"}, ""); - FakeGrpcSubscription bar_z_sub = makeWatch("type_url_bar", {"z"}); + auto bar_z_sub = makeWatch("type_url_bar", {"z"}); expectSendMessage("type_url_bar", {"zz", "z"}, ""); - FakeGrpcSubscription bar_zz_sub = makeWatch("type_url_bar", {"zz"}); + auto bar_zz_sub = makeWatch("type_url_bar", {"zz"}); expectSendMessage("type_url_bar", {"z"}, ""); expectSendMessage("type_url_bar", {}, ""); expectSendMessage("type_url_foo", {}, ""); @@ -190,9 +169,9 @@ TEST_F(GrpcMuxImplTest, ResetStream) { new Event::MockTimer(&dispatcher_); setup(); - FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); - FakeGrpcSubscription bar_sub = makeWatch("type_url_bar", {}); - FakeGrpcSubscription baz_sub = makeWatch("type_url_baz", {"z"}); + auto foo_sub = makeWatch("type_url_foo", {"x", "y"}); + auto bar_sub = makeWatch("type_url_bar", {}); + auto baz_sub = makeWatch("type_url_baz", {"z"}); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage("type_url_foo", {"x", "y"}, "", true); expectSendMessage("type_url_bar", {}, ""); @@ -202,7 +181,7 @@ TEST_F(GrpcMuxImplTest, ResetStream) { // Send another message for foo so that the node is cleared in the cached request. // This is to test that the the node is set again in the first message below. expectSendMessage("type_url_foo", {"z", "x", "y"}, ""); - FakeGrpcSubscription foo_z_sub = makeWatch("type_url_foo", {"z"}); + auto foo_z_sub = makeWatch("type_url_foo", {"z"}); EXPECT_CALL(callbacks_, onConfigUpdateFailed(Envoy::Config::ConfigUpdateFailureReason::ConnectionFailure, _)) @@ -227,7 +206,10 @@ TEST_F(GrpcMuxImplTest, ResetStream) { TEST_F(GrpcMuxImplTest, PauseResume) { setup(); InSequence s; - grpc_mux_->addWatch("type_url_foo", {"x", "y"}, callbacks_, resource_decoder_, {}); + GrpcMuxWatchPtr foo1; + GrpcMuxWatchPtr foo2; + GrpcMuxWatchPtr foo3; + auto foo = grpc_mux_->addWatch("type_url_foo", {"x", "y"}, callbacks_, resource_decoder_, {}); { ScopedResume a = grpc_mux_->pause("type_url_foo"); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); @@ -237,20 +219,21 @@ TEST_F(GrpcMuxImplTest, PauseResume) { { ScopedResume a = grpc_mux_->pause("type_url_bar"); expectSendMessage("type_url_foo", {"z", "x", "y"}, ""); - grpc_mux_->addWatch("type_url_foo", {"z"}, callbacks_, resource_decoder_, {}); + foo1 = grpc_mux_->addWatch("type_url_foo", {"z"}, callbacks_, resource_decoder_, {}); } { ScopedResume a = grpc_mux_->pause("type_url_foo"); - grpc_mux_->addWatch("type_url_foo", {"zz"}, callbacks_, resource_decoder_, {}); + foo2 = grpc_mux_->addWatch("type_url_foo", {"zz"}, callbacks_, resource_decoder_, {}); expectSendMessage("type_url_foo", {"zz", "z", "x", "y"}, ""); } // When nesting, we only have a single resumption. { ScopedResume a = grpc_mux_->pause("type_url_foo"); ScopedResume b = grpc_mux_->pause("type_url_foo"); - grpc_mux_->addWatch("type_url_foo", {"zzz"}, callbacks_, resource_decoder_, {}); + foo3 = grpc_mux_->addWatch("type_url_foo", {"zzz"}, callbacks_, resource_decoder_, {}); expectSendMessage("type_url_foo", {"zzz", "zz", "z", "x", "y"}, ""); } + grpc_mux_->pause("type_url_foo")->cancel(); } @@ -260,7 +243,7 @@ TEST_F(GrpcMuxImplTest, TypeUrlMismatch) { auto invalid_response = std::make_unique(); InSequence s; - FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); + auto foo_sub = makeWatch("type_url_foo", {"x", "y"}); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage("type_url_foo", {"x", "y"}, "", true); @@ -300,7 +283,7 @@ TEST_F(GrpcMuxImplTest, RpcErrorMessageTruncated) { setup(); auto invalid_response = std::make_unique(); InSequence s; - FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x", "y"}); + auto foo_sub = makeWatch("type_url_foo", {"x", "y"}); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage("type_url_foo", {"x", "y"}, "", true); @@ -368,7 +351,7 @@ TEST_F(GrpcMuxImplTest, ResourceTTL) { const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; InSequence s; auto* ttl_timer = new Event::MockTimer(&dispatcher_); - FakeGrpcSubscription eds_sub = makeWatch(type_url, {"x"}, callbacks_, resource_decoder); + auto eds_sub = makeWatch(type_url, {"x"}, callbacks_, resource_decoder); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage(type_url, {"x"}, "", true); @@ -525,7 +508,7 @@ TEST_F(GrpcMuxImplTest, WildcardWatch) { setup(); const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; - FakeGrpcSubscription foo_sub = makeWatch(type_url, {}, callbacks_, resource_decoder_); + auto foo_sub = makeWatch(type_url, {}, callbacks_, resource_decoder_); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage(type_url, {}, "", true); grpc_mux_->start(); @@ -559,9 +542,9 @@ TEST_F(GrpcMuxImplTest, WatchDemux) { const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; NiceMock foo_callbacks; - FakeGrpcSubscription foo_sub = makeWatch(type_url, {"x", "y"}, foo_callbacks, resource_decoder_); + auto foo_sub = makeWatch(type_url, {"x", "y"}, foo_callbacks, resource_decoder_); NiceMock bar_callbacks; - FakeGrpcSubscription bar_sub = makeWatch(type_url, {"y", "z"}, bar_callbacks, resource_decoder_); + auto bar_sub = makeWatch(type_url, {"y", "z"}, bar_callbacks, resource_decoder_); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); // Should dedupe the "x" resource. expectSendMessage(type_url, {"y", "z", "x"}, "", true); @@ -644,7 +627,7 @@ TEST_F(GrpcMuxImplTest, MultipleWatcherWithEmptyUpdates) { InSequence s; const std::string& type_url = Config::TypeUrl::get().ClusterLoadAssignment; NiceMock foo_callbacks; - FakeGrpcSubscription foo_sub = makeWatch(type_url, {"x", "y"}, foo_callbacks, resource_decoder_); + auto foo_sub = makeWatch(type_url, {"x", "y"}, foo_callbacks, resource_decoder_); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage(type_url, {"x", "y"}, "", true); @@ -666,7 +649,7 @@ TEST_F(GrpcMuxImplTest, SingleWatcherWithEmptyUpdates) { setup(); const std::string& type_url = Config::TypeUrl::get().Cluster; NiceMock foo_callbacks; - FakeGrpcSubscription foo_sub = makeWatch(type_url, {}, foo_callbacks, resource_decoder_); + auto foo_sub = makeWatch(type_url, {}, foo_callbacks, resource_decoder_); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage(type_url, {}, "", true); @@ -717,7 +700,7 @@ TEST_F(GrpcMuxImplTestWithMockTimeSystem, TooManyRequestsWithDefaultSettings) { } }; - FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x"}); + auto foo_sub = makeWatch("type_url_foo", {"x"}); expectSendMessage("type_url_foo", {"x"}, "", true); grpc_mux_->start(); @@ -756,7 +739,7 @@ TEST_F(GrpcMuxImplTest, TooManyRequestsWithEmptyRateLimitSettings) { } }; - FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x"}); + auto foo_sub = makeWatch("type_url_foo", {"x"}); expectSendMessage("type_url_foo", {"x"}, "", true); grpc_mux_->start(); @@ -816,7 +799,7 @@ TEST_F(GrpcMuxImplTest, TooManyRequestsWithCustomRateLimitSettings) { } }; - FakeGrpcSubscription foo_sub = makeWatch("type_url_foo", {"x"}); + auto foo_sub = makeWatch("type_url_foo", {"x"}); expectSendMessage("type_url_foo", {"x"}, "", true); grpc_mux_->start(); @@ -851,7 +834,7 @@ TEST_F(GrpcMuxImplTest, UnwatchedTypeAcceptsEmptyResources) { { // subscribe and unsubscribe to simulate a cluster added and removed expectSendMessage(type_url, {"y"}, "", true); - FakeGrpcSubscription temp_sub = makeWatch(type_url, {"y"}); + auto temp_sub = makeWatch(type_url, {"y"}); expectSendMessage(type_url, {}, ""); } @@ -868,7 +851,7 @@ TEST_F(GrpcMuxImplTest, UnwatchedTypeAcceptsEmptyResources) { // When we become interested in "x", we should send a request indicating that interest. expectSendMessage(type_url, {"x"}, "1", false, "bar"); - FakeGrpcSubscription sub = makeWatch(type_url, {"x"}); + auto sub = makeWatch(type_url, {"x"}); // Watch destroyed -> interest gone -> unsubscribe request. expectSendMessage(type_url, {}, "1", false, "bar"); @@ -892,7 +875,7 @@ TEST_F(GrpcMuxImplTest, UnwatchedTypeAcceptsResources) { { expectSendMessage(type_url, {"y"}, "", true); expectSendMessage(type_url, {}, ""); - FakeGrpcSubscription delete_immediately = makeWatch(type_url, {"y"}); + auto delete_immediately = makeWatch(type_url, {"y"}); } auto response = std::make_unique(); response->set_type_url(type_url); @@ -937,8 +920,8 @@ TEST_F(GrpcMuxImplTest, BadLocalInfoEmptyNodeName) { TEST_F(GrpcMuxImplTest, DynamicContextParameters) { setup(); InSequence s; - grpc_mux_->addWatch("foo", {"x", "y"}, callbacks_, resource_decoder_, {}); - grpc_mux_->addWatch("bar", {}, callbacks_, resource_decoder_, {}); + auto foo = grpc_mux_->addWatch("foo", {"x", "y"}, callbacks_, resource_decoder_, {}); + auto bar = grpc_mux_->addWatch("bar", {}, callbacks_, resource_decoder_, {}); EXPECT_CALL(*async_client_, startRaw(_, _, _, _)).WillOnce(Return(&async_stream_)); expectSendMessage("foo", {"x", "y"}, "", true); expectSendMessage("bar", {}, ""); @@ -951,6 +934,9 @@ TEST_F(GrpcMuxImplTest, DynamicContextParameters) { // Update to bar type should resend Node. expectSendMessage("bar", {}, "", true); local_info_.context_provider_.update_cb_handler_.runCallbacks("bar"); + // only destruction of foo watch is going to result in an unsubscribe message. + // bar watch is empty and its destruction doesn't change it resource list. + expectSendMessage("foo", {}, "", false); } } // namespace diff --git a/test/mocks/config/mocks.h b/test/mocks/config/mocks.h index cbf69387a929..ffcfdc796831 100644 --- a/test/mocks/config/mocks.h +++ b/test/mocks/config/mocks.h @@ -123,13 +123,6 @@ class MockGrpcMux : public GrpcMux { (const std::string& type_url, const absl::flat_hash_set& add_these_names)); - MOCK_METHOD(void, updateWatch, - (const std::string& type_url, Watch* watch, - const absl::flat_hash_set& resources, - const SubscriptionOptions& options)); - - MOCK_METHOD(void, removeWatch, (const std::string& type_url, Watch* watch)); - MOCK_METHOD(bool, paused, (const std::string& type_url), (const)); }; From e094de6f7d58ec45909c854290d4e194e2e7a778 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 3 Aug 2021 15:33:33 -0700 Subject: [PATCH 21/46] Removed mutex from XdsMux::AllMuxesState Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 5fd071837ca6..48bfc8169c60 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -20,28 +20,18 @@ namespace XdsMux { namespace { class AllMuxesState { public: - void insert(ShutdownableMux* mux) { - absl::WriterMutexLock locker(&lock_); - muxes_.insert(mux); - } + void insert(ShutdownableMux* mux) { muxes_.insert(mux); } - void erase(ShutdownableMux* mux) { - absl::WriterMutexLock locker(&lock_); - muxes_.erase(mux); - } + void erase(ShutdownableMux* mux) { muxes_.erase(mux); } void shutdownAll() { - absl::WriterMutexLock locker(&lock_); for (auto& mux : muxes_) { mux->shutdown(); } } private: - absl::flat_hash_set muxes_ ABSL_GUARDED_BY(lock_); - - // TODO(ggreenway): can this lock be removed? Is this code only run on the main thread? - absl::Mutex lock_; + absl::flat_hash_set muxes_; }; using AllMuxes = ThreadSafeSingleton; } // namespace From 08e71af6be3940ec84fb63a816733d14560b79d5 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 4 Aug 2021 09:47:23 -0700 Subject: [PATCH 22/46] Trying to fix failing tests Signed-off-by: Dmitri Dolguikh --- test/common/config/BUILD | 1 + 1 file changed, 1 insertion(+) diff --git a/test/common/config/BUILD b/test/common/config/BUILD index 3da081e5b97b..90a9cc768633 100644 --- a/test/common/config/BUILD +++ b/test/common/config/BUILD @@ -210,6 +210,7 @@ envoy_cc_test( "//source/common/config/xds_mux:grpc_mux_lib", "//source/common/protobuf", "//test/common/stats:stat_test_utility_lib", + "//test/config:v2_link_hacks", "//test/mocks:common_lib", "//test/mocks/config:config_mocks", "//test/mocks/event:event_mocks", From c564840bfc3fdf85965e573425af767fa6d2877d Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 4 Aug 2021 13:26:56 -0700 Subject: [PATCH 23/46] Trying to fix test failures Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/BUILD | 3 +-- test/common/config/BUILD | 2 +- test/common/config/xds_grpc_mux_impl_test.cc | 18 +++++++++--------- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/source/common/config/xds_mux/BUILD b/source/common/config/xds_mux/BUILD index dfb9f6786c7b..de337592a86f 100644 --- a/source/common/config/xds_mux/BUILD +++ b/source/common/config/xds_mux/BUILD @@ -61,10 +61,9 @@ envoy_cc_library( ":sotw_subscription_state_lib", "//envoy/event:dispatcher_interface", "//envoy/grpc:async_client_interface", - "//source/common/config:api_version_lib", - "//source/common/config:decoded_resource_lib", "//source/common/config:grpc_stream_lib", "//source/common/config:pausable_ack_queue_lib", + "//source/common/config:version_converter_lib", "//source/common/config:watch_map_lib", "//source/common/config:xds_context_params_lib", "//source/common/config:xds_resource_lib", diff --git a/test/common/config/BUILD b/test/common/config/BUILD index 90a9cc768633..3a4fd29baaf4 100644 --- a/test/common/config/BUILD +++ b/test/common/config/BUILD @@ -204,8 +204,8 @@ envoy_cc_test( name = "xds_grpc_mux_impl_test", srcs = ["xds_grpc_mux_impl_test.cc"], deps = [ - "//source/common/config:api_version_lib", "//source/common/config:protobuf_link_hacks", + "//source/common/config:resource_name_lib", "//source/common/config:version_converter_lib", "//source/common/config/xds_mux:grpc_mux_lib", "//source/common/protobuf", diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc index aab7f47c1439..8585e4972786 100644 --- a/test/common/config/xds_grpc_mux_impl_test.cc +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -3,18 +3,19 @@ #include "envoy/api/v2/discovery.pb.h" #include "envoy/config/endpoint/v3/endpoint.pb.h" #include "envoy/config/endpoint/v3/endpoint.pb.validate.h" +#include "envoy/event/timer.h" #include "envoy/service/discovery/v3/discovery.pb.h" #include "source/common/common/empty_string.h" -#include "source/common/config/api_version.h" #include "source/common/config/protobuf_link_hacks.h" +#include "source/common/config/resource_name.h" #include "source/common/config/utility.h" #include "source/common/config/version_converter.h" #include "source/common/config/xds_mux/grpc_mux_impl.h" #include "source/common/protobuf/protobuf.h" -#include "source/common/stats/isolated_store_impl.h" #include "test/common/stats/stat_test_utility.h" +#include "test/config/v2_link_hacks.h" #include "test/mocks/common.h" #include "test/mocks/config/mocks.h" #include "test/mocks/event/mocks.h" @@ -24,7 +25,6 @@ #include "test/test_common/logging.h" #include "test/test_common/resources.h" #include "test/test_common/simulated_time_system.h" -#include "test/test_common/test_runtime.h" #include "test/test_common/test_time.h" #include "test/test_common/utility.h" @@ -33,12 +33,14 @@ using testing::_; using testing::AtLeast; +using testing::DoAll; using testing::InSequence; using testing::Invoke; using testing::IsSubstring; using testing::NiceMock; using testing::Return; using testing::ReturnRef; +using testing::SaveArg; namespace Envoy { namespace Config { @@ -54,10 +56,8 @@ class GrpcMuxImplTestBase : public testing::Test { control_plane_stats_(Utility::generateControlPlaneStats(stats_)), control_plane_connected_state_( stats_.gauge("control_plane.connected_state", Stats::Gauge::ImportMode::NeverImport)), - control_plane_pending_requests_( - stats_.gauge("control_plane.pending_requests", Stats::Gauge::ImportMode::NeverImport)), - resource_decoder_(TestUtility::TestOpaqueResourceDecoderImpl< - envoy::config::endpoint::v3::ClusterLoadAssignment>("cluster_name")) {} + control_plane_pending_requests_(stats_.gauge("control_plane.pending_requests", + Stats::Gauge::ImportMode::NeverImport)) {} void setup() { grpc_mux_ = std::make_unique( @@ -123,13 +123,13 @@ class GrpcMuxImplTestBase : public testing::Test { NiceMock local_info_; std::unique_ptr grpc_mux_; NiceMock callbacks_; + TestUtility::TestOpaqueResourceDecoderImpl + resource_decoder_{"cluster_name"}; Stats::TestUtil::TestStore stats_; ControlPlaneStats control_plane_stats_; Envoy::Config::RateLimitSettings rate_limit_settings_; Stats::Gauge& control_plane_connected_state_; Stats::Gauge& control_plane_pending_requests_; - TestUtility::TestOpaqueResourceDecoderImpl - resource_decoder_; }; class GrpcMuxImplTest : public GrpcMuxImplTestBase { From b3d0f1ccdf801b0efa378b80c063af079e47894f Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 4 Aug 2021 15:50:44 -0700 Subject: [PATCH 24/46] Fixing tests Signed-off-by: Dmitri Dolguikh --- test/common/config/xds_grpc_mux_impl_test.cc | 2 -- 1 file changed, 2 deletions(-) diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc index 8585e4972786..0e2f40a2256f 100644 --- a/test/common/config/xds_grpc_mux_impl_test.cc +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -33,14 +33,12 @@ using testing::_; using testing::AtLeast; -using testing::DoAll; using testing::InSequence; using testing::Invoke; using testing::IsSubstring; using testing::NiceMock; using testing::Return; using testing::ReturnRef; -using testing::SaveArg; namespace Envoy { namespace Config { From a317d8414c4dd573da33fcb17177131f059d0593 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Mon, 16 Aug 2021 13:28:11 -0700 Subject: [PATCH 25/46] Fixing build failure: removed v2_link_hacks Signed-off-by: Dmitri Dolguikh --- test/common/config/xds_grpc_mux_impl_test.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc index 0e2f40a2256f..bfac63a6ca09 100644 --- a/test/common/config/xds_grpc_mux_impl_test.cc +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -15,7 +15,6 @@ #include "source/common/protobuf/protobuf.h" #include "test/common/stats/stat_test_utility.h" -#include "test/config/v2_link_hacks.h" #include "test/mocks/common.h" #include "test/mocks/config/mocks.h" #include "test/mocks/event/mocks.h" From 5dd416277885c6cb788a66bb55df30f4da6a326a Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Mon, 16 Aug 2021 15:25:53 -0700 Subject: [PATCH 26/46] Fixed formatting Signed-off-by: Dmitri Dolguikh --- test/common/config/BUILD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/common/config/BUILD b/test/common/config/BUILD index 1357df3a007a..481977fade4a 100644 --- a/test/common/config/BUILD +++ b/test/common/config/BUILD @@ -191,7 +191,6 @@ envoy_cc_test( "//test/test_common:simulated_time_system_lib", "//test/test_common:test_runtime_lib", "//test/test_common:utility_lib", - "@envoy_api//envoy/api/v2:pkg_cc_proto", "@envoy_api//envoy/config/endpoint/v3:pkg_cc_proto", "@envoy_api//envoy/service/discovery/v3:pkg_cc_proto", ], @@ -218,6 +217,7 @@ envoy_cc_test( "//test/test_common:simulated_time_system_lib", "//test/test_common:test_runtime_lib", "//test/test_common:utility_lib", + "@envoy_api//envoy/api/v2:pkg_cc_proto", "@envoy_api//envoy/config/endpoint/v3:pkg_cc_proto", "@envoy_api//envoy/service/discovery/v3:pkg_cc_proto", ], From 3568c2041450e7991a3c2615671569281b162a46 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Mon, 16 Aug 2021 18:03:11 -0700 Subject: [PATCH 27/46] Fixing build Signed-off-by: Dmitri Dolguikh --- test/common/config/BUILD | 1 - 1 file changed, 1 deletion(-) diff --git a/test/common/config/BUILD b/test/common/config/BUILD index 481977fade4a..90b0eed82a6e 100644 --- a/test/common/config/BUILD +++ b/test/common/config/BUILD @@ -179,7 +179,6 @@ envoy_cc_test( "//source/common/config/xds_mux:grpc_mux_lib", "//source/common/protobuf", "//test/common/stats:stat_test_utility_lib", - "//test/config:v2_link_hacks", "//test/mocks:common_lib", "//test/mocks/config:config_mocks", "//test/mocks/event:event_mocks", From 533bb6f3ae670aa66b23e7388040e4514ce500f2 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 09:09:01 -0700 Subject: [PATCH 28/46] trying to fix windows build Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 2 +- source/common/config/xds_mux/grpc_mux_impl.h | 13 ++++++------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 48bfc8169c60..0e62f1022d94 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -213,7 +213,7 @@ void GrpcMuxImpl::genericHandleResponse(const std::string& type_ur template void GrpcMuxImpl::start() { ENVOY_LOG(debug, "GrpcMuxImpl now trying to establish a stream"); - establishGrpcStream(); + grpc_stream_.establishNewStream(); } template diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index d439caf698c9..f43c5feca76d 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -132,7 +132,6 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, const SubscriptionOptions options_; }; - void establishGrpcStream() { grpc_stream_.establishNewStream(); } void sendGrpcMessage(RQ& msg_proto, S& sub_state); void maybeUpdateQueueSizeStat(uint64_t size) { grpc_stream_.maybeUpdateQueueSizeStat(size); } bool grpcStreamAvailable() { return grpc_stream_.grpcStreamAvailable(); } @@ -163,12 +162,12 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, // whether we *want* to send a (Delta)DiscoveryRequest). bool canSendDiscoveryRequest(const std::string& type_url); - // Checks whether we have something to say in a (Delta)DiscoveryRequest, which can be an ACK - // and/or a subscription update. (Does not check whether we *can* send that - // (Delta)DiscoveryRequest). Returns the type_url we should send the DeltaDiscoveryRequest for (if - // any). First, prioritizes ACKs over non-ACK subscription interest updates. Then, prioritizes - // non-ACK updates in the order the various types of subscriptions were activated (as tracked by - // subscription_ordering_). + // Checks whether we have something to say in a (Delta)DiscoveryRequest, which can be an ACK and/or + // a subscription update. (Does not check whether we *can* send that (Delta)DiscoveryRequest). + // Returns the type_url we should send the DeltaDiscoveryRequest for (if any). + // First, prioritizes ACKs over non-ACK subscription interest updates. + // Then, prioritizes non-ACK updates in the order the various types + // of subscriptions were activated (as tracked by subscription_ordering_). absl::optional whoWantsToSendDiscoveryRequest(); // Invoked when dynamic context parameters change for a resource type. From f8178dc8419a111d9a2fb21399c597e73055cd8f Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 10:21:31 -0700 Subject: [PATCH 29/46] Fixed formatting Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.h | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index f43c5feca76d..8659bc204e54 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -162,12 +162,12 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, // whether we *want* to send a (Delta)DiscoveryRequest). bool canSendDiscoveryRequest(const std::string& type_url); - // Checks whether we have something to say in a (Delta)DiscoveryRequest, which can be an ACK and/or - // a subscription update. (Does not check whether we *can* send that (Delta)DiscoveryRequest). - // Returns the type_url we should send the DeltaDiscoveryRequest for (if any). - // First, prioritizes ACKs over non-ACK subscription interest updates. - // Then, prioritizes non-ACK updates in the order the various types - // of subscriptions were activated (as tracked by subscription_ordering_). + // Checks whether we have something to say in a (Delta)DiscoveryRequest, which can be an ACK + // and/or a subscription update. (Does not check whether we *can* send that + // (Delta)DiscoveryRequest). Returns the type_url we should send the DeltaDiscoveryRequest for (if + // any). First, prioritizes ACKs over non-ACK subscription interest updates. Then, prioritizes + // non-ACK updates in the order the various types of subscriptions were activated (as tracked by + // subscription_ordering_). absl::optional whoWantsToSendDiscoveryRequest(); // Invoked when dynamic context parameters change for a resource type. From 5ac05535e86996ef5155699356f9470dbc951ac3 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 12:31:28 -0700 Subject: [PATCH 30/46] Trying to fix tests under windows Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 36 +------------------- 1 file changed, 1 insertion(+), 35 deletions(-) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index e29d34c89a11..ffddd967885a 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -56,15 +56,6 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { should_use_unified_(legacy_or_unified == LegacyOrUnified::Unified) {} void setup() { - if (isUnifiedMuxTest()) { - grpc_mux_ = std::make_unique( - std::unique_ptr(async_client_), dispatcher_, - *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( - "envoy.service.discovery.v2.AggregatedDiscoveryService.StreamAggregatedResources"), - envoy::config::core::v3::ApiVersion::AUTO, random_, stats_, rate_limit_settings_, - local_info_, false); - return; - } grpc_mux_ = std::make_unique( std::unique_ptr(async_client_), dispatcher_, *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( @@ -102,12 +93,6 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { } void remoteClose() { - if (isUnifiedMuxTest()) { - dynamic_cast(grpc_mux_.get()) - ->grpcStreamForTest() - .onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); - return; - } dynamic_cast(grpc_mux_.get()) ->grpcStreamForTest() .onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); @@ -115,40 +100,21 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { void onDiscoveryResponse( std::unique_ptr&& response) { - if (isUnifiedMuxTest()) { - dynamic_cast(grpc_mux_.get()) - ->onDiscoveryResponse(std::move(response), control_plane_stats_); - return; - } dynamic_cast(grpc_mux_.get()) ->onDiscoveryResponse(std::move(response), control_plane_stats_); } - void shutdownMux() { - if (isUnifiedMuxTest()) { - dynamic_cast(grpc_mux_.get())->shutdown(); - return; - } - dynamic_cast(grpc_mux_.get())->shutdown(); - } + void shutdownMux() { dynamic_cast(grpc_mux_.get())->shutdown(); } // the code is duplicated here, but all calls other than the check in return statement, return // different types. bool subscriptionExists(const std::string& type_url) const { - if (isUnifiedMuxTest()) { - auto* mux = dynamic_cast(grpc_mux_.get()); - auto& subscriptions = mux->subscriptions(); - auto sub = subscriptions.find(type_url); - return sub != subscriptions.end(); - } auto* mux = dynamic_cast(grpc_mux_.get()); auto& subscriptions = mux->subscriptions(); auto sub = subscriptions.find(type_url); return sub != subscriptions.end(); } - bool isUnifiedMuxTest() const { return should_use_unified_; } - NiceMock dispatcher_; NiceMock random_; Grpc::MockAsyncClient* async_client_; From 03f7a7c4d5dc2466e66ed183180e9b89b3c724f5 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 13:52:02 -0700 Subject: [PATCH 31/46] Revert "Trying to fix tests under windows" This reverts commit 5ac05535e86996ef5155699356f9470dbc951ac3. Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 36 +++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index ffddd967885a..e29d34c89a11 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -56,6 +56,15 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { should_use_unified_(legacy_or_unified == LegacyOrUnified::Unified) {} void setup() { + if (isUnifiedMuxTest()) { + grpc_mux_ = std::make_unique( + std::unique_ptr(async_client_), dispatcher_, + *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( + "envoy.service.discovery.v2.AggregatedDiscoveryService.StreamAggregatedResources"), + envoy::config::core::v3::ApiVersion::AUTO, random_, stats_, rate_limit_settings_, + local_info_, false); + return; + } grpc_mux_ = std::make_unique( std::unique_ptr(async_client_), dispatcher_, *Protobuf::DescriptorPool::generated_pool()->FindMethodByName( @@ -93,6 +102,12 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { } void remoteClose() { + if (isUnifiedMuxTest()) { + dynamic_cast(grpc_mux_.get()) + ->grpcStreamForTest() + .onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); + return; + } dynamic_cast(grpc_mux_.get()) ->grpcStreamForTest() .onRemoteClose(Grpc::Status::WellKnownGrpcStatus::Canceled, ""); @@ -100,21 +115,40 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { void onDiscoveryResponse( std::unique_ptr&& response) { + if (isUnifiedMuxTest()) { + dynamic_cast(grpc_mux_.get()) + ->onDiscoveryResponse(std::move(response), control_plane_stats_); + return; + } dynamic_cast(grpc_mux_.get()) ->onDiscoveryResponse(std::move(response), control_plane_stats_); } - void shutdownMux() { dynamic_cast(grpc_mux_.get())->shutdown(); } + void shutdownMux() { + if (isUnifiedMuxTest()) { + dynamic_cast(grpc_mux_.get())->shutdown(); + return; + } + dynamic_cast(grpc_mux_.get())->shutdown(); + } // the code is duplicated here, but all calls other than the check in return statement, return // different types. bool subscriptionExists(const std::string& type_url) const { + if (isUnifiedMuxTest()) { + auto* mux = dynamic_cast(grpc_mux_.get()); + auto& subscriptions = mux->subscriptions(); + auto sub = subscriptions.find(type_url); + return sub != subscriptions.end(); + } auto* mux = dynamic_cast(grpc_mux_.get()); auto& subscriptions = mux->subscriptions(); auto sub = subscriptions.find(type_url); return sub != subscriptions.end(); } + bool isUnifiedMuxTest() const { return should_use_unified_; } + NiceMock dispatcher_; NiceMock random_; Grpc::MockAsyncClient* async_client_; From de40b517f1d61d2d006e73171d357ad6d0087dc3 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 14:12:44 -0700 Subject: [PATCH 32/46] Trying to fix test failures on Windows Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 1 + source/common/config/xds_mux/grpc_mux_impl.h | 2 ++ 2 files changed, 3 insertions(+) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 0e62f1022d94..f84ae1f3671a 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -60,6 +60,7 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac } template GrpcMuxImpl::~GrpcMuxImpl() { + ENVOY_LOG(info, "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); AllMuxes::get().erase(this); } diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 8659bc204e54..c08ca26d73c6 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -221,6 +221,7 @@ class GrpcMuxDelta : public GrpcMuxImpl&) override { From 58837b75db4414e83b3b1fae7b4b47f619e5d43e Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 15:26:09 -0700 Subject: [PATCH 33/46] Trying to fix test failures on Windows Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 3 ++- source/common/config/xds_mux/grpc_mux_impl.h | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index f84ae1f3671a..7bfe27c078a7 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -60,7 +60,8 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac } template GrpcMuxImpl::~GrpcMuxImpl() { - ENVOY_LOG(info, "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); + std::cout << "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!" + << "\n"; AllMuxes::get().erase(this); } diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index c08ca26d73c6..3bbd37a59678 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -63,7 +63,7 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, Stats::Scope& scope, const RateLimitSettings& rate_limit_settings); - ~GrpcMuxImpl() override; + virtual ~GrpcMuxImpl(); // Causes all GrpcMuxImpl objects to stop sending any messages on `grpc_stream_` to fix a crash // on Envoy shutdown due to dangling pointers. This may not be the ideal fix; it is probably From 2a7a5affd15a386afbc114cf4a53157e68231064 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 16:30:52 -0700 Subject: [PATCH 34/46] Trying to fix test failures on Windows Signed-off-by: Dmitri Dolguikh --- source/common/config/grpc_stream.h | 6 ++++++ source/common/config/xds_mux/grpc_mux_impl.cc | 3 +-- test/common/config/new_grpc_mux_impl_test.cc | 1 + 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/source/common/config/grpc_stream.h b/source/common/config/grpc_stream.h index 23ee9a0d287b..b920e96797d9 100644 --- a/source/common/config/grpc_stream.h +++ b/source/common/config/grpc_stream.h @@ -57,12 +57,18 @@ class GrpcStream : public Grpc::AsyncStreamCallbacks, RetryInitialDelayMs, RetryMaxDelayMs, random_); } + ~GrpcStream() override { + std::cout << "in ~GrpcStream() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n"; + } + void establishNewStream() { ENVOY_LOG(debug, "Establishing new gRPC bidi stream for {}", service_method_.DebugString()); if (stream_ != nullptr) { ENVOY_LOG(warn, "gRPC bidi stream for {} already exists!", service_method_.DebugString()); return; } + std::cout << "GrpcStream::establishNewStream() " << &async_client_ + << " !!!!!!!!!!!!!!!!!!!!!!!!!\n"; stream_ = async_client_->start(service_method_, *this, Http::AsyncClient::StreamOptions()); if (stream_ == nullptr) { ENVOY_LOG(debug, "Unable to establish new stream to configuration server"); diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 7bfe27c078a7..769e6b3a1b78 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -60,8 +60,6 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac } template GrpcMuxImpl::~GrpcMuxImpl() { - std::cout << "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!" - << "\n"; AllMuxes::get().erase(this); } @@ -215,6 +213,7 @@ void GrpcMuxImpl::genericHandleResponse(const std::string& type_ur template void GrpcMuxImpl::start() { ENVOY_LOG(debug, "GrpcMuxImpl now trying to establish a stream"); + std::cout << "in GrpcMuxImpl::start() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n"; grpc_stream_.establishNewStream(); } diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index e29d34c89a11..f61a99715267 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -56,6 +56,7 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { should_use_unified_(legacy_or_unified == LegacyOrUnified::Unified) {} void setup() { + std::cout << "NewGrpcMuxImplTestBase " << async_client_ << " !!!!!!!!!!!!!!!\n"; if (isUnifiedMuxTest()) { grpc_mux_ = std::make_unique( std::unique_ptr(async_client_), dispatcher_, From dfe351583aced1208096188e5a51ce2259e61270 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 17:42:36 -0700 Subject: [PATCH 35/46] Revert "Trying to fix test failures on Windows" This reverts commit 2a7a5affd15a386afbc114cf4a53157e68231064. Signed-off-by: Dmitri Dolguikh --- source/common/config/grpc_stream.h | 6 ------ source/common/config/xds_mux/grpc_mux_impl.cc | 3 ++- test/common/config/new_grpc_mux_impl_test.cc | 1 - 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/source/common/config/grpc_stream.h b/source/common/config/grpc_stream.h index b920e96797d9..23ee9a0d287b 100644 --- a/source/common/config/grpc_stream.h +++ b/source/common/config/grpc_stream.h @@ -57,18 +57,12 @@ class GrpcStream : public Grpc::AsyncStreamCallbacks, RetryInitialDelayMs, RetryMaxDelayMs, random_); } - ~GrpcStream() override { - std::cout << "in ~GrpcStream() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n"; - } - void establishNewStream() { ENVOY_LOG(debug, "Establishing new gRPC bidi stream for {}", service_method_.DebugString()); if (stream_ != nullptr) { ENVOY_LOG(warn, "gRPC bidi stream for {} already exists!", service_method_.DebugString()); return; } - std::cout << "GrpcStream::establishNewStream() " << &async_client_ - << " !!!!!!!!!!!!!!!!!!!!!!!!!\n"; stream_ = async_client_->start(service_method_, *this, Http::AsyncClient::StreamOptions()); if (stream_ == nullptr) { ENVOY_LOG(debug, "Unable to establish new stream to configuration server"); diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 769e6b3a1b78..7bfe27c078a7 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -60,6 +60,8 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac } template GrpcMuxImpl::~GrpcMuxImpl() { + std::cout << "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!" + << "\n"; AllMuxes::get().erase(this); } @@ -213,7 +215,6 @@ void GrpcMuxImpl::genericHandleResponse(const std::string& type_ur template void GrpcMuxImpl::start() { ENVOY_LOG(debug, "GrpcMuxImpl now trying to establish a stream"); - std::cout << "in GrpcMuxImpl::start() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n"; grpc_stream_.establishNewStream(); } diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index f61a99715267..e29d34c89a11 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -56,7 +56,6 @@ class NewGrpcMuxImplTestBase : public testing::TestWithParam { should_use_unified_(legacy_or_unified == LegacyOrUnified::Unified) {} void setup() { - std::cout << "NewGrpcMuxImplTestBase " << async_client_ << " !!!!!!!!!!!!!!!\n"; if (isUnifiedMuxTest()) { grpc_mux_ = std::make_unique( std::unique_ptr(async_client_), dispatcher_, From ab0cab53054b6af53a8af76ab2158481594e0e1f Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 17:43:29 -0700 Subject: [PATCH 36/46] Revert "Trying to fix test failures on Windows" This reverts commit 58837b75db4414e83b3b1fae7b4b47f619e5d43e. Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 3 +-- source/common/config/xds_mux/grpc_mux_impl.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 7bfe27c078a7..f84ae1f3671a 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -60,8 +60,7 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac } template GrpcMuxImpl::~GrpcMuxImpl() { - std::cout << "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!" - << "\n"; + ENVOY_LOG(info, "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); AllMuxes::get().erase(this); } diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 3bbd37a59678..c08ca26d73c6 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -63,7 +63,7 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, Stats::Scope& scope, const RateLimitSettings& rate_limit_settings); - virtual ~GrpcMuxImpl(); + ~GrpcMuxImpl() override; // Causes all GrpcMuxImpl objects to stop sending any messages on `grpc_stream_` to fix a crash // on Envoy shutdown due to dangling pointers. This may not be the ideal fix; it is probably From 349de2102553a2fd400b8789ef235de380cec753 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 17:43:59 -0700 Subject: [PATCH 37/46] Revert "Trying to fix test failures on Windows" This reverts commit de40b517f1d61d2d006e73171d357ad6d0087dc3. Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 1 - source/common/config/xds_mux/grpc_mux_impl.h | 2 -- 2 files changed, 3 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index f84ae1f3671a..0e62f1022d94 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -60,7 +60,6 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac } template GrpcMuxImpl::~GrpcMuxImpl() { - ENVOY_LOG(info, "in ~GrpcMuxImpl() !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); AllMuxes::get().erase(this); } diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index c08ca26d73c6..8659bc204e54 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -221,7 +221,6 @@ class GrpcMuxDelta : public GrpcMuxImpl&) override { From cda496eb3c35f288cb7b5ea1fc5b238f2b99e544 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 17:46:10 -0700 Subject: [PATCH 38/46] Trying to fix test failures under Windows Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 2 +- source/common/config/xds_mux/grpc_mux_impl.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 0e62f1022d94..78beb5b60385 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -41,7 +41,7 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, envoy::config::core::v3::ApiVersion transport_api_version, - Grpc::RawAsyncClientPtr&& async_client, + Grpc::RawAsyncClientPtr async_client, Event::Dispatcher& dispatcher, const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, Stats::Scope& scope, diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 8659bc204e54..8495ea11070b 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -59,7 +59,7 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, envoy::config::core::v3::ApiVersion transport_api_version, - Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, + Grpc::RawAsyncClientPtr async_client, Event::Dispatcher& dispatcher, const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, Stats::Scope& scope, const RateLimitSettings& rate_limit_settings); From 3777c61d88da776b80b752742a1e247eb2dd3865 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 18:34:42 -0700 Subject: [PATCH 39/46] Revert "Trying to fix test failures under Windows" This reverts commit cda496eb3c35f288cb7b5ea1fc5b238f2b99e544. Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.cc | 2 +- source/common/config/xds_mux/grpc_mux_impl.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.cc b/source/common/config/xds_mux/grpc_mux_impl.cc index 78beb5b60385..0e62f1022d94 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.cc +++ b/source/common/config/xds_mux/grpc_mux_impl.cc @@ -41,7 +41,7 @@ GrpcMuxImpl::GrpcMuxImpl(std::unique_ptr subscription_state_fac bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, envoy::config::core::v3::ApiVersion transport_api_version, - Grpc::RawAsyncClientPtr async_client, + Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, Stats::Scope& scope, diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 8495ea11070b..8659bc204e54 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -59,7 +59,7 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, GrpcMuxImpl(std::unique_ptr subscription_state_factory, bool skip_subsequent_node, const LocalInfo::LocalInfo& local_info, envoy::config::core::v3::ApiVersion transport_api_version, - Grpc::RawAsyncClientPtr async_client, Event::Dispatcher& dispatcher, + Grpc::RawAsyncClientPtr&& async_client, Event::Dispatcher& dispatcher, const Protobuf::MethodDescriptor& service_method, Random::RandomGenerator& random, Stats::Scope& scope, const RateLimitSettings& rate_limit_settings); From 1b5564c0835359bb963240c6476f6bbe4beee74f Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 18:59:12 -0700 Subject: [PATCH 40/46] Trying to fix test failures under Windows Signed-off-by: Dmitri Dolguikh --- source/common/config/xds_mux/grpc_mux_impl.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/config/xds_mux/grpc_mux_impl.h b/source/common/config/xds_mux/grpc_mux_impl.h index 8659bc204e54..a1ec7f2332dd 100644 --- a/source/common/config/xds_mux/grpc_mux_impl.h +++ b/source/common/config/xds_mux/grpc_mux_impl.h @@ -155,8 +155,6 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, return transport_api_version_; } - GrpcStream grpc_stream_; - private: // Checks whether external conditions allow sending a DeltaDiscoveryRequest. (Does not check // whether we *want* to send a (Delta)DiscoveryRequest). @@ -173,6 +171,8 @@ class GrpcMuxImpl : public GrpcStreamCallbacks, // Invoked when dynamic context parameters change for a resource type. void onDynamicContextUpdate(absl::string_view resource_type_url); + GrpcStream grpc_stream_; + // Resource (N)ACKs we're waiting to send, stored in the order that they should be sent in. All // of our different resource types' ACKs are mixed together in this queue. See class for // description of how it interacts with pause() and resume(). From 95075f9ac8e79edc76a6424093b2dcfd9357a34e Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 21:26:48 -0700 Subject: [PATCH 41/46] Trying to fix test failures under Windows Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index e29d34c89a11..4860910cb8eb 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -194,6 +194,9 @@ TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { local_info_.context_provider_.update_cb_handler_.runCallbacks("bar"); expectSendMessage("foo", {}, {"x", "y"}); + + testing::Mock::VerifyAndClearExpectations(*async_client_); + testing::Mock::VerifyAndClearExpectations(async_stream_); } // Validate cached nonces are cleared on reconnection. From 1e58a3dfa27803a933692ed5f3f79e6cdbbc07e3 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Tue, 17 Aug 2021 21:37:38 -0700 Subject: [PATCH 42/46] Trying to fix test failures under Windows Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index 4860910cb8eb..9b0b3a4d76c4 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -195,8 +195,8 @@ TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { expectSendMessage("foo", {}, {"x", "y"}); - testing::Mock::VerifyAndClearExpectations(*async_client_); - testing::Mock::VerifyAndClearExpectations(async_stream_); + testing::Mock::VerifyAndClearExpectations(async_client_); + testing::Mock::VerifyAndClearExpectations(&async_stream_); } // Validate cached nonces are cleared on reconnection. From ba094fb848aacc5b2ef92937c2d4df27c67797ef Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 18 Aug 2021 07:37:10 -0700 Subject: [PATCH 43/46] Revert "Trying to fix test failures under Windows" This reverts commit 1e58a3dfa27803a933692ed5f3f79e6cdbbc07e3. Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index 9b0b3a4d76c4..4860910cb8eb 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -195,8 +195,8 @@ TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { expectSendMessage("foo", {}, {"x", "y"}); - testing::Mock::VerifyAndClearExpectations(async_client_); - testing::Mock::VerifyAndClearExpectations(&async_stream_); + testing::Mock::VerifyAndClearExpectations(*async_client_); + testing::Mock::VerifyAndClearExpectations(async_stream_); } // Validate cached nonces are cleared on reconnection. From e169379973f3ee8e8cd518ca9564f816375d5ff2 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 18 Aug 2021 08:21:59 -0700 Subject: [PATCH 44/46] Trying to fix test failures under Windows Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index 4860910cb8eb..603195428061 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -195,8 +195,7 @@ TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { expectSendMessage("foo", {}, {"x", "y"}); - testing::Mock::VerifyAndClearExpectations(*async_client_); - testing::Mock::VerifyAndClearExpectations(async_stream_); + testing::Mock::VerifyAndClearExpectations(async_client_); } // Validate cached nonces are cleared on reconnection. From 893ce726b6a6069512dfb56dbcd3c93dc05b8c06 Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 18 Aug 2021 11:04:55 -0700 Subject: [PATCH 45/46] Trying to fix test failures under Windows Signed-off-by: Dmitri Dolguikh --- test/common/config/new_grpc_mux_impl_test.cc | 2 -- 1 file changed, 2 deletions(-) diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index 603195428061..e29d34c89a11 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -194,8 +194,6 @@ TEST_P(NewGrpcMuxImplTest, DynamicContextParameters) { local_info_.context_provider_.update_cb_handler_.runCallbacks("bar"); expectSendMessage("foo", {}, {"x", "y"}); - - testing::Mock::VerifyAndClearExpectations(async_client_); } // Validate cached nonces are cleared on reconnection. From 1ab61ad655ef854c8f1b09d7c373b99c0b88e47e Mon Sep 17 00:00:00 2001 From: Dmitri Dolguikh Date: Wed, 18 Aug 2021 15:35:08 -0700 Subject: [PATCH 46/46] Trying to fix test failures under Windows Signed-off-by: Dmitri Dolguikh --- test/common/config/BUILD | 2 ++ test/common/config/new_grpc_mux_impl_test.cc | 1 + test/common/config/xds_grpc_mux_impl_test.cc | 1 + 3 files changed, 4 insertions(+) diff --git a/test/common/config/BUILD b/test/common/config/BUILD index 90b0eed82a6e..f662aaa300cc 100644 --- a/test/common/config/BUILD +++ b/test/common/config/BUILD @@ -179,6 +179,7 @@ envoy_cc_test( "//source/common/config/xds_mux:grpc_mux_lib", "//source/common/protobuf", "//test/common/stats:stat_test_utility_lib", + "//test/config:v2_link_hacks", "//test/mocks:common_lib", "//test/mocks/config:config_mocks", "//test/mocks/event:event_mocks", @@ -205,6 +206,7 @@ envoy_cc_test( "//source/common/config/xds_mux:grpc_mux_lib", "//source/common/protobuf", "//test/common/stats:stat_test_utility_lib", + "//test/config:v2_link_hacks", "//test/mocks:common_lib", "//test/mocks/config:config_mocks", "//test/mocks/event:event_mocks", diff --git a/test/common/config/new_grpc_mux_impl_test.cc b/test/common/config/new_grpc_mux_impl_test.cc index e29d34c89a11..b417e99377ef 100644 --- a/test/common/config/new_grpc_mux_impl_test.cc +++ b/test/common/config/new_grpc_mux_impl_test.cc @@ -14,6 +14,7 @@ #include "source/common/protobuf/protobuf.h" #include "test/common/stats/stat_test_utility.h" +#include "test/config/v2_link_hacks.h" #include "test/mocks/common.h" #include "test/mocks/config/mocks.h" #include "test/mocks/event/mocks.h" diff --git a/test/common/config/xds_grpc_mux_impl_test.cc b/test/common/config/xds_grpc_mux_impl_test.cc index bfac63a6ca09..0e2f40a2256f 100644 --- a/test/common/config/xds_grpc_mux_impl_test.cc +++ b/test/common/config/xds_grpc_mux_impl_test.cc @@ -15,6 +15,7 @@ #include "source/common/protobuf/protobuf.h" #include "test/common/stats/stat_test_utility.h" +#include "test/config/v2_link_hacks.h" #include "test/mocks/common.h" #include "test/mocks/config/mocks.h" #include "test/mocks/event/mocks.h"