implement new approach with UnstartedCallHandler

pull/35791/head
Mark D. Roth 1 year ago
parent c83f1bb586
commit 214126ae16
  1. 1
      BUILD
  2. 1
      Package.swift
  3. 2
      build_autogenerated.yaml
  4. 2
      gRPC-C++.podspec
  5. 2
      gRPC-Core.podspec
  6. 1
      grpc.gemspec
  7. 1
      package.xml
  8. 13
      src/core/BUILD
  9. 1286
      src/core/client_channel/client_channel.cc
  10. 12
      src/core/client_channel/client_channel.h
  11. 116
      src/core/client_channel/subchannel.cc
  12. 35
      src/core/client_channel/subchannel.h
  13. 5
      src/core/lib/surface/channel.h
  14. 13
      src/core/lib/surface/legacy_channel.cc
  15. 35
      src/core/lib/transport/call_destination.h
  16. 20
      src/core/lib/transport/call_spine.cc
  17. 109
      src/core/lib/transport/call_spine.h
  18. 21
      src/core/lib/transport/metadata_batch.h
  19. 1
      src/core/lib/transport/transport.h
  20. 1
      tools/doxygen/Doxyfile.c++.internal
  21. 1
      tools/doxygen/Doxyfile.core.internal

@ -3649,7 +3649,6 @@ grpc_cc_library(
"//src/core:arena_promise",
"//src/core:backend_metric_parser",
"//src/core:call_filters",
"//src/core:call_destination",
"//src/core:call_spine",
"//src/core:cancel_callback",
"//src/core:channel_args",

1
Package.swift generated

@ -1805,7 +1805,6 @@ let package = Package(
"src/core/lib/transport/batch_builder.h",
"src/core/lib/transport/bdp_estimator.cc",
"src/core/lib/transport/bdp_estimator.h",
"src/core/lib/transport/call_destination.h",
"src/core/lib/transport/call_filters.cc",
"src/core/lib/transport/call_filters.h",
"src/core/lib/transport/call_final_info.cc",

@ -1133,7 +1133,6 @@ libs:
- src/core/lib/surface/wait_for_cq_end_op.h
- src/core/lib/transport/batch_builder.h
- src/core/lib/transport/bdp_estimator.h
- src/core/lib/transport/call_destination.h
- src/core/lib/transport/call_filters.h
- src/core/lib/transport/call_final_info.h
- src/core/lib/transport/call_spine.h
@ -2599,7 +2598,6 @@ libs:
- src/core/lib/surface/wait_for_cq_end_op.h
- src/core/lib/transport/batch_builder.h
- src/core/lib/transport/bdp_estimator.h
- src/core/lib/transport/call_destination.h
- src/core/lib/transport/call_filters.h
- src/core/lib/transport/call_final_info.h
- src/core/lib/transport/call_spine.h

2
gRPC-C++.podspec generated

@ -1238,7 +1238,6 @@ Pod::Spec.new do |s|
'src/core/lib/surface/wait_for_cq_end_op.h',
'src/core/lib/transport/batch_builder.h',
'src/core/lib/transport/bdp_estimator.h',
'src/core/lib/transport/call_destination.h',
'src/core/lib/transport/call_filters.h',
'src/core/lib/transport/call_final_info.h',
'src/core/lib/transport/call_spine.h',
@ -2500,7 +2499,6 @@ Pod::Spec.new do |s|
'src/core/lib/surface/wait_for_cq_end_op.h',
'src/core/lib/transport/batch_builder.h',
'src/core/lib/transport/bdp_estimator.h',
'src/core/lib/transport/call_destination.h',
'src/core/lib/transport/call_filters.h',
'src/core/lib/transport/call_final_info.h',
'src/core/lib/transport/call_spine.h',

2
gRPC-Core.podspec generated

@ -1914,7 +1914,6 @@ Pod::Spec.new do |s|
'src/core/lib/transport/batch_builder.h',
'src/core/lib/transport/bdp_estimator.cc',
'src/core/lib/transport/bdp_estimator.h',
'src/core/lib/transport/call_destination.h',
'src/core/lib/transport/call_filters.cc',
'src/core/lib/transport/call_filters.h',
'src/core/lib/transport/call_final_info.cc',
@ -3279,7 +3278,6 @@ Pod::Spec.new do |s|
'src/core/lib/surface/wait_for_cq_end_op.h',
'src/core/lib/transport/batch_builder.h',
'src/core/lib/transport/bdp_estimator.h',
'src/core/lib/transport/call_destination.h',
'src/core/lib/transport/call_filters.h',
'src/core/lib/transport/call_final_info.h',
'src/core/lib/transport/call_spine.h',

1
grpc.gemspec generated

@ -1807,7 +1807,6 @@ Gem::Specification.new do |s|
s.files += %w( src/core/lib/transport/batch_builder.h )
s.files += %w( src/core/lib/transport/bdp_estimator.cc )
s.files += %w( src/core/lib/transport/bdp_estimator.h )
s.files += %w( src/core/lib/transport/call_destination.h )
s.files += %w( src/core/lib/transport/call_filters.cc )
s.files += %w( src/core/lib/transport/call_filters.h )
s.files += %w( src/core/lib/transport/call_final_info.cc )

1
package.xml generated

@ -1789,7 +1789,6 @@
<file baseinstalldir="/" name="src/core/lib/transport/batch_builder.h" role="src" />
<file baseinstalldir="/" name="src/core/lib/transport/bdp_estimator.cc" role="src" />
<file baseinstalldir="/" name="src/core/lib/transport/bdp_estimator.h" role="src" />
<file baseinstalldir="/" name="src/core/lib/transport/call_destination.h" role="src" />
<file baseinstalldir="/" name="src/core/lib/transport/call_filters.cc" role="src" />
<file baseinstalldir="/" name="src/core/lib/transport/call_filters.h" role="src" />
<file baseinstalldir="/" name="src/core/lib/transport/call_final_info.cc" role="src" />

@ -7214,18 +7214,6 @@ grpc_cc_library(
],
)
grpc_cc_library(
name = "call_destination",
hdrs = [
"lib/transport/call_destination.h",
],
deps = [
"call_spine",
"dual_ref_counted",
"//:gpr_platform",
],
)
grpc_cc_library(
name = "parsed_metadata",
srcs = [
@ -7292,6 +7280,7 @@ grpc_cc_library(
"1999",
"call_filters",
"call_final_info",
"dual_ref_counted",
"for_each",
"if",
"latch",

File diff suppressed because it is too large Load Diff

@ -96,11 +96,6 @@ class ClientChannel : public Channel {
void Ping(grpc_completion_queue* cq, void* tag) override;
// Creates a load balanced call on the channel.
CallInitiator CreateLoadBalancedCall(
ClientMetadataHandle client_initial_metadata,
absl::AnyInvocable<void()> on_commit, bool is_transparent_retry);
// Flag that this object gets stored in channel args as a raw pointer.
struct RawPointerChannelArgTag {};
static absl::string_view ChannelArgName() {
@ -111,6 +106,7 @@ class ClientChannel : public Channel {
class ResolverResultHandler;
class ClientChannelControlHelper;
class SubchannelWrapper;
class LoadBalancedCallDestination;
void CreateResolverLocked() ABSL_EXCLUSIVE_LOCKS_REQUIRED(*work_serializer_);
void DestroyResolverAndLbPolicyLocked()
@ -168,8 +164,7 @@ class ClientChannel : public Channel {
// context.
LoopCtl<absl::StatusOr<RefCountedPtr<ConnectedSubchannel>>> PickSubchannel(
LoadBalancingPolicy::SubchannelPicker& picker,
ClientMetadataHandle& client_initial_metadata,
CallInitiator& call_initiator);
UnstartedCallHandler& unstarted_handler);
//
// Fields set at construction and never modified.
@ -181,7 +176,6 @@ class ClientChannel : public Channel {
ClientChannelFactory* client_channel_factory_;
std::string default_authority_;
channelz::ChannelNode* channelz_node_;
OrphanablePtr<CallDestination> call_destination_;
//
// State for LB calls.
@ -201,7 +195,7 @@ class ClientChannel : public Channel {
//
struct ResolverDataForCalls {
RefCountedPtr<ConfigSelector> config_selector;
RefCountedPtr<CallFilters::Stack> filter_stack;
RefCountedPtr<CallDestination> call_destination;
};
Observable<absl::StatusOr<ResolverDataForCalls>> resolver_data_for_calls_;

@ -97,23 +97,34 @@ DebugOnlyTraceFlag grpc_trace_subchannel_refcount(false, "subchannel_refcount");
//
ConnectedSubchannel::ConnectedSubchannel(
RefCountedPtr<grpc_channel_stack> channel_stack, const ChannelArgs& args,
const ChannelArgs& args,
RefCountedPtr<channelz::SubchannelNode> channelz_subchannel)
: RefCounted<ConnectedSubchannel>(
GRPC_TRACE_FLAG_ENABLED(grpc_trace_subchannel_refcount)
? "ConnectedSubchannel"
: nullptr),
channel_stack_(std::move(channel_stack)),
args_(args),
channelz_subchannel_(std::move(channelz_subchannel)) {}
ConnectedSubchannel::~ConnectedSubchannel() {
//
// LegacyConnectedSubchannel
//
class LegacyConnectedSubchannel : public ConnectedSubchannel {
public:
LegacyConnectedSubchannel(
RefCountedPtr<grpc_channel_stack> channel_stack, const ChannelArgs& args,
RefCountedPtr<channelz::SubchannelNode> channelz_subchannel)
: ConnectedSubchannel(args, std::move(channelz_subchannel)),
channel_stack_(std::move(channel_stack)) {}
~LegacyConnectedSubchannel() override {
channel_stack_.reset(DEBUG_LOCATION, "ConnectedSubchannel");
}
void ConnectedSubchannel::StartWatch(
void StartWatch(
grpc_pollset_set* interested_parties,
OrphanablePtr<ConnectivityStateWatcherInterface> watcher) {
OrphanablePtr<ConnectivityStateWatcherInterface> watcher) override {
grpc_transport_op* op = grpc_make_transport_op(nullptr);
op->start_connectivity_watch = std::move(watcher);
op->start_connectivity_watch_state = GRPC_CHANNEL_READY;
@ -122,23 +133,25 @@ void ConnectedSubchannel::StartWatch(
elem->filter->start_transport_op(elem, op);
}
void ConnectedSubchannel::Ping(grpc_closure* on_initiate,
grpc_closure* on_ack) {
grpc_transport_op* op = grpc_make_transport_op(nullptr);
grpc_channel_element* elem;
op->send_ping.on_initiate = on_initiate;
op->send_ping.on_ack = on_ack;
elem = grpc_channel_stack_element(channel_stack_, 0);
elem->filter->start_transport_op(elem, op);
void Ping(absl::AnyInvocable<void(absl::Status)> on_ack) override {
Crash("call v3 ping method called in legacy impl");
}
void StartCall(UnstartedCallHandler) override {
Crash("call v3 StartCall() method called in legacy impl");
}
size_t ConnectedSubchannel::GetInitialCallSizeEstimate() const {
grpc_channel_stack* channel_stack() const override {
return channel_stack_.get();
}
size_t GetInitialCallSizeEstimate() const override {
return GPR_ROUND_UP_TO_ALIGNMENT_SIZE(sizeof(SubchannelCall)) +
channel_stack_->call_stack_size;
}
ArenaPromise<ServerMetadataHandle> ConnectedSubchannel::MakeCallPromise(
CallArgs call_args) {
ArenaPromise<ServerMetadataHandle> MakeCallPromise(
CallArgs call_args) override {
// If not using channelz, we just need to call the channel stack.
if (channelz_subchannel() == nullptr) {
return channel_stack_->MakeClientCallPromise(std::move(call_args));
@ -167,6 +180,66 @@ ArenaPromise<ServerMetadataHandle> ConnectedSubchannel::MakeCallPromise(
});
}
void Ping(grpc_closure* on_initiate, grpc_closure* on_ack) override {
grpc_transport_op* op = grpc_make_transport_op(nullptr);
grpc_channel_element* elem;
op->send_ping.on_initiate = on_initiate;
op->send_ping.on_ack = on_ack;
elem = grpc_channel_stack_element(channel_stack_, 0);
elem->filter->start_transport_op(elem, op);
}
private:
RefCountedPtr<grpc_channel_stack> channel_stack_;
};
//
// NewConnectedSubchannel
//
class NewConnectedSubchannel : public ConnectedSubchannel {
public:
NewConnectedSubchannel(
RefCountedPtr<CallFilters::Stack> filter_stack,
OrphanablePtr<Transport> transport, const ChannelArgs& args,
RefCountedPtr<channelz::SubchannelNode> channelz_subchannel)
: ConnectedSubchannel(args, std::move(channelz_subchannel)),
filter_stack_(std::move(filter_stack)),
transport_(std::move(transport)) {}
void StartWatch(
grpc_pollset_set* interested_parties,
OrphanablePtr<ConnectivityStateWatcherInterface> watcher) override {
// FIXME: add new transport API for this in v3 stack
}
void Ping(absl::AnyInvocable<void(absl::Status)> on_ack) override {
// FIXME: add new transport API for this in v3 stack
}
void StartCall(UnstartedCallHandler unstarted_handler) override {
auto handler = unstarted_handler.StartCall(filter_stack_);
transport_->StartCall(std::move(handler));
}
grpc_channel_stack* channel_stack() const override { return nullptr; }
size_t GetInitialCallSizeEstimate() const override { return 0; }
ArenaPromise<ServerMetadataHandle> MakeCallPromise(
CallArgs call_args) override {
Crash("legacy MakeCallPromise() method called in call v3 impl");
}
void Ping(grpc_closure* on_initiate, grpc_closure* on_ack) override {
Crash("legacy ping method called in call v3 impl");
}
private:
RefCountedPtr<CallFilters::Stack> filter_stack_;
OrphanablePtr<Transport> transport_;
};
//
// SubchannelCall
//
@ -789,11 +862,18 @@ bool Subchannel::PublishTransportLocked() {
key_.ToString().c_str(), stack.status().ToString().c_str());
return false;
}
connected_subchannel_ = MakeRefCounted<ConnectedSubchannel>(
connected_subchannel_ = MakeRefCounted<LegacyConnectedSubchannel>(
std::move(stack), args_, channelz_node_);
} else {
// Call v3 stack.
// FIXME
CallFilters::StackBuilder builder;
// FIXME: add filters registered for CLIENT_SUBCHANNEL
auto filter_stack = builder.Build();
connected_subchannel_ = MakeRefCounted<NewConnectedSubchannel>(
std::move(filter_stack),
OrphanablePtr<Transport>(
std::exchange(connecting_result_.transport, nullptr)),
args_, channelz_node_);
}
connecting_result_.Reset();
// Publish.

@ -68,17 +68,6 @@ class SubchannelCall;
class ConnectedSubchannel : public CallDestination {
public:
// Ctor for legacy stack.
ConnectedSubchannel(
RefCountedPtr<grpc_channel_stack> channel_stack, const ChannelArgs& args,
RefCountedPtr<channelz::SubchannelNode> channelz_subchannel);
// Ctor for call v3 stack.
ConnectedSubchannel(
Transport* transport, const ChannelArgs& args,
RefCountedPtr<channelz::SubchannelNode> channelz_subchannel);
~ConnectedSubchannel() override;
void Orphan() override {}
const ChannelArgs& args() const { return args_; }
@ -86,22 +75,26 @@ class ConnectedSubchannel : public CallDestination {
return channelz_subchannel_.get();
}
void StartWatch(grpc_pollset_set* interested_parties,
OrphanablePtr<ConnectivityStateWatcherInterface> watcher);
virtual void StartWatch(
grpc_pollset_set* interested_parties,
OrphanablePtr<ConnectivityStateWatcherInterface> watcher) = 0;
// Methods for v3 stack.
RefCountedPtr<CallFilters::Stack> GetStack();
void StartCall(CallHandler call_handler) override;
void Ping(absl::AnyInvocable<void(absl::Status)> on_ack);
virtual void Ping(absl::AnyInvocable<void(absl::Status)> on_ack) = 0;
// Methods for legacy stack.
grpc_channel_stack* channel_stack() const { return channel_stack_.get(); }
size_t GetInitialCallSizeEstimate() const;
ArenaPromise<ServerMetadataHandle> MakeCallPromise(CallArgs call_args);
void Ping(grpc_closure* on_initiate, grpc_closure* on_ack);
virtual grpc_channel_stack* channel_stack() const = 0;
virtual size_t GetInitialCallSizeEstimate() const = 0;
virtual ArenaPromise<ServerMetadataHandle> MakeCallPromise(
CallArgs call_args) = 0;
virtual void Ping(grpc_closure* on_initiate, grpc_closure* on_ack) = 0;
protected:
ConnectedSubchannel(
const ChannelArgs& args,
RefCountedPtr<channelz::SubchannelNode> channelz_subchannel);
private:
RefCountedPtr<grpc_channel_stack> channel_stack_;
ChannelArgs args_;
// ref counted pointer to the channelz node in this connected subchannel's
// owning subchannel.

@ -69,9 +69,6 @@ class Channel : public RefCounted<Channel>,
virtual void Orphan() = 0;
Arena* CreateArena();
void DestroyArena(Arena* arena);
virtual bool IsLame() const = 0;
// TODO(roth): This should return a C++ type.
@ -126,6 +123,8 @@ class Channel : public RefCounted<Channel>,
virtual void Ping(grpc_completion_queue* cq, void* tag) = 0;
// TODO(roth): Remove these methods when LegacyChannel goes away.
Arena* CreateArena();
void DestroyArena(Arena* arena);
virtual grpc_channel_stack* channel_stack() const { return nullptr; }
virtual bool is_client() const { return true; }
virtual bool is_promising() const { return true; }

@ -99,19 +99,6 @@ absl::StatusOr<OrphanablePtr<Channel>> LegacyChannel::Create(
builder.IsPromising(), std::move(target), args, std::move(*r));
}
namespace {
class NotReallyACallFactory final : public CallFactory {
public:
using CallFactory::CallFactory;
void Orphan() override { delete this; }
CallInitiator CreateCall(ClientMetadataHandle, Arena*) override {
Crash("NotReallyACallFactory::CreateCall should never be called");
}
};
} // namespace
LegacyChannel::LegacyChannel(bool is_client, bool is_promising,
std::string target,
const ChannelArgs& channel_args,

@ -1,35 +0,0 @@
// Copyright 2024 gRPC authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef GRPC_SRC_CORE_LIB_TRANSPORT_CALL_DESTINATION_H
#define GRPC_SRC_CORE_LIB_TRANSPORT_CALL_DESTINATION_H
#include <grpc/support/port_platform.h>
#include "src/core/lib/gprpp/dual_ref_counted.h"
#include "src/core/lib/transport/call_spine.h"
namespace grpc_core {
// CallDestination is responsible for the processing of a CallHandler.
// It might be a transport, the server API, or a subchannel on the client (for
// instance).
class CallDestination : public DualRefCounted<CallDestination> {
public:
virtual void StartCall(CallHandler call_handler) = 0;
};
} // namespace grpc_core
#endif // GRPC_SRC_CORE_LIB_TRANSPORT_CALL_DESTINATION_H

@ -89,10 +89,28 @@ void ForwardCall(CallHandler call_handler, CallInitiator call_initiator) {
});
}
CallHandler UnstartedCallHandler::StartCall(
RefCountedPtr<CallFilters::Stack> stack,
RefCountedPtr<CallDestination> call_destination) {
call_initiator->SpawnGuarded(
"send_initial_metadata",
[client_initial_metadata = std::move(client_initial_metadata_),
spine = spine_]() mutable {
GPR_DEBUG_ASSERT(GetContext<Activity>() == &spine->party());
return Map(spine_->client_initial_metadata().sender.Push(
std::move(client_initial_metadata)),
[](bool ok) { return StatusFlag(ok); });
});
// FIXME: attach stack and destination to CallHandler
return CallHandler(std::move(spine_));
}
CallInitiatorAndHandler MakeCall(
ClientMetadataHandle client_initial_metadata,
grpc_event_engine::experimental::EventEngine* event_engine, Arena* arena) {
auto spine = CallSpine::Create(event_engine, arena);
return {CallInitiator(spine), CallHandler(spine)};
return {CallInitiator(spine),
UnstartedCallHandler(std::move(client_initial_metadata), spine)};
}
} // namespace grpc_core

@ -19,6 +19,7 @@
#include <grpc/support/log.h>
#include "src/core/lib/gprpp/dual_ref_counted.h"
#include "src/core/lib/promise/detail/status.h"
#include "src/core/lib/promise/for_each.h"
#include "src/core/lib/promise/if.h"
@ -228,6 +229,10 @@ class CallInitiator {
explicit CallInitiator(RefCountedPtr<CallSpineInterface> spine)
: spine_(std::move(spine)) {}
// FIXME: should this method go away, since we now want this to be
// handled in UnstartedCallHandler instead?
// Note: this would require changing ServerTransport::AcceptCall() to
// act like a CallDestination
auto PushClientInitialMetadata(ClientMetadataHandle md) {
GPR_DEBUG_ASSERT(GetContext<Activity>() == &spine_->party());
return Map(spine_->client_initial_metadata().sender.Push(std::move(md)),
@ -323,10 +328,6 @@ class CallHandler {
explicit CallHandler(RefCountedPtr<CallSpineInterface> spine)
: spine_(std::move(spine)) {}
void SetStack(RefCountedPtr<CallFilters::Stack> stack) {
// FIXME: Implement.
}
auto PullClientInitialMetadata() {
GPR_DEBUG_ASSERT(GetContext<Activity>() == &spine_->party());
return Map(spine_->client_initial_metadata().receiver.Next(),
@ -407,12 +408,110 @@ class CallHandler {
RefCountedPtr<CallSpineInterface> spine_;
};
class UnstartedCallHandler;
// CallDestination is responsible for starting an UnstartedCallHandler
// and then processing operations on the resulting CallHandler.
//
// Examples of CallDestinations include:
// - a client transport
// - the server API
// - a load-balanced call in the client channel
// - a hijacking filter (see DelegatingCallDestination below)
//
// FIXME: do we want this to be ref-counted? that might not be
// desirable for the hijacking filter case, where we want the filter stack
// to own the filter rather than having every call take its own ref to every
// hijacking filter.
class CallDestination : public DualRefCounted<CallDestination> {
public:
virtual void StartCall(UnstartedCallHandler unstarted_call_handler) = 0;
};
// A delegating CallDestination for use as a hijacking filter.
// Implementations may look at the unprocessed initial metadata
// and decide to do one of two things:
//
// 1. It can be a no-op. In this case, it will simply pass the
// unstarted_call_handler to the wrapped CallDestination.
//
// 2. It can hijack the call by doing the following:
// - Start unstarted_call_handler and take ownership of the
// resulting handler.
// - Create a new CallInitiator/UnstartedCallHandler pair, and pass
// that new UnstartedCallHandler down to the wrapped CallDestination.
// - The implementation is then responsible for forwarding between
// the started handler and the new initiator. Note that in
// simple cases, this can be done via ForwardCall().
class DelegatingCallDestination : public CallDestination {
protected:
explicit DelegatingCallDestination(
RefCountedPtr<CallDestination> wrapped_destination)
: wrapped_destination_(std::move(wrapped_destination)) {}
CallDestination* wrapped_destination() const {
return wrapped_destination_.get();
}
private:
RefCountedPtr<CallDestination> wrapped_destination_;
};
class UnstartedCallHandler {
public:
UnstartedCallHandler(RefCountedPtr<CallSpineInterface> spine,
ClientMetadataHandle client_initial_metadata)
: spine_(std::move(spine)),
client_initial_metadata_(std::move(client_initial_metadata)) {}
// Returns the client initial metadata, which has not yet been
// processed by the stack that will ultimately be used for this call.
ClientMetadataHandle& UnprocessedClientInitialMetadata() {
return client_initial_metadata_;
}
// Starts the call using the specified stack.
// This must be called only once, and the UnstartedCallHandler object
// may not be used after this is called.
CallHandler StartCall(RefCountedPtr<CallFilters::Stack> stack);
template <typename ContextType>
void SetContext(ContextType context) {
// FIXME: implement
}
template <typename Promise>
auto CancelIfFails(Promise promise) {
return spine_->CancelIfFails(std::move(promise));
}
template <typename PromiseFactory>
void SpawnGuarded(absl::string_view name, PromiseFactory promise_factory) {
spine_->SpawnGuarded(name, std::move(promise_factory));
}
template <typename PromiseFactory>
void SpawnInfallible(absl::string_view name, PromiseFactory promise_factory) {
spine_->SpawnInfallible(name, std::move(promise_factory));
}
template <typename PromiseFactory>
auto SpawnWaitable(absl::string_view name, PromiseFactory promise_factory) {
return spine_->party().SpawnWaitable(name, std::move(promise_factory));
}
private:
RefCountedPtr<CallSpineInterface> spine_;
ClientMetadataHandle client_initial_metadata_;
};
struct CallInitiatorAndHandler {
CallInitiator initiator;
CallHandler handler;
UnstartedCallHandler unstarted_handler;
};
CallInitiatorAndHandler MakeCall(
ClientMetadataHandle client_initial_metadata,
grpc_event_engine::experimental::EventEngine* event_engine, Arena* arena);
template <typename CallHalf>

@ -29,6 +29,7 @@
#include <utility>
#include "absl/container/inlined_vector.h"
#include "absl/functional/any_invocable.h"
#include "absl/functional/function_ref.h"
#include "absl/meta/type_traits.h"
#include "absl/strings/numbers.h"
@ -503,6 +504,25 @@ struct WaitForReady {
static std::string DisplayValue(ValueType x);
};
// Annotation added by retry code to indicate a transparent retry.
struct IsTransparentRetry {
static absl::string_view DebugKey() { return "IsTransparentRetry"; }
static constexpr bool kRepeatable = false;
using ValueType = bool;
static std::string DisplayValue(ValueType x) { return x ? "true" : "false"; }
};
// Annotation added by retry or no-retry filters to pass the on-commit
// callback to the load balanced call.
struct LoadBalancingOnCommit {
static absl::string_view DebugKey() { return "LoadBalancingOnCommit"; }
static constexpr bool kRepeatable = false;
using ValueType = absl::AnyInvocable<void()>;
static std::string DisplayValue(const ValueType& x) {
return x == nullptr ? "unset" : "set";
}
};
// Annotation added by a transport to note that server trailing metadata
// is a Trailers-Only response.
struct GrpcTrailersOnly {
@ -1511,6 +1531,7 @@ using grpc_metadata_batch_base = grpc_core::MetadataMap<
grpc_core::GrpcStreamNetworkState, grpc_core::PeerString,
grpc_core::GrpcStatusContext, grpc_core::GrpcStatusFromWire,
grpc_core::GrpcCallWasCancelled, grpc_core::WaitForReady,
grpc_core::IsTransparentRetry, grpc_core::LoadBalancingOnCommit,
grpc_core::GrpcTrailersOnly, grpc_core::GrpcTarPit,
grpc_core::GrpcRegisteredMethod GRPC_CUSTOM_CLIENT_METADATA
GRPC_CUSTOM_SERVER_METADATA>;

@ -540,6 +540,7 @@ class FilterStackTransport {
~FilterStackTransport() = default;
};
// FIXME: should this just be an alias for CallDestination?
class ClientTransport {
public:
virtual void StartCall(CallHandler call_handler) = 0;

@ -2806,7 +2806,6 @@ src/core/lib/transport/batch_builder.cc \
src/core/lib/transport/batch_builder.h \
src/core/lib/transport/bdp_estimator.cc \
src/core/lib/transport/bdp_estimator.h \
src/core/lib/transport/call_destination.h \
src/core/lib/transport/call_filters.cc \
src/core/lib/transport/call_filters.h \
src/core/lib/transport/call_final_info.cc \

@ -2583,7 +2583,6 @@ src/core/lib/transport/batch_builder.cc \
src/core/lib/transport/batch_builder.h \
src/core/lib/transport/bdp_estimator.cc \
src/core/lib/transport/bdp_estimator.h \
src/core/lib/transport/call_destination.h \
src/core/lib/transport/call_filters.cc \
src/core/lib/transport/call_filters.h \
src/core/lib/transport/call_final_info.cc \

Loading…
Cancel
Save