mirror of https://github.com/grpc/grpc.git
[chaotic-good] Fork a legacy copy of the transport stack again (#38786)
adding a legacy version of chaotic good, used when a new experiment is disabled.
Aim will be to merge this, then land #38286 atop it, putting that change under the control of this experiment.
Closes #38786
COPYBARA_INTEGRATE_REVIEW=https://github.com/grpc/grpc/pull/38786 from ctiller:legacy-again e2b0c59817
PiperOrigin-RevId: 728822723
pull/35404/merge
parent
dbb582a7e0
commit
d8a31278c3
46 changed files with 10398 additions and 352 deletions
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,252 @@ |
||||
// Copyright 2023 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CHAOTIC_GOOD_TRANSPORT_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CHAOTIC_GOOD_TRANSPORT_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstdint> |
||||
#include <limits> |
||||
#include <memory> |
||||
#include <utility> |
||||
|
||||
#include "absl/strings/escaping.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/control_endpoint.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/data_endpoints.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/lib/debug/trace.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/event_engine/tcp_socket_utils.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/match_promise.h" |
||||
#include "src/core/lib/promise/mpsc.h" |
||||
#include "src/core/lib/promise/seq.h" |
||||
#include "src/core/lib/promise/try_join.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
#include "src/core/lib/transport/call_spine.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
inline std::vector<PromiseEndpoint> OneDataEndpoint(PromiseEndpoint endpoint) { |
||||
std::vector<PromiseEndpoint> ep; |
||||
ep.emplace_back(std::move(endpoint)); |
||||
return ep; |
||||
} |
||||
|
||||
// One received frame: the header, and the serialized bytes of the payload.
|
||||
// The payload may not yet be received into memory, so the accessor for that
|
||||
// returns a promise that will need to be resolved prior to inspecting the
|
||||
// bytes.
|
||||
// In this way we can pull bytes from various different data connections and
|
||||
// read them in any order, but still have a trivial reassembly in the receiving
|
||||
// call promise.
|
||||
class IncomingFrame { |
||||
public: |
||||
template <typename T> |
||||
IncomingFrame(FrameHeader header, T payload, size_t remove_padding) |
||||
: header_(header), |
||||
payload_(std::move(payload)), |
||||
remove_padding_(remove_padding) {} |
||||
|
||||
const FrameHeader& header() { return header_; } |
||||
|
||||
auto Payload() { |
||||
return Map( |
||||
MatchPromise( |
||||
std::move(payload_), |
||||
[](absl::StatusOr<SliceBuffer> status) { return status; }, |
||||
[](DataEndpoints::ReadTicket ticket) { return ticket.Await(); }), |
||||
[remove_padding = |
||||
remove_padding_](absl::StatusOr<SliceBuffer> payload) { |
||||
if (payload.ok()) payload->RemoveLastNBytesNoInline(remove_padding); |
||||
return payload; |
||||
}); |
||||
} |
||||
|
||||
private: |
||||
FrameHeader header_; |
||||
std::variant<absl::StatusOr<SliceBuffer>, DataEndpoints::ReadTicket> payload_; |
||||
size_t remove_padding_; |
||||
}; |
||||
|
||||
class ChaoticGoodTransport : public RefCounted<ChaoticGoodTransport> { |
||||
public: |
||||
struct Options { |
||||
uint32_t encode_alignment = 64; |
||||
uint32_t decode_alignment = 64; |
||||
uint32_t inlined_payload_size_threshold = 8 * 1024; |
||||
}; |
||||
|
||||
ChaoticGoodTransport( |
||||
PromiseEndpoint control_endpoint, |
||||
std::vector<PendingConnection> pending_data_endpoints, |
||||
std::shared_ptr<grpc_event_engine::experimental::EventEngine> |
||||
event_engine, |
||||
Options options, bool enable_tracing) |
||||
: event_engine_(std::move(event_engine)), |
||||
control_endpoint_(std::move(control_endpoint), event_engine_.get()), |
||||
data_endpoints_(std::move(pending_data_endpoints), event_engine_.get(), |
||||
enable_tracing), |
||||
options_(options) {} |
||||
|
||||
auto WriteFrame(const FrameInterface& frame) { |
||||
FrameHeader header = frame.MakeHeader(); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: WriteFrame to:" |
||||
<< ResolvedAddressToString(control_endpoint_.GetPeerAddress()) |
||||
.value_or("<<unknown peer address>>") |
||||
<< " " << frame.ToString(); |
||||
return If( |
||||
// If we have no data endpoints, OR this is a small payload
|
||||
data_endpoints_.empty() || |
||||
header.payload_length <= options_.inlined_payload_size_threshold, |
||||
// ... then write it to the control endpoint
|
||||
[this, &header, &frame]() { |
||||
SliceBuffer output; |
||||
header.Serialize(output.AddTiny(FrameHeader::kFrameHeaderSize)); |
||||
frame.SerializePayload(output); |
||||
return control_endpoint_.Write(std::move(output)); |
||||
}, |
||||
// ... otherwise write it to a data connection
|
||||
[this, header, &frame]() mutable { |
||||
SliceBuffer payload; |
||||
// Temporarily give a bogus connection id to get padding right
|
||||
header.payload_connection_id = 1; |
||||
const size_t padding = header.Padding(options_.encode_alignment); |
||||
frame.SerializePayload(payload); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Send " << payload.Length() |
||||
<< "b payload on data channel; add " << padding << " bytes for " |
||||
<< options_.encode_alignment << " alignment"; |
||||
if (padding != 0) { |
||||
auto slice = MutableSlice::CreateUninitialized(padding); |
||||
memset(slice.data(), 0, padding); |
||||
payload.AppendIndexed(Slice(std::move(slice))); |
||||
} |
||||
return Seq(data_endpoints_.Write(std::move(payload)), |
||||
[this, header](uint32_t connection_id) mutable { |
||||
header.payload_connection_id = connection_id + 1; |
||||
SliceBuffer header_frame; |
||||
header.Serialize( |
||||
header_frame.AddTiny(FrameHeader::kFrameHeaderSize)); |
||||
return control_endpoint_.Write(std::move(header_frame)); |
||||
}); |
||||
}); |
||||
} |
||||
|
||||
// Common outbound loop for both client and server (these vary only over the
|
||||
// frame type).
|
||||
template <typename Frame> |
||||
auto TransportWriteLoop(MpscReceiver<Frame>& outgoing_frames) { |
||||
return Loop([self = Ref(), &outgoing_frames] { |
||||
return TrySeq( |
||||
// Get next outgoing frame.
|
||||
outgoing_frames.Next(), |
||||
// Serialize and write it out.
|
||||
[self = self.get()](Frame client_frame) { |
||||
return self->WriteFrame( |
||||
absl::ConvertVariantTo<FrameInterface&>(client_frame)); |
||||
}, |
||||
[]() -> LoopCtl<absl::Status> { |
||||
// The write failures will be caught in TrySeq and exit loop.
|
||||
// Therefore, only need to return Continue() in the last lambda
|
||||
// function.
|
||||
return Continue(); |
||||
}); |
||||
}); |
||||
} |
||||
|
||||
// Read frame header and payloads for control and data portions of one frame.
|
||||
// Resolves to StatusOr<IncomingFrame>.
|
||||
auto ReadFrameBytes() { |
||||
return TrySeq( |
||||
control_endpoint_.ReadSlice(FrameHeader::kFrameHeaderSize), |
||||
[this](Slice read_buffer) { |
||||
auto frame_header = |
||||
FrameHeader::Parse(reinterpret_cast<const uint8_t*>( |
||||
GRPC_SLICE_START_PTR(read_buffer.c_slice()))); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: ReadHeader from:" |
||||
<< ResolvedAddressToString(control_endpoint_.GetPeerAddress()) |
||||
.value_or("<<unknown peer address>>") |
||||
<< " " |
||||
<< (frame_header.ok() ? frame_header->ToString() |
||||
: frame_header.status().ToString()); |
||||
return frame_header; |
||||
}, |
||||
[this](FrameHeader frame_header) { |
||||
return If( |
||||
// If the payload is on the connection frame
|
||||
frame_header.payload_connection_id == 0, |
||||
// ... then read the data immediately and return an IncomingFrame
|
||||
// that contains the payload.
|
||||
// We need to do this here so that we do not create head of line
|
||||
// blocking issues reading later control frames (but waiting for a
|
||||
// call to get scheduled time to read the payload).
|
||||
[this, frame_header]() { |
||||
return Map(control_endpoint_.Read(frame_header.payload_length), |
||||
[frame_header](absl::StatusOr<SliceBuffer> payload) |
||||
-> absl::StatusOr<IncomingFrame> { |
||||
if (!payload.ok()) return payload.status(); |
||||
return IncomingFrame(frame_header, |
||||
std::move(payload), 0); |
||||
}); |
||||
}, |
||||
// ... otherwise issue a read to the appropriate data endpoint,
|
||||
// which will return a read ticket - which can be used later
|
||||
// in the call promise to asynchronously wait for those bytes
|
||||
// to be available.
|
||||
[this, frame_header]() -> absl::StatusOr<IncomingFrame> { |
||||
const auto padding = |
||||
frame_header.Padding(options_.decode_alignment); |
||||
return IncomingFrame( |
||||
frame_header, |
||||
data_endpoints_.Read(frame_header.payload_connection_id - 1, |
||||
frame_header.payload_length + padding), |
||||
padding); |
||||
}); |
||||
}); |
||||
} |
||||
|
||||
template <typename T> |
||||
absl::StatusOr<T> DeserializeFrame(const FrameHeader& header, |
||||
SliceBuffer payload) { |
||||
T frame; |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Deserialize " << header << " with payload " |
||||
<< absl::CEscape(payload.JoinIntoString()); |
||||
CHECK_EQ(header.payload_length, payload.Length()); |
||||
auto s = frame.Deserialize(header, std::move(payload)); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: DeserializeFrame " |
||||
<< (s.ok() ? frame.ToString() : s.ToString()); |
||||
if (s.ok()) return std::move(frame); |
||||
return std::move(s); |
||||
} |
||||
|
||||
private: |
||||
std::shared_ptr<grpc_event_engine::experimental::EventEngine> event_engine_; |
||||
ControlEndpoint control_endpoint_; |
||||
DataEndpoints data_endpoints_; |
||||
const Options options_; |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CHAOTIC_GOOD_TRANSPORT_H
|
@ -0,0 +1,334 @@ |
||||
// 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.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/client/chaotic_good_connector.h" |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstdint> |
||||
#include <memory> |
||||
#include <utility> |
||||
|
||||
#include "absl/log/check.h" |
||||
#include "absl/log/log.h" |
||||
#include "absl/random/bit_gen_ref.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/client_channel/client_channel_factory.h" |
||||
#include "src/core/client_channel/client_channel_filter.h" |
||||
#include "src/core/config/core_configuration.h" |
||||
#include "src/core/ext/transport/chaotic_good/chaotic_good_frame.pb.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/client_transport.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/handshaker/handshaker.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/event_engine/channel_args_endpoint_config.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/event_engine/extensions/chaotic_good_extension.h" |
||||
#include "src/core/lib/event_engine/query_extensions.h" |
||||
#include "src/core/lib/event_engine/tcp_socket_utils.h" |
||||
#include "src/core/lib/iomgr/closure.h" |
||||
#include "src/core/lib/iomgr/error.h" |
||||
#include "src/core/lib/iomgr/event_engine_shims/endpoint.h" |
||||
#include "src/core/lib/iomgr/exec_ctx.h" |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/all_ok.h" |
||||
#include "src/core/lib/promise/context.h" |
||||
#include "src/core/lib/promise/event_engine_wakeup_scheduler.h" |
||||
#include "src/core/lib/promise/latch.h" |
||||
#include "src/core/lib/promise/race.h" |
||||
#include "src/core/lib/promise/sleep.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
#include "src/core/lib/promise/wait_for_callback.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/surface/channel.h" |
||||
#include "src/core/lib/surface/channel_create.h" |
||||
#include "src/core/lib/transport/error_utils.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/util/debug_location.h" |
||||
#include "src/core/util/no_destruct.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
#include "src/core/util/time.h" |
||||
|
||||
using grpc_event_engine::experimental::ChannelArgsEndpointConfig; |
||||
using grpc_event_engine::experimental::EventEngine; |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
namespace { |
||||
|
||||
const int32_t kTimeoutSecs = 120; |
||||
|
||||
struct ConnectPromiseEndpointResult { |
||||
PromiseEndpoint endpoint; |
||||
ChannelArgs channel_args; |
||||
}; |
||||
|
||||
using ConnectResultLatch = std::shared_ptr< |
||||
InterActivityLatch<absl::StatusOr<ConnectPromiseEndpointResult>>>; |
||||
|
||||
absl::StatusOr<ConnectPromiseEndpointResult> ResultFromHandshake( |
||||
absl::StatusOr<HandshakerArgs*> result) { |
||||
if (!result.ok()) { |
||||
return result.status(); |
||||
} |
||||
HandshakerArgs* args = *result; |
||||
if (args->endpoint == nullptr) { |
||||
return absl::InternalError("Handshake complete with empty endpoint."); |
||||
} |
||||
return ConnectPromiseEndpointResult{ |
||||
PromiseEndpoint(grpc_event_engine::experimental:: |
||||
grpc_take_wrapped_event_engine_endpoint( |
||||
(*result)->endpoint.release()), |
||||
std::move(args->read_buffer)), |
||||
args->args}; |
||||
} |
||||
|
||||
void OnConnect(absl::StatusOr<std::unique_ptr<EventEngine::Endpoint>> endpoint, |
||||
RefCountedPtr<HandshakeManager> handshake_mgr, |
||||
const ChannelArgs& channel_args, Timestamp deadline, |
||||
ConnectResultLatch result_latch) { |
||||
if (!endpoint.ok()) { |
||||
auto endpoint_status = endpoint.status(); |
||||
auto error = GRPC_ERROR_CREATE_REFERENCING("connect endpoint failed", |
||||
&endpoint_status, 1); |
||||
result_latch->Set(error); |
||||
return; |
||||
} |
||||
auto* chaotic_good_ext = grpc_event_engine::experimental::QueryExtension< |
||||
grpc_event_engine::experimental::ChaoticGoodExtension>(endpoint->get()); |
||||
if (chaotic_good_ext != nullptr) { |
||||
chaotic_good_ext->EnableStatsCollection(/*is_control_channel=*/true); |
||||
chaotic_good_ext->UseMemoryQuota(ResourceQuota::Default()->memory_quota()); |
||||
} |
||||
handshake_mgr->DoHandshake( |
||||
OrphanablePtr<grpc_endpoint>( |
||||
grpc_event_engine_endpoint_create(std::move(*endpoint))), |
||||
channel_args, deadline, nullptr /* acceptor */, |
||||
[result_latch = std::move(result_latch), |
||||
handshake_mgr](absl::StatusOr<HandshakerArgs*> result) { |
||||
result_latch->Set(ResultFromHandshake(std::move(result))); |
||||
}); |
||||
} |
||||
|
||||
auto ConnectPromiseEndpoint(EventEngine::ResolvedAddress addr, |
||||
const ChannelArgs& channel_args, |
||||
Timestamp deadline) { |
||||
auto event_engine = channel_args.GetObjectRef<EventEngine>(); |
||||
auto result_latch = std::make_shared< |
||||
InterActivityLatch<absl::StatusOr<ConnectPromiseEndpointResult>>>(); |
||||
auto handshake_mgr = MakeRefCounted<HandshakeManager>(); |
||||
auto connect_hdl = event_engine->Connect( |
||||
[result_latch, channel_args, handshake_mgr, |
||||
deadline](absl::StatusOr<std::unique_ptr<EventEngine::Endpoint>> |
||||
endpoint) mutable { |
||||
ExecCtx exec_ctx; |
||||
OnConnect(std::move(endpoint), std::move(handshake_mgr), channel_args, |
||||
deadline, std::move(result_latch)); |
||||
}, |
||||
addr, ChannelArgsEndpointConfig(channel_args), |
||||
ResourceQuota::Default()->memory_quota()->CreateMemoryAllocator( |
||||
"data_endpoint_connection"), |
||||
std::chrono::seconds(kTimeoutSecs)); |
||||
return OnCancel( |
||||
[result_latch, await = result_latch->Wait()]() { return await(); }, |
||||
[handshake_mgr, connect_hdl, event_engine]() { |
||||
handshake_mgr->Shutdown(absl::CancelledError()); |
||||
event_engine->CancelConnect(connect_hdl); |
||||
}); |
||||
} |
||||
|
||||
struct ConnectChaoticGoodResult { |
||||
ConnectPromiseEndpointResult connect_result; |
||||
chaotic_good_frame::Settings server_settings; |
||||
}; |
||||
|
||||
class SettingsHandshake : public RefCounted<SettingsHandshake> { |
||||
public: |
||||
explicit SettingsHandshake(ConnectPromiseEndpointResult connect_result) |
||||
: connect_result_(std::move(connect_result)) {} |
||||
|
||||
auto Handshake(chaotic_good_frame::Settings client_settings) { |
||||
SettingsFrame frame; |
||||
frame.body = client_settings; |
||||
SliceBuffer send_buffer; |
||||
frame.MakeHeader().Serialize( |
||||
send_buffer.AddTiny(FrameHeader::kFrameHeaderSize)); |
||||
frame.SerializePayload(send_buffer); |
||||
return TrySeq( |
||||
connect_result_.endpoint.Write(std::move(send_buffer)), |
||||
[this]() { |
||||
return connect_result_.endpoint.ReadSlice( |
||||
FrameHeader::kFrameHeaderSize); |
||||
}, |
||||
[](Slice frame_header) { |
||||
return FrameHeader::Parse(frame_header.data()); |
||||
}, |
||||
[this](FrameHeader frame_header) { |
||||
server_header_ = frame_header; |
||||
return connect_result_.endpoint.Read(frame_header.payload_length); |
||||
}, |
||||
[this](SliceBuffer payload) { |
||||
return server_frame_.Deserialize(server_header_, std::move(payload)); |
||||
}, |
||||
[self = Ref()]() { |
||||
return ConnectChaoticGoodResult{std::move(self->connect_result_), |
||||
std::move(self->server_frame_.body)}; |
||||
}); |
||||
} |
||||
|
||||
private: |
||||
ConnectPromiseEndpointResult connect_result_; |
||||
FrameHeader server_header_; |
||||
SettingsFrame server_frame_; |
||||
}; |
||||
|
||||
auto ConnectChaoticGood(EventEngine::ResolvedAddress addr, |
||||
const ChannelArgs& channel_args, Timestamp deadline, |
||||
chaotic_good_frame::Settings client_settings) { |
||||
return TrySeq( |
||||
ConnectPromiseEndpoint(addr, channel_args, deadline), |
||||
[client_settings](ConnectPromiseEndpointResult connect_result) { |
||||
return MakeRefCounted<SettingsHandshake>(std::move(connect_result)) |
||||
->Handshake(client_settings); |
||||
}); |
||||
} |
||||
|
||||
} // namespace
|
||||
|
||||
void ChaoticGoodConnector::Connect(const Args& args, Result* result, |
||||
grpc_closure* notify) { |
||||
auto event_engine = args.channel_args.GetObjectRef<EventEngine>(); |
||||
auto arena = SimpleArenaAllocator(0)->MakeArena(); |
||||
auto result_notifier = std::make_unique<ResultNotifier>(args, result, notify); |
||||
arena->SetContext(event_engine.get()); |
||||
auto resolved_addr = EventEngine::ResolvedAddress( |
||||
reinterpret_cast<const sockaddr*>(args.address->addr), args.address->len); |
||||
CHECK_NE(resolved_addr.address(), nullptr); |
||||
auto* result_notifier_ptr = result_notifier.get(); |
||||
auto activity = MakeActivity( |
||||
[result_notifier_ptr, resolved_addr]() mutable { |
||||
chaotic_good_frame::Settings client_settings; |
||||
client_settings.set_data_channel(false); |
||||
result_notifier_ptr->config.PrepareClientOutgoingSettings( |
||||
client_settings); |
||||
return TrySeq( |
||||
ConnectChaoticGood( |
||||
resolved_addr, result_notifier_ptr->args.channel_args, |
||||
Timestamp::Now() + Duration::FromSecondsAsDouble(kTimeoutSecs), |
||||
std::move(client_settings)), |
||||
[resolved_addr, |
||||
result_notifier_ptr](ConnectChaoticGoodResult result) { |
||||
auto connector = MakeRefCounted<ConnectionCreator>( |
||||
resolved_addr, result.connect_result.channel_args); |
||||
auto parse_status = |
||||
result_notifier_ptr->config.ReceiveServerIncomingSettings( |
||||
result.server_settings, *connector); |
||||
if (!parse_status.ok()) { |
||||
return parse_status; |
||||
} |
||||
auto transport = MakeOrphanable<ChaoticGoodClientTransport>( |
||||
result.connect_result.channel_args, |
||||
std::move(result.connect_result.endpoint), |
||||
std::move(result_notifier_ptr->config), std::move(connector)); |
||||
result_notifier_ptr->result->transport = transport.release(); |
||||
result_notifier_ptr->result->channel_args = |
||||
result.connect_result.channel_args; |
||||
return absl::OkStatus(); |
||||
}); |
||||
}, |
||||
EventEngineWakeupScheduler(event_engine), |
||||
[result_notifier = std::move(result_notifier)](absl::Status status) { |
||||
result_notifier->Run(status); |
||||
}, |
||||
arena); |
||||
MutexLock lock(&mu_); |
||||
if (is_shutdown_) return; |
||||
connect_activity_ = std::move(activity); |
||||
} |
||||
|
||||
PendingConnection ChaoticGoodConnector::ConnectionCreator::Connect( |
||||
absl::string_view id) { |
||||
chaotic_good_frame::Settings settings; |
||||
settings.set_data_channel(true); |
||||
settings.add_connection_id(id); |
||||
return PendingConnection( |
||||
id, |
||||
Map(ConnectChaoticGood( |
||||
address_, args_, |
||||
Timestamp::Now() + Duration::FromSecondsAsDouble(kTimeoutSecs), |
||||
std::move(settings)), |
||||
[](absl::StatusOr<ConnectChaoticGoodResult> result) |
||||
-> absl::StatusOr<PromiseEndpoint> { |
||||
if (!result.ok()) return result.status(); |
||||
return std::move(result->connect_result.endpoint); |
||||
})); |
||||
} |
||||
|
||||
namespace { |
||||
|
||||
class ChaoticGoodChannelFactory final : public ClientChannelFactory { |
||||
public: |
||||
RefCountedPtr<Subchannel> CreateSubchannel( |
||||
const grpc_resolved_address& address, const ChannelArgs& args) override { |
||||
return Subchannel::Create(MakeOrphanable<ChaoticGoodConnector>(), address, |
||||
args); |
||||
} |
||||
}; |
||||
|
||||
} // namespace
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
grpc_channel* grpc_chaotic_good_legacy_channel_create( |
||||
const char* target, const grpc_channel_args* args) { |
||||
grpc_core::ExecCtx exec_ctx; |
||||
GRPC_TRACE_LOG(api, INFO) |
||||
<< "grpc_chaotic_good_channel_create(target=" << target |
||||
<< ", args=" << (void*)args << ")"; |
||||
grpc_channel* channel = nullptr; |
||||
grpc_error_handle error; |
||||
// Create channel.
|
||||
auto r = grpc_core::ChannelCreate( |
||||
target, |
||||
grpc_core::CoreConfiguration::Get() |
||||
.channel_args_preconditioning() |
||||
.PreconditionChannelArgs(args) |
||||
.SetObject( |
||||
grpc_core::NoDestructSingleton< |
||||
grpc_core::chaotic_good_legacy::ChaoticGoodChannelFactory>:: |
||||
Get()) |
||||
.Set(GRPC_ARG_USE_V3_STACK, true), |
||||
GRPC_CLIENT_CHANNEL, nullptr); |
||||
if (r.ok()) { |
||||
return r->release()->c_ptr(); |
||||
} |
||||
LOG(ERROR) << "Failed to create chaotic good client channel: " << r.status(); |
||||
error = absl_status_to_grpc_error(r.status()); |
||||
intptr_t integer; |
||||
grpc_status_code status = GRPC_STATUS_INTERNAL; |
||||
if (grpc_error_get_int(error, grpc_core::StatusIntProperty::kRpcStatus, |
||||
&integer)) { |
||||
status = static_cast<grpc_status_code>(integer); |
||||
} |
||||
channel = grpc_lame_client_channel_create( |
||||
target, status, "Failed to create chaotic good client channel"); |
||||
return channel; |
||||
} |
@ -0,0 +1,101 @@ |
||||
// 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CLIENT_CHAOTIC_GOOD_CONNECTOR_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CLIENT_CHAOTIC_GOOD_CONNECTOR_H |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstddef> |
||||
#include <cstdint> |
||||
#include <memory> |
||||
|
||||
#include "absl/random/random.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/client_channel/connector.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/config.h" |
||||
#include "src/core/handshaker/handshaker.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/event_engine/channel_args_endpoint_config.h" |
||||
#include "src/core/lib/iomgr/closure.h" |
||||
#include "src/core/lib/iomgr/endpoint.h" |
||||
#include "src/core/lib/iomgr/error.h" |
||||
#include "src/core/lib/iomgr/exec_ctx.h" |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/inter_activity_latch.h" |
||||
#include "src/core/lib/promise/wait_for_callback.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/util/notification.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
#include "src/core/util/sync.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
class ChaoticGoodConnector final : public SubchannelConnector { |
||||
public: |
||||
void Connect(const Args& args, Result* result, grpc_closure* notify) override; |
||||
void Shutdown(grpc_error_handle) override { |
||||
ActivityPtr connect_activity; |
||||
MutexLock lock(&mu_); |
||||
is_shutdown_ = true; |
||||
connect_activity = std::move(connect_activity_); |
||||
}; |
||||
|
||||
private: |
||||
class ConnectionCreator final : public ClientConnectionFactory { |
||||
public: |
||||
ConnectionCreator( |
||||
grpc_event_engine::experimental::EventEngine::ResolvedAddress address, |
||||
const ChannelArgs& args) |
||||
: address_(address), args_(args) {} |
||||
PendingConnection Connect(absl::string_view id) override; |
||||
void Orphaned() override {}; |
||||
|
||||
private: |
||||
grpc_event_engine::experimental::EventEngine::ResolvedAddress address_; |
||||
ChannelArgs args_; |
||||
}; |
||||
|
||||
struct ResultNotifier { |
||||
ResultNotifier(const Args& args, Result* result, grpc_closure* notify) |
||||
: args(args), |
||||
config(args.channel_args), |
||||
result(result), |
||||
notify(notify) {} |
||||
|
||||
Args args; |
||||
Config config; |
||||
Result* result; |
||||
grpc_closure* notify; |
||||
|
||||
void Run(absl::Status status, DebugLocation location = {}) { |
||||
ExecCtx::Run(location, std::exchange(notify, nullptr), status); |
||||
} |
||||
}; |
||||
|
||||
Mutex mu_; |
||||
bool is_shutdown_ ABSL_GUARDED_BY(mu_) = false; |
||||
ActivityPtr connect_activity_ ABSL_GUARDED_BY(mu_); |
||||
}; |
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
grpc_channel* grpc_chaotic_good_legacy_channel_create( |
||||
const char* target, const grpc_channel_args* args); |
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CLIENT_CHAOTIC_GOOD_CONNECTOR_H
|
@ -0,0 +1,390 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/client_transport.h" |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/slice.h> |
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstdint> |
||||
#include <cstdlib> |
||||
#include <memory> |
||||
#include <string> |
||||
#include <tuple> |
||||
#include <utility> |
||||
|
||||
#include "absl/log/check.h" |
||||
#include "absl/log/log.h" |
||||
#include "absl/random/bit_gen_ref.h" |
||||
#include "absl/random/random.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/chaotic_good_transport.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/event_engine/query_extensions.h" |
||||
#include "src/core/lib/iomgr/exec_ctx.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/map.h" |
||||
#include "src/core/lib/promise/switch.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
void ChaoticGoodClientTransport::Orphan() { |
||||
AbortWithError(); |
||||
RefCountedPtr<Party> party; |
||||
{ |
||||
MutexLock lock(&mu_); |
||||
party = std::move(party_); |
||||
} |
||||
party.reset(); |
||||
Unref(); |
||||
} |
||||
|
||||
RefCountedPtr<ChaoticGoodClientTransport::Stream> |
||||
ChaoticGoodClientTransport::LookupStream(uint32_t stream_id) { |
||||
MutexLock lock(&mu_); |
||||
auto it = stream_map_.find(stream_id); |
||||
if (it == stream_map_.end()) { |
||||
return nullptr; |
||||
} |
||||
return it->second; |
||||
} |
||||
|
||||
auto ChaoticGoodClientTransport::PushFrameIntoCall( |
||||
ServerInitialMetadataFrame frame, RefCountedPtr<Stream> stream) { |
||||
DCHECK(stream->message_reassembly.in_message_boundary()); |
||||
auto headers = ServerMetadataGrpcFromProto(frame.body); |
||||
if (!headers.ok()) { |
||||
LOG_EVERY_N_SEC(INFO, 10) << "Encode headers failed: " << headers.status(); |
||||
return Immediate(StatusFlag(Failure{})); |
||||
} |
||||
return Immediate(stream->call.PushServerInitialMetadata(std::move(*headers))); |
||||
} |
||||
|
||||
auto ChaoticGoodClientTransport::PushFrameIntoCall( |
||||
MessageFrame frame, RefCountedPtr<Stream> stream) { |
||||
return stream->message_reassembly.PushFrameInto(std::move(frame), |
||||
stream->call); |
||||
} |
||||
|
||||
auto ChaoticGoodClientTransport::PushFrameIntoCall( |
||||
BeginMessageFrame frame, RefCountedPtr<Stream> stream) { |
||||
return stream->message_reassembly.PushFrameInto(std::move(frame), |
||||
stream->call); |
||||
} |
||||
|
||||
auto ChaoticGoodClientTransport::PushFrameIntoCall( |
||||
MessageChunkFrame frame, RefCountedPtr<Stream> stream) { |
||||
return stream->message_reassembly.PushFrameInto(std::move(frame), |
||||
stream->call); |
||||
} |
||||
|
||||
auto ChaoticGoodClientTransport::PushFrameIntoCall( |
||||
ServerTrailingMetadataFrame frame, RefCountedPtr<Stream> stream) { |
||||
auto trailers = ServerMetadataGrpcFromProto(frame.body); |
||||
if (!trailers.ok()) { |
||||
stream->call.PushServerTrailingMetadata( |
||||
CancelledServerMetadataFromStatus(trailers.status())); |
||||
} else if (!stream->message_reassembly.in_message_boundary() && |
||||
(*trailers) |
||||
->get(GrpcStatusMetadata()) |
||||
.value_or(GRPC_STATUS_UNKNOWN) == GRPC_STATUS_OK) { |
||||
stream->call.PushServerTrailingMetadata(CancelledServerMetadataFromStatus( |
||||
GRPC_STATUS_INTERNAL, |
||||
"End of call received while still receiving last message - this is a " |
||||
"protocol error")); |
||||
} else { |
||||
stream->call.PushServerTrailingMetadata(std::move(*trailers)); |
||||
} |
||||
return Immediate(Success{}); |
||||
} |
||||
|
||||
template <typename T> |
||||
auto ChaoticGoodClientTransport::DispatchFrame( |
||||
RefCountedPtr<ChaoticGoodTransport> transport, |
||||
IncomingFrame incoming_frame) { |
||||
auto stream = LookupStream(incoming_frame.header().stream_id); |
||||
return GRPC_LATENT_SEE_PROMISE( |
||||
"ChaoticGoodClientTransport::DispatchFrame", |
||||
If( |
||||
stream != nullptr, |
||||
[this, &stream, &incoming_frame, &transport]() { |
||||
auto& call = stream->call; |
||||
// TODO(ctiller): instead of SpawnWaitable here we probably want a
|
||||
// small queue to push into, so that the call can proceed
|
||||
// asynchronously to other calls regardless of frame ordering.
|
||||
return call.SpawnWaitable( |
||||
"push-frame", [this, stream = std::move(stream), |
||||
incoming_frame = std::move(incoming_frame), |
||||
transport = std::move(transport)]() mutable { |
||||
return TrySeq( |
||||
incoming_frame.Payload(), |
||||
[transport = std::move(transport), |
||||
header = incoming_frame.header()](SliceBuffer payload) { |
||||
return transport->DeserializeFrame<T>( |
||||
header, std::move(payload)); |
||||
}, |
||||
[stream = std::move(stream), this](T frame) mutable { |
||||
auto& call = stream->call; |
||||
return Map(call.CancelIfFails(PushFrameIntoCall( |
||||
std::move(frame), std::move(stream))), |
||||
[](auto) { return absl::OkStatus(); }); |
||||
}); |
||||
}); |
||||
}, |
||||
[]() { return absl::OkStatus(); })); |
||||
} |
||||
|
||||
auto ChaoticGoodClientTransport::TransportReadLoop( |
||||
RefCountedPtr<ChaoticGoodTransport> transport) { |
||||
return Loop([this, transport = std::move(transport)] { |
||||
return TrySeq( |
||||
transport->ReadFrameBytes(), |
||||
[this, transport](IncomingFrame incoming_frame) mutable { |
||||
return Switch(incoming_frame.header().type, |
||||
Case<FrameType::kServerInitialMetadata>([&, this]() { |
||||
return DispatchFrame<ServerInitialMetadataFrame>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kServerTrailingMetadata>([&, this]() { |
||||
return DispatchFrame<ServerTrailingMetadataFrame>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kMessage>([&, this]() { |
||||
return DispatchFrame<MessageFrame>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kBeginMessage>([&, this]() { |
||||
return DispatchFrame<BeginMessageFrame>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kMessageChunk>([&, this]() { |
||||
return DispatchFrame<MessageChunkFrame>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Default([&]() { |
||||
LOG_EVERY_N_SEC(INFO, 10) |
||||
<< "Bad frame type: " |
||||
<< incoming_frame.header().ToString(); |
||||
return absl::OkStatus(); |
||||
})); |
||||
}, |
||||
[]() -> LoopCtl<absl::Status> { return Continue{}; }); |
||||
}); |
||||
} |
||||
|
||||
auto ChaoticGoodClientTransport::OnTransportActivityDone( |
||||
absl::string_view what) { |
||||
return [self = RefAsSubclass<ChaoticGoodClientTransport>(), |
||||
what](absl::Status status) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Client transport " << self.get() << " closed (via " |
||||
<< what << "): " << status; |
||||
self->AbortWithError(); |
||||
}; |
||||
} |
||||
|
||||
ChaoticGoodClientTransport::ChaoticGoodClientTransport( |
||||
const ChannelArgs& args, PromiseEndpoint control_endpoint, Config config, |
||||
RefCountedPtr<ClientConnectionFactory>) |
||||
: allocator_(args.GetObject<ResourceQuota>() |
||||
->memory_quota() |
||||
->CreateMemoryAllocator("chaotic-good")), |
||||
outgoing_frames_(4), |
||||
message_chunker_(config.MakeMessageChunker()) { |
||||
auto event_engine = |
||||
args.GetObjectRef<grpc_event_engine::experimental::EventEngine>(); |
||||
auto transport = MakeRefCounted<ChaoticGoodTransport>( |
||||
std::move(control_endpoint), config.TakePendingDataEndpoints(), |
||||
event_engine, config.MakeTransportOptions(), config.tracing_enabled()); |
||||
auto party_arena = SimpleArenaAllocator(0)->MakeArena(); |
||||
party_arena->SetContext<grpc_event_engine::experimental::EventEngine>( |
||||
event_engine.get()); |
||||
party_ = Party::Make(std::move(party_arena)); |
||||
party_->Spawn( |
||||
"client-chaotic-writer", |
||||
GRPC_LATENT_SEE_PROMISE("ClientTransportWriteLoop", |
||||
transport->TransportWriteLoop(outgoing_frames_)), |
||||
OnTransportActivityDone("write_loop")); |
||||
party_->Spawn( |
||||
"client-chaotic-reader", |
||||
GRPC_LATENT_SEE_PROMISE("ClientTransportReadLoop", |
||||
TransportReadLoop(std::move(transport))), |
||||
OnTransportActivityDone("read_loop")); |
||||
} |
||||
|
||||
ChaoticGoodClientTransport::~ChaoticGoodClientTransport() { party_.reset(); } |
||||
|
||||
void ChaoticGoodClientTransport::AbortWithError() { |
||||
// Mark transport as unavailable when the endpoint write/read failed.
|
||||
// Close all the available pipes.
|
||||
outgoing_frames_.MarkClosed(); |
||||
ReleasableMutexLock lock(&mu_); |
||||
StreamMap stream_map = std::move(stream_map_); |
||||
stream_map_.clear(); |
||||
state_tracker_.SetState(GRPC_CHANNEL_SHUTDOWN, |
||||
absl::UnavailableError("transport closed"), |
||||
"transport closed"); |
||||
lock.Release(); |
||||
for (auto& pair : stream_map) { |
||||
auto stream = std::move(pair.second); |
||||
auto& call = stream->call; |
||||
call.SpawnInfallible("cancel", [stream = std::move(stream)]() mutable { |
||||
stream->call.PushServerTrailingMetadata(ServerMetadataFromStatus( |
||||
absl::UnavailableError("Transport closed."))); |
||||
}); |
||||
} |
||||
} |
||||
|
||||
uint32_t ChaoticGoodClientTransport::MakeStream(CallHandler call_handler) { |
||||
MutexLock lock(&mu_); |
||||
const uint32_t stream_id = next_stream_id_++; |
||||
const bool on_done_added = |
||||
call_handler.OnDone([self = RefAsSubclass<ChaoticGoodClientTransport>(), |
||||
stream_id](bool cancelled) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Client call " << self.get() << " id=" << stream_id |
||||
<< " done: cancelled=" << cancelled; |
||||
if (cancelled) { |
||||
self->outgoing_frames_.MakeSender().UnbufferedImmediateSend( |
||||
CancelFrame{stream_id}); |
||||
} |
||||
MutexLock lock(&self->mu_); |
||||
self->stream_map_.erase(stream_id); |
||||
}); |
||||
if (!on_done_added) return 0; |
||||
stream_map_.emplace(stream_id, |
||||
MakeRefCounted<Stream>(std::move(call_handler))); |
||||
return stream_id; |
||||
} |
||||
|
||||
namespace { |
||||
absl::Status BooleanSuccessToTransportError(bool success) { |
||||
return success ? absl::OkStatus() |
||||
: absl::UnavailableError("Transport closed."); |
||||
} |
||||
} // namespace
|
||||
|
||||
auto ChaoticGoodClientTransport::CallOutboundLoop(uint32_t stream_id, |
||||
CallHandler call_handler) { |
||||
auto send_fragment = [stream_id, |
||||
outgoing_frames = |
||||
outgoing_frames_.MakeSender()](auto frame) mutable { |
||||
frame.stream_id = stream_id; |
||||
return Map(outgoing_frames.Send(std::move(frame)), |
||||
BooleanSuccessToTransportError); |
||||
}; |
||||
auto send_message = |
||||
[stream_id, outgoing_frames = outgoing_frames_.MakeSender(), |
||||
message_chunker = message_chunker_](MessageHandle message) mutable { |
||||
return Map(message_chunker.Send(std::move(message), stream_id, |
||||
outgoing_frames), |
||||
BooleanSuccessToTransportError); |
||||
}; |
||||
return GRPC_LATENT_SEE_PROMISE( |
||||
"CallOutboundLoop", |
||||
TrySeq( |
||||
// Wait for initial metadata then send it out.
|
||||
call_handler.PullClientInitialMetadata(), |
||||
[send_fragment](ClientMetadataHandle md) mutable { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Sending initial metadata: " |
||||
<< md->DebugString(); |
||||
ClientInitialMetadataFrame frame; |
||||
frame.body = ClientMetadataProtoFromGrpc(*md); |
||||
return send_fragment(std::move(frame)); |
||||
}, |
||||
// Continuously send client frame with client to server messages.
|
||||
ForEach(MessagesFrom(call_handler), std::move(send_message)), |
||||
[send_fragment]() mutable { |
||||
ClientEndOfStream frame; |
||||
return send_fragment(std::move(frame)); |
||||
}, |
||||
[call_handler]() mutable { |
||||
return Map(call_handler.WasCancelled(), [](bool cancelled) { |
||||
if (cancelled) return absl::CancelledError(); |
||||
return absl::OkStatus(); |
||||
}); |
||||
})); |
||||
} |
||||
|
||||
void ChaoticGoodClientTransport::StartCall(CallHandler call_handler) { |
||||
// At this point, the connection is set up.
|
||||
// Start sending data frames.
|
||||
call_handler.SpawnGuarded( |
||||
"outbound_loop", [self = RefAsSubclass<ChaoticGoodClientTransport>(), |
||||
call_handler]() mutable { |
||||
const uint32_t stream_id = self->MakeStream(call_handler); |
||||
return If( |
||||
stream_id != 0, |
||||
[stream_id, call_handler = std::move(call_handler), |
||||
self = std::move(self)]() { |
||||
return Map( |
||||
self->CallOutboundLoop(stream_id, call_handler), |
||||
[stream_id, sender = self->outgoing_frames_.MakeSender()]( |
||||
absl::Status result) mutable { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Call " << stream_id |
||||
<< " finished with " << result.ToString(); |
||||
if (!result.ok()) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Send cancel"; |
||||
if (!sender.UnbufferedImmediateSend( |
||||
CancelFrame{stream_id})) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Send cancel failed"; |
||||
} |
||||
} |
||||
return result; |
||||
}); |
||||
}, |
||||
[]() { return absl::OkStatus(); }); |
||||
}); |
||||
} |
||||
|
||||
void ChaoticGoodClientTransport::PerformOp(grpc_transport_op* op) { |
||||
MutexLock lock(&mu_); |
||||
bool did_stuff = false; |
||||
if (op->start_connectivity_watch != nullptr) { |
||||
state_tracker_.AddWatcher(op->start_connectivity_watch_state, |
||||
std::move(op->start_connectivity_watch)); |
||||
did_stuff = true; |
||||
} |
||||
if (op->stop_connectivity_watch != nullptr) { |
||||
state_tracker_.RemoveWatcher(op->stop_connectivity_watch); |
||||
did_stuff = true; |
||||
} |
||||
if (op->set_accept_stream) { |
||||
Crash("set_accept_stream not supported on clients"); |
||||
} |
||||
if (!did_stuff) { |
||||
Crash(absl::StrCat("unimplemented transport perform op: ", |
||||
grpc_transport_op_string(op))); |
||||
} |
||||
ExecCtx::Run(DEBUG_LOCATION, op->on_consumed, absl::OkStatus()); |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,128 @@ |
||||
// Copyright 2022 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CLIENT_TRANSPORT_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CLIENT_TRANSPORT_H |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/event_engine/memory_allocator.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/support/port_platform.h> |
||||
#include <stdint.h> |
||||
#include <stdio.h> |
||||
|
||||
#include <cstdint> |
||||
#include <initializer_list> // IWYU pragma: keep |
||||
#include <map> |
||||
#include <memory> |
||||
#include <optional> |
||||
#include <tuple> |
||||
#include <type_traits> |
||||
#include <utility> |
||||
#include <variant> |
||||
|
||||
#include "absl/base/thread_annotations.h" |
||||
#include "absl/container/flat_hash_map.h" |
||||
#include "absl/random/random.h" |
||||
#include "absl/status/status.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/chaotic_good_transport.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/config.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/message_reassembly.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/pending_connection.h" |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/context.h" |
||||
#include "src/core/lib/promise/for_each.h" |
||||
#include "src/core/lib/promise/if.h" |
||||
#include "src/core/lib/promise/inter_activity_pipe.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/mpsc.h" |
||||
#include "src/core/lib/promise/pipe.h" |
||||
#include "src/core/lib/promise/poll.h" |
||||
#include "src/core/lib/promise/try_join.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/transport/metadata_batch.h" // IWYU pragma: keep |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/lib/transport/transport.h" |
||||
#include "src/core/util/sync.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
class ChaoticGoodClientTransport final : public ClientTransport { |
||||
public: |
||||
ChaoticGoodClientTransport(const ChannelArgs& args, |
||||
PromiseEndpoint control_endpoint, Config config, |
||||
RefCountedPtr<ClientConnectionFactory> connector); |
||||
~ChaoticGoodClientTransport() override; |
||||
|
||||
FilterStackTransport* filter_stack_transport() override { return nullptr; } |
||||
ClientTransport* client_transport() override { return this; } |
||||
ServerTransport* server_transport() override { return nullptr; } |
||||
absl::string_view GetTransportName() const override { return "chaotic_good"; } |
||||
void SetPollset(grpc_stream*, grpc_pollset*) override {} |
||||
void SetPollsetSet(grpc_stream*, grpc_pollset_set*) override {} |
||||
void PerformOp(grpc_transport_op*) override; |
||||
void Orphan() override; |
||||
|
||||
void StartCall(CallHandler call_handler) override; |
||||
void AbortWithError(); |
||||
|
||||
private: |
||||
struct Stream : public RefCounted<Stream> { |
||||
explicit Stream(CallHandler call) : call(std::move(call)) {} |
||||
CallHandler call; |
||||
MessageReassembly message_reassembly; |
||||
}; |
||||
using StreamMap = absl::flat_hash_map<uint32_t, RefCountedPtr<Stream>>; |
||||
|
||||
uint32_t MakeStream(CallHandler call_handler); |
||||
RefCountedPtr<Stream> LookupStream(uint32_t stream_id); |
||||
auto CallOutboundLoop(uint32_t stream_id, CallHandler call_handler); |
||||
auto OnTransportActivityDone(absl::string_view what); |
||||
template <typename T> |
||||
auto DispatchFrame(RefCountedPtr<ChaoticGoodTransport> transport, |
||||
IncomingFrame incoming_frame); |
||||
auto TransportReadLoop(RefCountedPtr<ChaoticGoodTransport> transport); |
||||
// Push one frame into a call
|
||||
auto PushFrameIntoCall(ServerInitialMetadataFrame frame, |
||||
RefCountedPtr<Stream> stream); |
||||
auto PushFrameIntoCall(MessageFrame frame, RefCountedPtr<Stream> stream); |
||||
auto PushFrameIntoCall(ServerTrailingMetadataFrame frame, |
||||
RefCountedPtr<Stream> stream); |
||||
auto PushFrameIntoCall(BeginMessageFrame frame, RefCountedPtr<Stream> stream); |
||||
auto PushFrameIntoCall(MessageChunkFrame frame, RefCountedPtr<Stream> stream); |
||||
|
||||
grpc_event_engine::experimental::MemoryAllocator allocator_; |
||||
// Max buffer is set to 4, so that for stream writes each time it will queue
|
||||
// at most 2 frames.
|
||||
MpscReceiver<ClientFrame> outgoing_frames_; |
||||
Mutex mu_; |
||||
uint32_t next_stream_id_ ABSL_GUARDED_BY(mu_) = 1; |
||||
// Map of stream incoming server frames, key is stream_id.
|
||||
StreamMap stream_map_ ABSL_GUARDED_BY(mu_); |
||||
RefCountedPtr<Party> party_; |
||||
ConnectivityStateTracker state_tracker_ ABSL_GUARDED_BY(mu_){ |
||||
"chaotic_good_client", GRPC_CHANNEL_READY}; |
||||
MessageChunker message_chunker_; |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CLIENT_TRANSPORT_H
|
@ -0,0 +1,229 @@ |
||||
// 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CONFIG_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CONFIG_H |
||||
|
||||
#include <vector> |
||||
|
||||
#include "absl/container/flat_hash_set.h" |
||||
#include "src/core/ext/transport/chaotic_good/chaotic_good_frame.pb.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/chaotic_good_transport.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/message_chunker.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/pending_connection.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/event_engine/extensions/tcp_trace.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
#define GRPC_ARG_CHAOTIC_GOOD_ALIGNMENT "grpc.chaotic_good.alignment" |
||||
#define GRPC_ARG_CHAOTIC_GOOD_MAX_RECV_CHUNK_SIZE \ |
||||
"grpc.chaotic_good.max_recv_chunk_size" |
||||
#define GRPC_ARG_CHAOTIC_GOOD_MAX_SEND_CHUNK_SIZE \ |
||||
"grpc.chaotic_good.max_send_chunk_size" |
||||
#define GRPC_ARG_CHAOTIC_GOOD_INLINED_PAYLOAD_SIZE_THRESHOLD \ |
||||
"grpc.chaotic_good.inlined_payload_size_threshold" |
||||
|
||||
// Transport configuration.
|
||||
// Most of our configuration is derived from channel args, and then exchanged
|
||||
// via settings frames to define a final shared configuration between client and
|
||||
// server.
|
||||
class Config { |
||||
public: |
||||
explicit Config( |
||||
const ChannelArgs& channel_args, |
||||
std::initializer_list<chaotic_good_frame::Settings::Features> |
||||
supported_features = {chaotic_good_frame::Settings::CHUNKING}) |
||||
: supported_features_(supported_features) { |
||||
decode_alignment_ = |
||||
std::max(1, channel_args.GetInt(GRPC_ARG_CHAOTIC_GOOD_ALIGNMENT) |
||||
.value_or(decode_alignment_)); |
||||
max_recv_chunk_size_ = std::max( |
||||
0, channel_args.GetInt(GRPC_ARG_CHAOTIC_GOOD_MAX_RECV_CHUNK_SIZE) |
||||
.value_or(max_recv_chunk_size_)); |
||||
max_send_chunk_size_ = std::max( |
||||
0, channel_args.GetInt(GRPC_ARG_CHAOTIC_GOOD_MAX_SEND_CHUNK_SIZE) |
||||
.value_or(max_send_chunk_size_)); |
||||
if (max_recv_chunk_size_ == 0 || max_send_chunk_size_ == 0) { |
||||
max_recv_chunk_size_ = 0; |
||||
max_send_chunk_size_ = 0; |
||||
} |
||||
inline_payload_size_threshold_ = std::max( |
||||
0, channel_args |
||||
.GetInt(GRPC_ARG_CHAOTIC_GOOD_INLINED_PAYLOAD_SIZE_THRESHOLD) |
||||
.value_or(inline_payload_size_threshold_)); |
||||
tracing_enabled_ = |
||||
channel_args.GetBool(GRPC_ARG_TCP_TRACING_ENABLED).value_or(false); |
||||
} |
||||
|
||||
Config(const Config&) = delete; |
||||
Config& operator=(const Config&) = delete; |
||||
Config(Config&&) = default; |
||||
Config& operator=(Config&&) = default; |
||||
|
||||
~Config() = default; |
||||
|
||||
void ServerAddPendingDataEndpoint(PendingConnection endpoint) { |
||||
pending_data_endpoints_.emplace_back(std::move(endpoint)); |
||||
} |
||||
|
||||
std::vector<PendingConnection> TakePendingDataEndpoints() { |
||||
return std::move(pending_data_endpoints_); |
||||
} |
||||
|
||||
void PrepareServerOutgoingSettings(chaotic_good_frame::Settings& settings) { |
||||
for (const auto& pending_data_endpoint : pending_data_endpoints_) { |
||||
settings.add_connection_id(pending_data_endpoint.id()); |
||||
} |
||||
PrepareOutgoingSettings(settings); |
||||
} |
||||
|
||||
void PrepareClientOutgoingSettings(chaotic_good_frame::Settings& settings) { |
||||
CHECK_EQ(pending_data_endpoints_.size(), 0u); |
||||
PrepareOutgoingSettings(settings); |
||||
} |
||||
|
||||
absl::Status ReceiveServerIncomingSettings( |
||||
const chaotic_good_frame::Settings& settings, |
||||
ClientConnectionFactory& connector) { |
||||
absl::flat_hash_set<chaotic_good_frame::Settings::Features> |
||||
supported_features; |
||||
for (const auto feature : settings.supported_features()) { |
||||
if (chaotic_good_frame::Settings::Features_IsValid(feature)) { |
||||
const auto valid_feature = |
||||
static_cast<chaotic_good_frame::Settings::Features>(feature); |
||||
if (supported_features_.contains(valid_feature)) { |
||||
supported_features.insert(valid_feature); |
||||
} |
||||
} |
||||
} |
||||
supported_features_.swap(supported_features); |
||||
for (const auto& connection_id : settings.connection_id()) { |
||||
pending_data_endpoints_.emplace_back(connector.Connect(connection_id)); |
||||
} |
||||
return ReceiveIncomingSettings(settings); |
||||
} |
||||
|
||||
absl::Status ReceiveClientIncomingSettings( |
||||
const chaotic_good_frame::Settings& settings) { |
||||
absl::flat_hash_set<chaotic_good_frame::Settings::Features> |
||||
supported_features; |
||||
for (const auto feature : settings.supported_features()) { |
||||
if (!chaotic_good_frame::Settings::Features_IsValid(feature)) { |
||||
return absl::InternalError(absl::StrCat( |
||||
"Unsupported feature present in chaotic-good handshake: ", |
||||
feature)); |
||||
} |
||||
const auto valid_feature = |
||||
static_cast<chaotic_good_frame::Settings::Features>(feature); |
||||
if (!supported_features_.contains(valid_feature)) { |
||||
return absl::InternalError(absl::StrCat( |
||||
"Unsupported feature present in chaotic-good handshake: ", |
||||
chaotic_good_frame::Settings::Features_Name(valid_feature))); |
||||
} |
||||
supported_features.insert(valid_feature); |
||||
} |
||||
supported_features_.swap(supported_features); |
||||
if (settings.connection_id_size() != 0) { |
||||
return absl::InternalError("Client cannot specify connection ids"); |
||||
} |
||||
return ReceiveIncomingSettings(settings); |
||||
} |
||||
|
||||
// Factory: make transport options from the settings derived here-in.
|
||||
ChaoticGoodTransport::Options MakeTransportOptions() const { |
||||
ChaoticGoodTransport::Options options; |
||||
options.encode_alignment = encode_alignment_; |
||||
options.decode_alignment = decode_alignment_; |
||||
options.inlined_payload_size_threshold = inline_payload_size_threshold_; |
||||
return options; |
||||
} |
||||
|
||||
// Factory: create a message chunker based on negotiated settings.
|
||||
MessageChunker MakeMessageChunker() const { |
||||
return MessageChunker(max_send_chunk_size_, encode_alignment_); |
||||
} |
||||
|
||||
bool tracing_enabled() const { return tracing_enabled_; } |
||||
|
||||
void TestOnlySetChunkSizes(uint32_t size) { |
||||
max_send_chunk_size_ = size; |
||||
max_recv_chunk_size_ = size; |
||||
} |
||||
|
||||
uint32_t encode_alignment() const { return encode_alignment_; } |
||||
uint32_t decode_alignment() const { return decode_alignment_; } |
||||
uint32_t max_send_chunk_size() const { return max_send_chunk_size_; } |
||||
// TODO(ctiller): use this to verify that chunk limits are being observed.
|
||||
uint32_t max_recv_chunk_size() const { return max_recv_chunk_size_; } |
||||
uint32_t inline_payload_size_threshold() const { |
||||
return inline_payload_size_threshold_; |
||||
} |
||||
|
||||
std::string ToString() const { |
||||
return absl::StrCat(GRPC_DUMP_ARGS(tracing_enabled_, encode_alignment_, |
||||
decode_alignment_, max_send_chunk_size_, |
||||
max_recv_chunk_size_, |
||||
inline_payload_size_threshold_)); |
||||
} |
||||
|
||||
template <typename Sink> |
||||
friend void AbslStringify(Sink& sink, const Config& config) { |
||||
sink.Append(config.ToString()); |
||||
} |
||||
|
||||
bool supports_chunking() const { |
||||
return supported_features_.contains(chaotic_good_frame::Settings::CHUNKING); |
||||
} |
||||
|
||||
private: |
||||
// Fill-in a settings frame to be sent with the results of the negotiation so
|
||||
// far. For the client this will be whatever we got from channel args; for the
|
||||
// server this is called *AFTER* ReceiveIncomingSettings and so contains the
|
||||
// result of mixing the server channel args with the client settings frame.
|
||||
void PrepareOutgoingSettings(chaotic_good_frame::Settings& settings) const { |
||||
settings.set_alignment(decode_alignment_); |
||||
settings.set_max_chunk_size(max_recv_chunk_size_); |
||||
} |
||||
|
||||
// Receive a settings frame from our peer and integrate its settings with our
|
||||
// own.
|
||||
absl::Status ReceiveIncomingSettings( |
||||
const chaotic_good_frame::Settings& settings) { |
||||
if (settings.alignment() != 0) encode_alignment_ = settings.alignment(); |
||||
max_send_chunk_size_ = |
||||
std::min(max_send_chunk_size_, settings.max_chunk_size()); |
||||
if (!supports_chunking() || settings.max_chunk_size() == 0) { |
||||
max_recv_chunk_size_ = 0; |
||||
max_send_chunk_size_ = 0; |
||||
} |
||||
return absl::OkStatus(); |
||||
} |
||||
|
||||
bool tracing_enabled_ = false; |
||||
uint32_t encode_alignment_ = 64; |
||||
uint32_t decode_alignment_ = 64; |
||||
uint32_t max_send_chunk_size_ = 1024 * 1024; |
||||
uint32_t max_recv_chunk_size_ = 1024 * 1024; |
||||
uint32_t inline_payload_size_threshold_ = 8 * 1024; |
||||
std::vector<PendingConnection> pending_data_endpoints_; |
||||
absl::flat_hash_set<chaotic_good_frame::Settings::Features> |
||||
supported_features_; |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CONFIG_H
|
@ -0,0 +1,73 @@ |
||||
// 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.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/control_endpoint.h" |
||||
|
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/event_engine/tcp_socket_utils.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
auto ControlEndpoint::Buffer::Pull() { |
||||
return [this]() -> Poll<SliceBuffer> { |
||||
Waker waker; |
||||
auto cleanup = absl::MakeCleanup([&waker]() { waker.Wakeup(); }); |
||||
MutexLock lock(&mu_); |
||||
if (queued_output_.Length() == 0) { |
||||
flush_waker_ = GetContext<Activity>()->MakeNonOwningWaker(); |
||||
return Pending{}; |
||||
} |
||||
waker = std::move(write_waker_); |
||||
return std::move(queued_output_); |
||||
}; |
||||
} |
||||
|
||||
ControlEndpoint::ControlEndpoint( |
||||
PromiseEndpoint endpoint, |
||||
grpc_event_engine::experimental::EventEngine* event_engine) |
||||
: endpoint_(std::make_shared<PromiseEndpoint>(std::move(endpoint))) { |
||||
auto arena = SimpleArenaAllocator(0)->MakeArena(); |
||||
arena->SetContext(event_engine); |
||||
write_party_ = Party::Make(arena); |
||||
CHECK(event_engine != nullptr); |
||||
write_party_->arena()->SetContext(event_engine); |
||||
write_party_->Spawn( |
||||
"flush-control", |
||||
GRPC_LATENT_SEE_PROMISE( |
||||
"FlushLoop", Loop([endpoint = endpoint_, buffer = buffer_]() { |
||||
return AddErrorPrefix( |
||||
"CONTROL_CHANNEL: ", |
||||
TrySeq( |
||||
// Pull one set of buffered writes
|
||||
buffer->Pull(), |
||||
// And write them
|
||||
[endpoint, buffer = buffer.get()](SliceBuffer flushing) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Flush " << flushing.Length() |
||||
<< " bytes from " << buffer << " to " |
||||
<< ResolvedAddressToString(endpoint->GetPeerAddress()) |
||||
.value_or("<<unknown peer address>>"); |
||||
return endpoint->Write(std::move(flushing)); |
||||
}, |
||||
// Then repeat
|
||||
[]() -> LoopCtl<absl::Status> { return Continue{}; })); |
||||
})), |
||||
[](absl::Status) {}); |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,102 @@ |
||||
// 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CONTROL_ENDPOINT_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CONTROL_ENDPOINT_H |
||||
|
||||
#include "absl/cleanup/cleanup.h" |
||||
#include "src/core/lib/promise/party.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/util/sync.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
// Wrapper around PromiseEndpoint.
|
||||
// Buffers all of the small writes that get enqueued to this endpoint, and then
|
||||
// uses a separate party to flush them to the wire.
|
||||
// In doing so we get to batch up effectively all the writes from the transport
|
||||
// (since party wakeups are sticky), and then flush all the writes in one go.
|
||||
class ControlEndpoint { |
||||
private: |
||||
class Buffer : public RefCounted<Buffer> { |
||||
public: |
||||
// Queue some buffer to be written.
|
||||
// We cap the queue size so that we don't infinitely buffer on one
|
||||
// connection - if the cap is hit, this queue operation will not resolve
|
||||
// until it empties.
|
||||
// Returns a promise that resolves to Empty{} when the data has been queued.
|
||||
auto Queue(SliceBuffer&& buffer) { |
||||
return [buffer = std::move(buffer), this]() mutable -> Poll<Empty> { |
||||
Waker waker; |
||||
auto cleanup = absl::MakeCleanup([&waker]() { waker.Wakeup(); }); |
||||
MutexLock lock(&mu_); |
||||
if (queued_output_.Length() != 0 && |
||||
queued_output_.Length() + buffer.Length() > MaxQueued()) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Delay control write" |
||||
<< " write_length=" << buffer.Length() |
||||
<< " already_buffered=" << queued_output_.Length() |
||||
<< " queue=" << this; |
||||
write_waker_ = GetContext<Activity>()->MakeNonOwningWaker(); |
||||
return Pending{}; |
||||
} |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Queue control write " << buffer.Length() |
||||
<< " bytes on " << this; |
||||
waker = std::move(flush_waker_); |
||||
queued_output_.Append(buffer); |
||||
return Empty{}; |
||||
}; |
||||
} |
||||
|
||||
auto Pull(); |
||||
|
||||
private: |
||||
size_t MaxQueued() const { return 1024 * 1024; } |
||||
|
||||
Mutex mu_; |
||||
Waker write_waker_ ABSL_GUARDED_BY(mu_); |
||||
Waker flush_waker_ ABSL_GUARDED_BY(mu_); |
||||
SliceBuffer queued_output_ ABSL_GUARDED_BY(mu_); |
||||
}; |
||||
|
||||
public: |
||||
ControlEndpoint(PromiseEndpoint endpoint, |
||||
grpc_event_engine::experimental::EventEngine* event_engine); |
||||
|
||||
// Write some data to the control endpoint; returns a promise that resolves
|
||||
// to Empty{} -- it's not possible to see errors from this api.
|
||||
auto Write(SliceBuffer&& bytes) { return buffer_->Queue(std::move(bytes)); } |
||||
|
||||
// Read operations are simply passthroughs to the underlying promise endpoint.
|
||||
auto ReadSlice(size_t length) { |
||||
return AddErrorPrefix("CONTROL_CHANNEL: ", endpoint_->ReadSlice(length)); |
||||
} |
||||
auto Read(size_t length) { |
||||
return AddErrorPrefix("CONTROL_CHANNEL: ", endpoint_->Read(length)); |
||||
} |
||||
auto GetPeerAddress() const { return endpoint_->GetPeerAddress(); } |
||||
auto GetLocalAddress() const { return endpoint_->GetLocalAddress(); } |
||||
|
||||
private: |
||||
std::shared_ptr<PromiseEndpoint> endpoint_; |
||||
RefCountedPtr<Party> write_party_; |
||||
RefCountedPtr<Buffer> buffer_ = MakeRefCounted<Buffer>(); |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_CONTROL_ENDPOINT_H
|
@ -0,0 +1,301 @@ |
||||
// 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.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/data_endpoints.h" |
||||
|
||||
#include <atomic> |
||||
#include <cstddef> |
||||
#include <cstdint> |
||||
|
||||
#include "absl/cleanup/cleanup.h" |
||||
#include "absl/strings/escaping.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/pending_connection.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/event_engine/extensions/tcp_trace.h" |
||||
#include "src/core/lib/event_engine/query_extensions.h" |
||||
#include "src/core/lib/event_engine/tcp_socket_utils.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/seq.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
namespace data_endpoints_detail { |
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
// OutputBuffer
|
||||
|
||||
bool OutputBuffer::Accept(SliceBuffer& buffer) { |
||||
if (pending_.Length() != 0 && |
||||
pending_.Length() + buffer.Length() > pending_max_) { |
||||
return false; |
||||
} |
||||
pending_.Append(buffer); |
||||
return true; |
||||
} |
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
// OutputBuffers
|
||||
|
||||
Poll<uint32_t> OutputBuffers::PollWrite(SliceBuffer& output_buffer) { |
||||
Waker waker; |
||||
auto cleanup = absl::MakeCleanup([&waker]() { waker.Wakeup(); }); |
||||
const auto length = output_buffer.Length(); |
||||
MutexLock lock(&mu_); |
||||
for (size_t i = 0; i < buffers_.size(); ++i) { |
||||
if (buffers_[i].has_value() && buffers_[i]->Accept(output_buffer)) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Queue " << length << " data onto endpoint " << i |
||||
<< " queue " << this; |
||||
waker = buffers_[i]->TakeWaker(); |
||||
return i; |
||||
} |
||||
} |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: No data endpoint ready for " << length |
||||
<< " bytes on queue " << this; |
||||
write_waker_ = GetContext<Activity>()->MakeNonOwningWaker(); |
||||
return Pending{}; |
||||
} |
||||
|
||||
Poll<SliceBuffer> OutputBuffers::PollNext(uint32_t connection_id) { |
||||
Waker waker; |
||||
auto cleanup = absl::MakeCleanup([&waker]() { waker.Wakeup(); }); |
||||
MutexLock lock(&mu_); |
||||
auto& buffer = buffers_[connection_id]; |
||||
CHECK(buffer.has_value()); |
||||
if (buffer->HavePending()) { |
||||
waker = std::move(write_waker_); |
||||
return buffer->TakePending(); |
||||
} |
||||
buffer->SetWaker(); |
||||
return Pending{}; |
||||
} |
||||
|
||||
void OutputBuffers::AddEndpoint(uint32_t connection_id) { |
||||
Waker waker; |
||||
auto cleanup = absl::MakeCleanup([&waker]() { waker.Wakeup(); }); |
||||
MutexLock lock(&mu_); |
||||
if (buffers_.size() < connection_id + 1) { |
||||
buffers_.resize(connection_id + 1); |
||||
} |
||||
CHECK(!buffers_[connection_id].has_value()) << GRPC_DUMP_ARGS(connection_id); |
||||
buffers_[connection_id].emplace(); |
||||
waker = std::move(write_waker_); |
||||
ready_endpoints_.fetch_add(1, std::memory_order_relaxed); |
||||
} |
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
// InputQueues
|
||||
|
||||
InputQueues::InputQueues() : read_requests_(), read_request_waker_() {} |
||||
|
||||
absl::StatusOr<uint64_t> InputQueues::CreateTicket(uint32_t connection_id, |
||||
size_t length) { |
||||
Waker waker; |
||||
auto cleanup = absl::MakeCleanup([&waker]() { waker.Wakeup(); }); |
||||
MutexLock lock(&mu_); |
||||
if (connection_id >= read_requests_.size()) { |
||||
return absl::UnavailableError( |
||||
absl::StrCat("Invalid connection id: ", connection_id)); |
||||
} |
||||
uint64_t ticket = next_ticket_id_; |
||||
++next_ticket_id_; |
||||
auto r = ReadRequest{length, ticket}; |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: New read ticket on #" << connection_id << " " << r; |
||||
read_requests_[connection_id].push_back(r); |
||||
outstanding_reads_.emplace(ticket, Waker{}); |
||||
waker = std::move(read_request_waker_[connection_id]); |
||||
return ticket; |
||||
} |
||||
|
||||
Poll<absl::StatusOr<SliceBuffer>> InputQueues::PollRead(uint64_t ticket) { |
||||
MutexLock lock(&mu_); |
||||
auto it = outstanding_reads_.find(ticket); |
||||
CHECK(it != outstanding_reads_.end()) << " ticket=" << ticket; |
||||
if (auto* waker = std::get_if<Waker>(&it->second)) { |
||||
*waker = GetContext<Activity>()->MakeNonOwningWaker(); |
||||
return Pending{}; |
||||
} |
||||
auto result = std::move(std::get<absl::StatusOr<SliceBuffer>>(it->second)); |
||||
outstanding_reads_.erase(it); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Poll for ticket #" << ticket |
||||
<< " completes: " << result.status(); |
||||
return result; |
||||
} |
||||
|
||||
Poll<std::vector<InputQueues::ReadRequest>> InputQueues::PollNext( |
||||
uint32_t connection_id) { |
||||
MutexLock lock(&mu_); |
||||
auto& q = read_requests_[connection_id]; |
||||
if (q.empty()) { |
||||
read_request_waker_[connection_id] = |
||||
GetContext<Activity>()->MakeNonOwningWaker(); |
||||
return Pending{}; |
||||
} |
||||
auto r = std::move(q); |
||||
q.clear(); |
||||
return r; |
||||
} |
||||
|
||||
void InputQueues::CompleteRead(uint64_t ticket, |
||||
absl::StatusOr<SliceBuffer> buffer) { |
||||
Waker waker; |
||||
auto cleanup = absl::MakeCleanup([&waker]() { waker.Wakeup(); }); |
||||
MutexLock lock(&mu_); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Complete ticket #" << ticket << ": " << buffer.status(); |
||||
auto it = outstanding_reads_.find(ticket); |
||||
if (it == outstanding_reads_.end()) return; // cancelled
|
||||
waker = std::move(std::get<Waker>(it->second)); |
||||
it->second.emplace<absl::StatusOr<SliceBuffer>>(std::move(buffer)); |
||||
} |
||||
|
||||
void InputQueues::CancelTicket(uint64_t ticket) { |
||||
MutexLock lock(&mu_); |
||||
outstanding_reads_.erase(ticket); |
||||
} |
||||
|
||||
void InputQueues::AddEndpoint(uint32_t connection_id) { |
||||
MutexLock lock(&mu_); |
||||
CHECK_EQ(read_requests_.size(), read_request_waker_.size()); |
||||
if (read_requests_.size() <= connection_id) { |
||||
read_requests_.resize(connection_id + 1); |
||||
read_request_waker_.resize(connection_id + 1); |
||||
} |
||||
} |
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
// Endpoint
|
||||
|
||||
auto Endpoint::WriteLoop(uint32_t id, |
||||
RefCountedPtr<OutputBuffers> output_buffers, |
||||
std::shared_ptr<PromiseEndpoint> endpoint) { |
||||
output_buffers->AddEndpoint(id); |
||||
return Loop([id, endpoint = std::move(endpoint), |
||||
output_buffers = std::move(output_buffers)]() { |
||||
return TrySeq( |
||||
output_buffers->Next(id), |
||||
[endpoint, id](SliceBuffer buffer) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: Write " << buffer.Length() |
||||
<< "b to data endpoint #" << id; |
||||
return endpoint->Write(std::move(buffer)); |
||||
}, |
||||
[]() -> LoopCtl<absl::Status> { return Continue{}; }); |
||||
}); |
||||
} |
||||
|
||||
auto Endpoint::ReadLoop(uint32_t id, RefCountedPtr<InputQueues> input_queues, |
||||
std::shared_ptr<PromiseEndpoint> endpoint) { |
||||
return Loop([id, endpoint, input_queues = std::move(input_queues)]() { |
||||
return TrySeq( |
||||
input_queues->Next(id), |
||||
[endpoint, input_queues]( |
||||
std::vector<data_endpoints_detail::InputQueues::ReadRequest> |
||||
requests) { |
||||
return TrySeqContainer( |
||||
std::move(requests), Empty{}, |
||||
[endpoint, input_queues]( |
||||
data_endpoints_detail::InputQueues::ReadRequest read_request, |
||||
Empty) { |
||||
return Seq(endpoint->Read(read_request.length), |
||||
[ticket = read_request.ticket, |
||||
|
||||
input_queues](absl::StatusOr<SliceBuffer> buffer) { |
||||
input_queues->CompleteRead(ticket, |
||||
std::move(buffer)); |
||||
return Empty{}; |
||||
}); |
||||
}); |
||||
}, |
||||
[]() -> LoopCtl<absl::Status> { return Continue{}; }); |
||||
}); |
||||
} |
||||
|
||||
Endpoint::Endpoint(uint32_t id, RefCountedPtr<OutputBuffers> output_buffers, |
||||
RefCountedPtr<InputQueues> input_queues, |
||||
PendingConnection pending_connection, bool enable_tracing, |
||||
grpc_event_engine::experimental::EventEngine* event_engine) { |
||||
input_queues->AddEndpoint(id); |
||||
auto arena = SimpleArenaAllocator(0)->MakeArena(); |
||||
arena->SetContext(event_engine); |
||||
party_ = Party::Make(arena); |
||||
party_->Spawn( |
||||
"write", |
||||
[id, enable_tracing, output_buffers = std::move(output_buffers), |
||||
input_queues = std::move(input_queues), |
||||
pending_connection = std::move(pending_connection), |
||||
arena = std::move(arena)]() mutable { |
||||
return TrySeq( |
||||
pending_connection.Await(), |
||||
[id, enable_tracing, output_buffers = std::move(output_buffers), |
||||
input_queues = std::move(input_queues), |
||||
arena = std::move(arena)](PromiseEndpoint ep) mutable { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: data endpoint " << id << " to " |
||||
<< grpc_event_engine::experimental::ResolvedAddressToString( |
||||
ep.GetPeerAddress()) |
||||
.value_or("<<unknown peer address>>") |
||||
<< " ready"; |
||||
auto endpoint = std::make_shared<PromiseEndpoint>(std::move(ep)); |
||||
// Enable RxMemoryAlignment and RPC receive coalescing after the
|
||||
// transport setup is complete. At this point all the settings
|
||||
// frames should have been read.
|
||||
endpoint->EnforceRxMemoryAlignmentAndCoalescing(); |
||||
if (enable_tracing) { |
||||
auto* epte = grpc_event_engine::experimental::QueryExtension< |
||||
grpc_event_engine::experimental::TcpTraceExtension>( |
||||
endpoint->GetEventEngineEndpoint().get()); |
||||
if (epte != nullptr) epte->InitializeAndReturnTcpTracer(); |
||||
} |
||||
auto read_party = Party::Make(std::move(arena)); |
||||
read_party->Spawn( |
||||
"read", |
||||
[id, input_queues = std::move(input_queues), endpoint]() { |
||||
return ReadLoop(id, input_queues, endpoint); |
||||
}, |
||||
[](absl::Status) {}); |
||||
return Map( |
||||
WriteLoop(id, std::move(output_buffers), std::move(endpoint)), |
||||
[read_party](auto x) { return x; }); |
||||
}); |
||||
}, |
||||
[](absl::Status) {}); |
||||
} |
||||
|
||||
} // namespace data_endpoints_detail
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
// DataEndpoints
|
||||
|
||||
DataEndpoints::DataEndpoints( |
||||
std::vector<PendingConnection> endpoints_vec, |
||||
grpc_event_engine::experimental::EventEngine* event_engine, |
||||
bool enable_tracing) |
||||
: output_buffers_(MakeRefCounted<data_endpoints_detail::OutputBuffers>()), |
||||
input_queues_(MakeRefCounted<data_endpoints_detail::InputQueues>()) { |
||||
CHECK(event_engine != nullptr); |
||||
for (size_t i = 0; i < endpoints_vec.size(); ++i) { |
||||
endpoints_.emplace_back(i, output_buffers_, input_queues_, |
||||
std::move(endpoints_vec[i]), enable_tracing, |
||||
event_engine); |
||||
} |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,225 @@ |
||||
// 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_DATA_ENDPOINTS_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_DATA_ENDPOINTS_H |
||||
|
||||
#include <atomic> |
||||
#include <cstdint> |
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/pending_connection.h" |
||||
#include "src/core/lib/promise/party.h" |
||||
#include "src/core/lib/promise/promise.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
namespace data_endpoints_detail { |
||||
|
||||
// Buffered writes for one data endpoint
|
||||
class OutputBuffer { |
||||
public: |
||||
bool Accept(SliceBuffer& buffer); |
||||
Waker TakeWaker() { return std::move(flush_waker_); } |
||||
void SetWaker() { |
||||
flush_waker_ = GetContext<Activity>()->MakeNonOwningWaker(); |
||||
} |
||||
bool HavePending() const { return pending_.Length() > 0; } |
||||
SliceBuffer TakePending() { return std::move(pending_); } |
||||
|
||||
private: |
||||
Waker flush_waker_; |
||||
size_t pending_max_ = 1024 * 1024; |
||||
SliceBuffer pending_; |
||||
}; |
||||
|
||||
// The set of output buffers for all connected data endpoints
|
||||
class OutputBuffers : public RefCounted<OutputBuffers> { |
||||
public: |
||||
auto Write(SliceBuffer output_buffer) { |
||||
return [output_buffer = std::move(output_buffer), this]() mutable { |
||||
return PollWrite(output_buffer); |
||||
}; |
||||
} |
||||
|
||||
auto Next(uint32_t connection_id) { |
||||
return [this, connection_id]() { return PollNext(connection_id); }; |
||||
} |
||||
|
||||
void AddEndpoint(uint32_t connection_id); |
||||
|
||||
uint32_t ReadyEndpoints() const { |
||||
return ready_endpoints_.load(std::memory_order_relaxed); |
||||
} |
||||
|
||||
private: |
||||
Poll<uint32_t> PollWrite(SliceBuffer& output_buffer); |
||||
Poll<SliceBuffer> PollNext(uint32_t connection_id); |
||||
|
||||
Mutex mu_; |
||||
std::vector<std::optional<OutputBuffer>> buffers_ ABSL_GUARDED_BY(mu_); |
||||
Waker write_waker_ ABSL_GUARDED_BY(mu_); |
||||
std::atomic<uint32_t> ready_endpoints_{0}; |
||||
}; |
||||
|
||||
class InputQueues : public RefCounted<InputQueues> { |
||||
public: |
||||
// One outstanding read.
|
||||
// ReadTickets get filed by read requests, and all tickets are fullfilled
|
||||
// by an endpoint.
|
||||
// A call may Await a ticket to get the bytes back later (or it may skip that
|
||||
// step - in which case the bytes are thrown away after reading).
|
||||
// This decoupling is necessary to ensure that cancelled reads by calls do not
|
||||
// cause data corruption for other calls.
|
||||
class ReadTicket { |
||||
public: |
||||
ReadTicket(absl::StatusOr<uint64_t> ticket, |
||||
RefCountedPtr<InputQueues> input_queues) |
||||
: ticket_(std::move(ticket)), input_queues_(std::move(input_queues)) {} |
||||
|
||||
ReadTicket(const ReadTicket&) = delete; |
||||
ReadTicket& operator=(const ReadTicket&) = delete; |
||||
ReadTicket(ReadTicket&& other) noexcept |
||||
: ticket_(std::move(other.ticket_)), |
||||
input_queues_(std::move(other.input_queues_)) {} |
||||
ReadTicket& operator=(ReadTicket&& other) noexcept { |
||||
ticket_ = std::move(other.ticket_); |
||||
input_queues_ = std::move(other.input_queues_); |
||||
return *this; |
||||
} |
||||
|
||||
~ReadTicket() { |
||||
if (input_queues_ != nullptr && ticket_.ok()) { |
||||
input_queues_->CancelTicket(*ticket_); |
||||
} |
||||
} |
||||
|
||||
auto Await() { |
||||
return If( |
||||
ticket_.ok(), |
||||
[&]() { |
||||
return |
||||
[ticket = *ticket_, input_queues = std::move(input_queues_)]() { |
||||
return input_queues->PollRead(ticket); |
||||
}; |
||||
}, |
||||
[&]() { |
||||
return Immediate(absl::StatusOr<SliceBuffer>(ticket_.status())); |
||||
}); |
||||
} |
||||
|
||||
private: |
||||
absl::StatusOr<uint64_t> ticket_; |
||||
RefCountedPtr<InputQueues> input_queues_; |
||||
}; |
||||
|
||||
struct ReadRequest { |
||||
size_t length; |
||||
uint64_t ticket; |
||||
|
||||
template <typename Sink> |
||||
friend void AbslStringify(Sink& sink, const ReadRequest& req) { |
||||
sink.Append(absl::StrCat("read#", req.ticket, ":", req.length, "b")); |
||||
} |
||||
}; |
||||
|
||||
explicit InputQueues(); |
||||
|
||||
ReadTicket Read(uint32_t connection_id, size_t length) { |
||||
return ReadTicket(CreateTicket(connection_id, length), Ref()); |
||||
} |
||||
|
||||
auto Next(uint32_t connection_id) { |
||||
return [this, connection_id]() { return PollNext(connection_id); }; |
||||
} |
||||
|
||||
void CompleteRead(uint64_t ticket, absl::StatusOr<SliceBuffer> buffer); |
||||
|
||||
void CancelTicket(uint64_t ticket); |
||||
|
||||
void AddEndpoint(uint32_t connection_id); |
||||
|
||||
private: |
||||
using ReadState = std::variant<absl::StatusOr<SliceBuffer>, Waker>; |
||||
|
||||
absl::StatusOr<uint64_t> CreateTicket(uint32_t connection_id, size_t length); |
||||
Poll<absl::StatusOr<SliceBuffer>> PollRead(uint64_t ticket); |
||||
Poll<std::vector<ReadRequest>> PollNext(uint32_t connection_id); |
||||
|
||||
Mutex mu_; |
||||
uint64_t next_ticket_id_ ABSL_GUARDED_BY(mu_) = 0; |
||||
std::vector<std::vector<ReadRequest>> read_requests_ ABSL_GUARDED_BY(mu_); |
||||
std::vector<Waker> read_request_waker_; |
||||
absl::flat_hash_map<uint64_t, ReadState> outstanding_reads_ |
||||
ABSL_GUARDED_BY(mu_); |
||||
}; |
||||
|
||||
class Endpoint final { |
||||
public: |
||||
Endpoint(uint32_t id, RefCountedPtr<OutputBuffers> output_buffers, |
||||
RefCountedPtr<InputQueues> input_queues, |
||||
PendingConnection pending_connection, bool enable_tracing, |
||||
grpc_event_engine::experimental::EventEngine* event_engine); |
||||
|
||||
private: |
||||
static auto WriteLoop(uint32_t id, |
||||
RefCountedPtr<OutputBuffers> output_buffers, |
||||
std::shared_ptr<PromiseEndpoint> endpoint); |
||||
static auto ReadLoop(uint32_t id, RefCountedPtr<InputQueues> input_queues, |
||||
std::shared_ptr<PromiseEndpoint> endpoint); |
||||
|
||||
RefCountedPtr<Party> party_; |
||||
}; |
||||
|
||||
} // namespace data_endpoints_detail
|
||||
|
||||
// Collection of data connections.
|
||||
class DataEndpoints { |
||||
public: |
||||
using ReadTicket = data_endpoints_detail::InputQueues::ReadTicket; |
||||
|
||||
explicit DataEndpoints( |
||||
std::vector<PendingConnection> endpoints, |
||||
grpc_event_engine::experimental::EventEngine* event_engine, |
||||
bool enable_tracing); |
||||
|
||||
// Try to queue output_buffer against a data endpoint.
|
||||
// Returns a promise that resolves to the data endpoint connection id
|
||||
// selected.
|
||||
// Connection ids returned by this class are 0 based (which is different
|
||||
// to how chaotic good communicates them on the wire - those are 1 based
|
||||
// to allow for the control channel identification)
|
||||
auto Write(SliceBuffer output_buffer) { |
||||
return output_buffers_->Write(std::move(output_buffer)); |
||||
} |
||||
|
||||
ReadTicket Read(uint32_t connection_id, uint32_t length) { |
||||
return input_queues_->Read(connection_id, length); |
||||
} |
||||
|
||||
bool empty() const { return output_buffers_->ReadyEndpoints() == 0; } |
||||
|
||||
private: |
||||
RefCountedPtr<data_endpoints_detail::OutputBuffers> output_buffers_; |
||||
RefCountedPtr<data_endpoints_detail::InputQueues> input_queues_; |
||||
Mutex mu_; |
||||
std::vector<data_endpoints_detail::Endpoint> endpoints_ ABSL_GUARDED_BY(mu_); |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_DATA_ENDPOINTS_H
|
@ -0,0 +1,291 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
|
||||
#include <grpc/slice.h> |
||||
#include <grpc/support/port_platform.h> |
||||
#include <string.h> |
||||
|
||||
#include <cstdint> |
||||
#include <limits> |
||||
#include <type_traits> |
||||
#include <utility> |
||||
|
||||
#include "absl/log/check.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/ext/transport/chaotic_good/chaotic_good_frame.pb.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/lib/promise/context.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/slice/slice.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/util/bitset.h" |
||||
#include "src/core/util/no_destruct.h" |
||||
#include "src/core/util/status_helper.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
absl::Status ReadProto(SliceBuffer payload, |
||||
google::protobuf::MessageLite& msg) { |
||||
auto payload_slice = payload.JoinIntoSlice(); |
||||
const bool ok = |
||||
msg.ParseFromArray(payload_slice.data(), payload_slice.length()); |
||||
return ok ? absl::OkStatus() : absl::InternalError("Protobuf parse error"); |
||||
} |
||||
|
||||
void WriteProto(const google::protobuf::MessageLite& msg, SliceBuffer& output) { |
||||
auto length = msg.ByteSizeLong(); |
||||
auto slice = MutableSlice::CreateUninitialized(length); |
||||
CHECK(msg.SerializeToArray(slice.data(), length)); |
||||
output.AppendIndexed(Slice(std::move(slice))); |
||||
} |
||||
|
||||
uint32_t ProtoPayloadSize(const google::protobuf::MessageLite& msg) { |
||||
auto length = msg.ByteSizeLong(); |
||||
CHECK_LE(length, std::numeric_limits<uint32_t>::max()); |
||||
return static_cast<uint32_t>(length); |
||||
} |
||||
|
||||
absl::Status ReadTransportProto(const FrameHeader& header, SliceBuffer payload, |
||||
google::protobuf::MessageLite& body) { |
||||
if (header.stream_id != 0) { |
||||
return absl::InternalError("Expected stream id 0"); |
||||
} |
||||
return ReadProto(std::move(payload), body); |
||||
} |
||||
|
||||
absl::Status ReadStreamProto(const FrameHeader& header, SliceBuffer payload, |
||||
google::protobuf::MessageLite& body, |
||||
uint32_t& stream_id) { |
||||
if (header.stream_id == 0) { |
||||
return absl::InternalError("Expected non-zero stream id"); |
||||
} |
||||
stream_id = header.stream_id; |
||||
return ReadProto(std::move(payload), body); |
||||
} |
||||
|
||||
absl::Status ReadEmptyFrame(const FrameHeader& header, uint32_t& stream_id) { |
||||
if (header.stream_id == 0) { |
||||
return absl::InternalError("Expected non-zero stream id"); |
||||
} |
||||
if (header.payload_length != 0) { |
||||
return absl::InternalError(absl::StrCat("Expected zero payload length on ", |
||||
FrameTypeString(header.type))); |
||||
} |
||||
stream_id = header.stream_id; |
||||
return absl::OkStatus(); |
||||
} |
||||
|
||||
namespace { |
||||
|
||||
struct ClientMetadataEncoder { |
||||
void Encode(HttpPathMetadata, |
||||
const typename HttpPathMetadata::ValueType& value) { |
||||
out.set_path(value.as_string_view()); |
||||
} |
||||
|
||||
void Encode(HttpAuthorityMetadata, |
||||
const typename HttpAuthorityMetadata::ValueType& value) { |
||||
out.set_authority(value.as_string_view()); |
||||
} |
||||
|
||||
void Encode(GrpcTimeoutMetadata, |
||||
const typename GrpcTimeoutMetadata::ValueType& value) { |
||||
auto now = Timestamp::Now(); |
||||
if (now > value) { |
||||
out.set_timeout_ms(0); |
||||
} else { |
||||
out.set_timeout_ms((value - now).millis()); |
||||
} |
||||
} |
||||
|
||||
template <typename Which> |
||||
void Encode(Which, const typename Which::ValueType& value) { |
||||
EncodeWithWarning(Slice::FromExternalString(Which::key()), |
||||
Slice(Which::Encode(value))); |
||||
} |
||||
|
||||
void EncodeWithWarning(const Slice& key, const Slice& value) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "encoding known key " << key.as_string_view() |
||||
<< " with unknown encoding"; |
||||
Encode(key, value); |
||||
} |
||||
|
||||
void Encode(const Slice& key, const Slice& value) { |
||||
auto* unk = out.add_unknown_metadata(); |
||||
unk->set_key(key.as_string_view()); |
||||
unk->set_value(value.as_string_view()); |
||||
} |
||||
|
||||
chaotic_good_frame::ClientMetadata out; |
||||
}; |
||||
|
||||
struct ServerMetadataEncoder { |
||||
void Encode(GrpcStatusMetadata, grpc_status_code code) { |
||||
out.set_status(code); |
||||
} |
||||
|
||||
void Encode(GrpcMessageMetadata, const Slice& value) { |
||||
out.set_message(value.as_string_view()); |
||||
} |
||||
|
||||
template <typename Which> |
||||
void Encode(Which, const typename Which::ValueType& value) { |
||||
EncodeWithWarning(Slice::FromExternalString(Which::key()), |
||||
Slice(Which::Encode(value))); |
||||
} |
||||
|
||||
void EncodeWithWarning(const Slice& key, const Slice& value) { |
||||
LOG_EVERY_N_SEC(INFO, 10) << "encoding known key " << key.as_string_view() |
||||
<< " with unknown encoding"; |
||||
Encode(key, value); |
||||
} |
||||
|
||||
void Encode(const Slice& key, const Slice& value) { |
||||
auto* unk = out.add_unknown_metadata(); |
||||
unk->set_key(key.as_string_view()); |
||||
unk->set_value(value.as_string_view()); |
||||
} |
||||
|
||||
chaotic_good_frame::ServerMetadata out; |
||||
}; |
||||
|
||||
template <typename T, typename M> |
||||
absl::StatusOr<T> ReadUnknownFields(const M& msg, T md) { |
||||
absl::Status error = absl::OkStatus(); |
||||
for (const auto& unk : msg.unknown_metadata()) { |
||||
md->Append(unk.key(), Slice::FromCopiedString(unk.value()), |
||||
[&error](absl::string_view error_msg, const Slice&) { |
||||
if (!error.ok()) return; |
||||
error = absl::InternalError(error_msg); |
||||
}); |
||||
} |
||||
if (!error.ok()) return error; |
||||
return std::move(md); |
||||
} |
||||
|
||||
} // namespace
|
||||
|
||||
chaotic_good_frame::ClientMetadata ClientMetadataProtoFromGrpc( |
||||
const ClientMetadata& md) { |
||||
ClientMetadataEncoder e; |
||||
md.Encode(&e); |
||||
return std::move(e.out); |
||||
} |
||||
|
||||
absl::StatusOr<ClientMetadataHandle> ClientMetadataGrpcFromProto( |
||||
chaotic_good_frame::ClientMetadata& metadata) { |
||||
auto md = Arena::MakePooled<ClientMetadata>(); |
||||
md->Set(GrpcStatusFromWire(), true); |
||||
if (metadata.has_path()) { |
||||
md->Set(HttpPathMetadata(), Slice::FromCopiedString(metadata.path())); |
||||
} |
||||
if (metadata.has_authority()) { |
||||
md->Set(HttpAuthorityMetadata(), |
||||
Slice::FromCopiedString(metadata.authority())); |
||||
} |
||||
if (metadata.has_timeout_ms()) { |
||||
md->Set(GrpcTimeoutMetadata(), |
||||
Timestamp::Now() + Duration::Milliseconds(metadata.timeout_ms())); |
||||
} |
||||
return ReadUnknownFields(metadata, std::move(md)); |
||||
} |
||||
|
||||
chaotic_good_frame::ServerMetadata ServerMetadataProtoFromGrpc( |
||||
const ServerMetadata& md) { |
||||
ServerMetadataEncoder e; |
||||
md.Encode(&e); |
||||
return std::move(e.out); |
||||
} |
||||
|
||||
absl::StatusOr<ServerMetadataHandle> ServerMetadataGrpcFromProto( |
||||
chaotic_good_frame::ServerMetadata& metadata) { |
||||
auto md = Arena::MakePooled<ServerMetadata>(); |
||||
md->Set(GrpcStatusFromWire(), true); |
||||
if (metadata.has_status()) { |
||||
md->Set(GrpcStatusMetadata(), |
||||
static_cast<grpc_status_code>(metadata.status())); |
||||
} |
||||
if (metadata.has_message()) { |
||||
md->Set(GrpcMessageMetadata(), Slice::FromCopiedString(metadata.message())); |
||||
} |
||||
return ReadUnknownFields(metadata, std::move(md)); |
||||
} |
||||
|
||||
absl::Status MessageFrame::Deserialize(const FrameHeader& header, |
||||
SliceBuffer payload) { |
||||
CHECK_EQ(header.type, FrameType::kMessage); |
||||
if (header.stream_id == 0) { |
||||
return absl::InternalError("Expected non-zero stream id"); |
||||
} |
||||
stream_id = header.stream_id; |
||||
message = Arena::MakePooled<Message>(std::move(payload), 0); |
||||
return absl::OkStatus(); |
||||
} |
||||
|
||||
FrameHeader MessageFrame::MakeHeader() const { |
||||
auto length = message->payload()->Length(); |
||||
CHECK_LE(length, std::numeric_limits<uint32_t>::max()); |
||||
return FrameHeader{FrameType::kMessage, 0, stream_id, |
||||
static_cast<uint32_t>(length)}; |
||||
} |
||||
|
||||
void MessageFrame::SerializePayload(SliceBuffer& payload) const { |
||||
CHECK_NE(stream_id, 0u); |
||||
payload.Append(*message->payload()); |
||||
} |
||||
|
||||
std::string MessageFrame::ToString() const { |
||||
std::string out = absl::StrCat("MessageFrame{stream_id=", stream_id); |
||||
if (message.get() != nullptr) { |
||||
absl::StrAppend(&out, ", message=", message->DebugString().c_str()); |
||||
} |
||||
absl::StrAppend(&out, "}"); |
||||
return out; |
||||
} |
||||
|
||||
absl::Status MessageChunkFrame::Deserialize(const FrameHeader& header, |
||||
SliceBuffer payload) { |
||||
CHECK_EQ(header.type, FrameType::kMessageChunk); |
||||
if (header.stream_id == 0) { |
||||
return absl::InternalError("Expected non-zero stream id"); |
||||
} |
||||
stream_id = header.stream_id; |
||||
this->payload = std::move(payload); |
||||
return absl::OkStatus(); |
||||
} |
||||
|
||||
FrameHeader MessageChunkFrame::MakeHeader() const { |
||||
auto length = payload.Length(); |
||||
CHECK_LE(length, std::numeric_limits<uint32_t>::max()); |
||||
return FrameHeader{FrameType::kMessageChunk, 0, stream_id, |
||||
static_cast<uint32_t>(length)}; |
||||
} |
||||
|
||||
void MessageChunkFrame::SerializePayload(SliceBuffer& payload) const { |
||||
CHECK_NE(stream_id, 0u); |
||||
payload.Append(this->payload); |
||||
} |
||||
|
||||
std::string MessageChunkFrame::ToString() const { |
||||
return absl::StrCat("MessageChunk{stream_id=", stream_id, |
||||
", payload=", payload.Length(), "b}"); |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,192 @@ |
||||
// Copyright 2022 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_FRAME_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_FRAME_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstdint> |
||||
#include <string> |
||||
#include <variant> |
||||
|
||||
#include "absl/random/bit_gen_ref.h" |
||||
#include "absl/status/status.h" |
||||
#include "src/core/ext/transport/chaotic_good/chaotic_good_frame.pb.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/transport/message.h" |
||||
#include "src/core/lib/transport/metadata.h" |
||||
#include "src/core/util/match.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
class FrameInterface { |
||||
public: |
||||
virtual absl::Status Deserialize(const FrameHeader& header, |
||||
SliceBuffer payload) = 0; |
||||
virtual FrameHeader MakeHeader() const = 0; |
||||
virtual void SerializePayload(SliceBuffer& payload) const = 0; |
||||
virtual std::string ToString() const = 0; |
||||
|
||||
template <typename Sink> |
||||
friend void AbslStringify(Sink& sink, const FrameInterface& frame) { |
||||
sink.Append(frame.ToString()); |
||||
} |
||||
|
||||
protected: |
||||
~FrameInterface() = default; |
||||
}; |
||||
|
||||
inline std::ostream& operator<<(std::ostream& os, const FrameInterface& frame) { |
||||
return os << frame.ToString(); |
||||
} |
||||
|
||||
chaotic_good_frame::ClientMetadata ClientMetadataProtoFromGrpc( |
||||
const ClientMetadata& md); |
||||
absl::StatusOr<ClientMetadataHandle> ClientMetadataGrpcFromProto( |
||||
chaotic_good_frame::ClientMetadata& metadata); |
||||
chaotic_good_frame::ServerMetadata ServerMetadataProtoFromGrpc( |
||||
const ServerMetadata& md); |
||||
absl::StatusOr<ServerMetadataHandle> ServerMetadataGrpcFromProto( |
||||
chaotic_good_frame::ServerMetadata& metadata); |
||||
|
||||
absl::Status ReadProto(SliceBuffer payload, google::protobuf::MessageLite& msg); |
||||
void WriteProto(const google::protobuf::MessageLite& msg, SliceBuffer& output); |
||||
uint32_t ProtoPayloadSize(const google::protobuf::MessageLite& msg); |
||||
absl::Status ReadTransportProto(const FrameHeader& header, SliceBuffer payload, |
||||
google::protobuf::MessageLite& body); |
||||
absl::Status ReadStreamProto(const FrameHeader& header, SliceBuffer payload, |
||||
google::protobuf::MessageLite& body, |
||||
uint32_t& stream_id); |
||||
absl::Status ReadEmptyFrame(const FrameHeader& header, uint32_t& stream_id); |
||||
|
||||
// Generic implementation of a transport-bound frame (stream_id is zero always)
|
||||
// with a protobuf specified body.
|
||||
|
||||
template <FrameType frame_type, typename Body> |
||||
struct ProtoTransportFrame final : public FrameInterface { |
||||
absl::Status Deserialize(const FrameHeader& header, |
||||
SliceBuffer payload) override { |
||||
DCHECK_EQ(header.type, frame_type); |
||||
return ReadTransportProto(header, std::move(payload), body); |
||||
} |
||||
FrameHeader MakeHeader() const override { |
||||
return FrameHeader{frame_type, 0, 0, ProtoPayloadSize(body)}; |
||||
} |
||||
void SerializePayload(SliceBuffer& payload) const override { |
||||
WriteProto(body, payload); |
||||
} |
||||
std::string ToString() const override { |
||||
return absl::StrCat(FrameTypeString(frame_type), "{", |
||||
body.ShortDebugString(), "}"); |
||||
} |
||||
|
||||
Body body; |
||||
}; |
||||
|
||||
// Generic implementation of a stream-bound frame with a protobuf specified
|
||||
// body.
|
||||
template <FrameType frame_type, typename Body> |
||||
struct ProtoStreamFrame final : public FrameInterface { |
||||
absl::Status Deserialize(const FrameHeader& header, |
||||
SliceBuffer payload) override { |
||||
DCHECK_EQ(header.type, frame_type); |
||||
return ReadStreamProto(header, std::move(payload), body, stream_id); |
||||
} |
||||
FrameHeader MakeHeader() const override { |
||||
return FrameHeader{frame_type, 0, stream_id, ProtoPayloadSize(body)}; |
||||
} |
||||
void SerializePayload(SliceBuffer& payload) const override { |
||||
DCHECK_NE(stream_id, 0u); |
||||
WriteProto(body, payload); |
||||
} |
||||
std::string ToString() const override { |
||||
return absl::StrCat(FrameTypeString(frame_type), "{@", stream_id, "; ", |
||||
body.ShortDebugString(), "}"); |
||||
} |
||||
|
||||
Body body; |
||||
uint32_t stream_id; |
||||
}; |
||||
|
||||
// Generic implementation of an empty stream frame.
|
||||
template <FrameType frame_type> |
||||
struct EmptyStreamFrame final : public FrameInterface { |
||||
EmptyStreamFrame() = default; |
||||
explicit EmptyStreamFrame(uint32_t stream_id) : stream_id(stream_id) {} |
||||
absl::Status Deserialize(const FrameHeader& header, SliceBuffer) override { |
||||
DCHECK_EQ(header.type, frame_type); |
||||
return ReadEmptyFrame(header, stream_id); |
||||
} |
||||
FrameHeader MakeHeader() const override { |
||||
return FrameHeader{frame_type, 0, stream_id, 0}; |
||||
} |
||||
void SerializePayload(SliceBuffer&) const override {} |
||||
std::string ToString() const override { return FrameTypeString(frame_type); } |
||||
|
||||
uint32_t stream_id; |
||||
}; |
||||
|
||||
using SettingsFrame = |
||||
ProtoTransportFrame<FrameType::kSettings, chaotic_good_frame::Settings>; |
||||
using ClientInitialMetadataFrame = |
||||
ProtoStreamFrame<FrameType::kClientInitialMetadata, |
||||
chaotic_good_frame::ClientMetadata>; |
||||
using BeginMessageFrame = ProtoStreamFrame<FrameType::kBeginMessage, |
||||
chaotic_good_frame::BeginMessage>; |
||||
using ClientEndOfStream = EmptyStreamFrame<FrameType::kClientEndOfStream>; |
||||
using ServerInitialMetadataFrame = |
||||
ProtoStreamFrame<FrameType::kServerInitialMetadata, |
||||
chaotic_good_frame::ServerMetadata>; |
||||
using ServerTrailingMetadataFrame = |
||||
ProtoStreamFrame<FrameType::kServerTrailingMetadata, |
||||
chaotic_good_frame::ServerMetadata>; |
||||
using CancelFrame = EmptyStreamFrame<FrameType::kCancel>; |
||||
|
||||
struct MessageFrame final : public FrameInterface { |
||||
absl::Status Deserialize(const FrameHeader& header, |
||||
SliceBuffer payload) override; |
||||
FrameHeader MakeHeader() const override; |
||||
void SerializePayload(SliceBuffer& payload) const override; |
||||
std::string ToString() const override; |
||||
|
||||
uint32_t stream_id; |
||||
MessageHandle message; |
||||
}; |
||||
|
||||
struct MessageChunkFrame final : public FrameInterface { |
||||
absl::Status Deserialize(const FrameHeader& header, |
||||
SliceBuffer payload) override; |
||||
FrameHeader MakeHeader() const override; |
||||
void SerializePayload(SliceBuffer& payload) const override; |
||||
std::string ToString() const override; |
||||
|
||||
uint32_t stream_id; |
||||
SliceBuffer payload; |
||||
}; |
||||
|
||||
using ClientFrame = |
||||
std::variant<ClientInitialMetadataFrame, MessageFrame, BeginMessageFrame, |
||||
MessageChunkFrame, ClientEndOfStream, CancelFrame>; |
||||
using ServerFrame = |
||||
std::variant<ServerInitialMetadataFrame, MessageFrame, BeginMessageFrame, |
||||
MessageChunkFrame, ServerTrailingMetadataFrame>; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_FRAME_H
|
@ -0,0 +1,97 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstdint> |
||||
|
||||
#include "absl/status/status.h" |
||||
#include "absl/strings/str_cat.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
namespace { |
||||
void WriteLittleEndianUint32(uint32_t value, uint8_t* data) { |
||||
data[0] = static_cast<uint8_t>(value); |
||||
data[1] = static_cast<uint8_t>(value >> 8); |
||||
data[2] = static_cast<uint8_t>(value >> 16); |
||||
data[3] = static_cast<uint8_t>(value >> 24); |
||||
} |
||||
|
||||
uint32_t ReadLittleEndianUint32(const uint8_t* data) { |
||||
return static_cast<uint32_t>(data[0]) | |
||||
(static_cast<uint32_t>(data[1]) << 8) | |
||||
(static_cast<uint32_t>(data[2]) << 16) | |
||||
(static_cast<uint32_t>(data[3]) << 24); |
||||
} |
||||
} // namespace
|
||||
|
||||
// Serializes a frame header into a buffer of 24 bytes.
|
||||
void FrameHeader::Serialize(uint8_t* data) const { |
||||
WriteLittleEndianUint32((static_cast<uint32_t>(type) << 16) | |
||||
static_cast<uint32_t>(payload_connection_id), |
||||
data); |
||||
WriteLittleEndianUint32(stream_id, data + 4); |
||||
WriteLittleEndianUint32(payload_length, data + 8); |
||||
} |
||||
|
||||
// Parses a frame header from a buffer of 24 bytes. All 24 bytes are consumed.
|
||||
absl::StatusOr<FrameHeader> FrameHeader::Parse(const uint8_t* data) { |
||||
FrameHeader header; |
||||
const uint32_t type_and_conn_id = ReadLittleEndianUint32(data); |
||||
if (type_and_conn_id & 0xff000000u) { |
||||
return absl::InternalError("Non-zero reserved byte received"); |
||||
} |
||||
header.type = static_cast<FrameType>(type_and_conn_id >> 16); |
||||
header.payload_connection_id = type_and_conn_id & 0xffff; |
||||
header.stream_id = ReadLittleEndianUint32(data + 4); |
||||
header.payload_length = ReadLittleEndianUint32(data + 8); |
||||
return header; |
||||
} |
||||
|
||||
std::string FrameHeader::ToString() const { |
||||
return absl::StrCat("[type:", type, " conn:", payload_connection_id, |
||||
" stream_id:", stream_id, |
||||
" payload_length:", payload_length, "]"); |
||||
} |
||||
|
||||
std::string FrameTypeString(FrameType type) { |
||||
switch (type) { |
||||
case FrameType::kSettings: |
||||
return "Settings"; |
||||
case FrameType::kClientInitialMetadata: |
||||
return "ClientInitialMetadata"; |
||||
case FrameType::kClientEndOfStream: |
||||
return "ClientEndOfStream"; |
||||
case FrameType::kMessage: |
||||
return "Message"; |
||||
case FrameType::kServerInitialMetadata: |
||||
return "ServerInitialMetadata"; |
||||
case FrameType::kServerTrailingMetadata: |
||||
return "ServerTrailingMetadata"; |
||||
case FrameType::kCancel: |
||||
return "Cancel"; |
||||
case FrameType::kBeginMessage: |
||||
return "BeginMessage"; |
||||
case FrameType::kMessageChunk: |
||||
return "MessageChunk"; |
||||
} |
||||
return absl::StrCat("Unknown[", static_cast<int>(type), "]"); |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,92 @@ |
||||
// Copyright 2022 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_FRAME_HEADER_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_FRAME_HEADER_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
#include <stddef.h> |
||||
|
||||
#include <cstdint> |
||||
|
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/util/bitset.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
// Remember to add new frame types to frame_fuzzer.cc
|
||||
enum class FrameType : uint8_t { |
||||
kSettings = 0x00, |
||||
kClientInitialMetadata = 0x80, |
||||
kClientEndOfStream = 0x81, |
||||
kServerInitialMetadata = 0x91, |
||||
kServerTrailingMetadata = 0x92, |
||||
kMessage = 0xa0, |
||||
kBeginMessage = 0xa1, |
||||
kMessageChunk = 0xa2, |
||||
kCancel = 0xff, |
||||
}; |
||||
|
||||
std::string FrameTypeString(FrameType type); |
||||
|
||||
inline std::ostream& operator<<(std::ostream& out, FrameType type) { |
||||
return out << FrameTypeString(type); |
||||
} |
||||
|
||||
template <typename Sink> |
||||
void AbslStringify(Sink& sink, FrameType type) { |
||||
sink.Append(FrameTypeString(type)); |
||||
} |
||||
|
||||
struct FrameHeader { |
||||
FrameType type = FrameType::kCancel; |
||||
uint16_t payload_connection_id = 0; |
||||
uint32_t stream_id = 0; |
||||
uint32_t payload_length = 0; |
||||
|
||||
// Parses a frame header from a buffer of 12 bytes. All 12 bytes are consumed.
|
||||
static absl::StatusOr<FrameHeader> Parse(const uint8_t* data); |
||||
// Serializes a frame header into a buffer of 12 bytes.
|
||||
void Serialize(uint8_t* data) const; |
||||
// Report contents as a string
|
||||
std::string ToString() const; |
||||
// Required padding to maintain alignment.
|
||||
uint32_t Padding(uint32_t alignment) const { |
||||
if (payload_connection_id == 0) { |
||||
return 0; |
||||
} |
||||
if (payload_length % alignment == 0) { |
||||
return 0; |
||||
} |
||||
return alignment - (payload_length % alignment); |
||||
} |
||||
|
||||
bool operator==(const FrameHeader& h) const { |
||||
return type == h.type && stream_id == h.stream_id && |
||||
payload_connection_id == h.payload_connection_id && |
||||
payload_length == h.payload_length; |
||||
} |
||||
// Frame header size is fixed to 12 bytes.
|
||||
enum { kFrameHeaderSize = 12 }; |
||||
}; |
||||
|
||||
inline std::ostream& operator<<(std::ostream& out, const FrameHeader& h) { |
||||
return out << h.ToString(); |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_FRAME_HEADER_H
|
@ -0,0 +1,138 @@ |
||||
// 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_MESSAGE_CHUNKER_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_MESSAGE_CHUNKER_H |
||||
|
||||
#include <cstdint> |
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/lib/promise/if.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/map.h" |
||||
#include "src/core/lib/promise/seq.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
namespace message_chunker_detail { |
||||
struct ChunkResult { |
||||
MessageChunkFrame frame; |
||||
bool done; |
||||
}; |
||||
|
||||
// Given a set of parameters defining the desired size, produce frames of chunks
|
||||
// of payload to send one at a time.
|
||||
class PayloadChunker { |
||||
public: |
||||
PayloadChunker(uint32_t max_chunk_size, uint32_t alignment, |
||||
uint32_t stream_id, SliceBuffer payload) |
||||
: max_chunk_size_(max_chunk_size), |
||||
alignment_(alignment), |
||||
stream_id_(stream_id), |
||||
payload_(std::move(payload)) {} |
||||
PayloadChunker(const PayloadChunker&) = delete; |
||||
PayloadChunker(PayloadChunker&&) = default; |
||||
PayloadChunker& operator=(const PayloadChunker&) = delete; |
||||
PayloadChunker& operator=(PayloadChunker&&) = delete; |
||||
|
||||
ChunkResult NextChunk() { |
||||
auto remaining = payload_.Length(); |
||||
ChunkResult result; |
||||
if (remaining > max_chunk_size_) { |
||||
auto take = max_chunk_size_; |
||||
if (remaining / 2 < max_chunk_size_) { |
||||
// If there's very little remaining, then we make the last two chunks
|
||||
// about equal sized to make later load balancing a little easier to
|
||||
// reason about (nobody likes a teeny tiny straggler).
|
||||
take = remaining / 2; |
||||
// But we try to keep alignment rounded lengths for the first chunk.
|
||||
// This way we don't accidentally split things up such that padding is
|
||||
// needed (and potentially then copying elsewhere).
|
||||
if (alignment_ != 0 && take % alignment_ != 0) { |
||||
take += alignment_ - (take % alignment_); |
||||
if (take > max_chunk_size_) take = max_chunk_size_; |
||||
} |
||||
} |
||||
payload_.MoveFirstNBytesIntoSliceBuffer(take, result.frame.payload); |
||||
result.frame.stream_id = stream_id_; |
||||
result.done = false; |
||||
} else { |
||||
result.frame.payload = std::move(payload_); |
||||
result.frame.stream_id = stream_id_; |
||||
result.done = true; |
||||
} |
||||
return result; |
||||
} |
||||
|
||||
private: |
||||
uint32_t max_chunk_size_; |
||||
uint32_t alignment_; |
||||
uint32_t stream_id_; |
||||
SliceBuffer payload_; |
||||
}; |
||||
} // namespace message_chunker_detail
|
||||
|
||||
// Helper to send message payloads (possibly chunked!) between client & server.
|
||||
class MessageChunker { |
||||
public: |
||||
MessageChunker(uint32_t max_chunk_size, uint32_t alignment) |
||||
: max_chunk_size_(max_chunk_size), alignment_(alignment) {} |
||||
|
||||
template <typename Output> |
||||
auto Send(MessageHandle message, uint32_t stream_id, Output& output) { |
||||
return If( |
||||
ShouldChunk(*message), |
||||
[&]() { |
||||
BeginMessageFrame begin; |
||||
begin.body.set_length(message->payload()->Length()); |
||||
begin.stream_id = stream_id; |
||||
return Seq(output.Send(std::move(begin)), |
||||
Loop([chunker = message_chunker_detail::PayloadChunker( |
||||
max_chunk_size_, alignment_, stream_id, |
||||
std::move(*message->payload())), |
||||
&output]() mutable { |
||||
auto next = chunker.NextChunk(); |
||||
return Map(output.Send(std::move(next.frame)), |
||||
[done = next.done](bool x) -> LoopCtl<bool> { |
||||
if (!done) return Continue{}; |
||||
return x; |
||||
}); |
||||
})); |
||||
}, |
||||
[&]() { |
||||
MessageFrame frame; |
||||
frame.message = std::move(message); |
||||
frame.stream_id = stream_id; |
||||
return output.Send(std::move(frame)); |
||||
}); |
||||
} |
||||
|
||||
uint32_t max_chunk_size() const { return max_chunk_size_; } |
||||
uint32_t alignment() const { return alignment_; } |
||||
|
||||
private: |
||||
bool ShouldChunk(Message& message) { |
||||
return max_chunk_size_ != 0 && |
||||
message.payload()->Length() > max_chunk_size_; |
||||
} |
||||
|
||||
const uint32_t max_chunk_size_; |
||||
const uint32_t alignment_; |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_MESSAGE_CHUNKER_H
|
@ -0,0 +1,118 @@ |
||||
// 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_MESSAGE_REASSEMBLY_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_MESSAGE_REASSEMBLY_H |
||||
|
||||
#include "absl/log/log.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/lib/transport/call_spine.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
// Reassemble chunks of messages into messages, and enforce invariants about
|
||||
// never having two messages in flight on the same stream.
|
||||
class MessageReassembly { |
||||
public: |
||||
void FailCall(CallInitiator& call, absl::string_view msg) { |
||||
LOG_EVERY_N_SEC(INFO, 10) << "Call failed during reassembly: " << msg; |
||||
call.Cancel(); |
||||
} |
||||
void FailCall(CallHandler& call, absl::string_view msg) { |
||||
LOG_EVERY_N_SEC(INFO, 10) << "Call failed during reassembly: " << msg; |
||||
call.PushServerTrailingMetadata( |
||||
CancelledServerMetadataFromStatus(GRPC_STATUS_INTERNAL, msg)); |
||||
} |
||||
|
||||
template <typename Sink> |
||||
auto PushFrameInto(MessageFrame frame, Sink& sink) { |
||||
return If( |
||||
in_message_boundary(), |
||||
[&]() { return sink.PushMessage(std::move(frame.message)); }, |
||||
[&]() { |
||||
FailCall(sink, |
||||
"Received full message without completing previous chunked " |
||||
"message"); |
||||
return Immediate(StatusFlag(Failure{})); |
||||
}); |
||||
} |
||||
|
||||
template <typename Sink> |
||||
auto PushFrameInto(BeginMessageFrame frame, Sink& sink) { |
||||
bool ok = false; |
||||
if (!in_message_boundary()) { |
||||
FailCall(sink, |
||||
"Received begin message without completing previous chunked " |
||||
"message"); |
||||
} else if (frame.body.length() == 0) { |
||||
FailCall(sink, |
||||
"Received begin message for an empty message (not allowed)"); |
||||
} else if (frame.body.length() > std::numeric_limits<size_t>::max() / 2) { |
||||
FailCall(sink, "Received too large begin message"); |
||||
} else { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< this << " begin message " << frame.body.ShortDebugString(); |
||||
chunk_receiver_ = std::make_unique<ChunkReceiver>(); |
||||
chunk_receiver_->bytes_remaining = frame.body.length(); |
||||
ok = true; |
||||
} |
||||
return Immediate(StatusFlag(ok)); |
||||
} |
||||
|
||||
template <typename Sink> |
||||
auto PushFrameInto(MessageChunkFrame frame, Sink& sink) { |
||||
bool ok = false; |
||||
bool done = false; |
||||
if (in_message_boundary()) { |
||||
FailCall(sink, "Received message chunk without BeginMessage"); |
||||
} else if (chunk_receiver_->bytes_remaining < frame.payload.Length()) { |
||||
FailCall(sink, "Message chunks are longer than BeginMessage declared"); |
||||
} else { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: " << this << " got chunk " << frame.payload.Length() |
||||
<< "b in message with " << chunk_receiver_->bytes_remaining |
||||
<< "b left"; |
||||
chunk_receiver_->bytes_remaining -= frame.payload.Length(); |
||||
chunk_receiver_->incoming.Append(frame.payload); |
||||
ok = true; |
||||
done = chunk_receiver_->bytes_remaining == 0; |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: " << this << " " << GRPC_DUMP_ARGS(ok, done); |
||||
} |
||||
return If( |
||||
done, |
||||
[&]() { |
||||
auto message = Arena::MakePooled<Message>( |
||||
std::move(chunk_receiver_->incoming), 0); |
||||
chunk_receiver_.reset(); |
||||
return sink.PushMessage(std::move(message)); |
||||
}, |
||||
[ok]() { return StatusFlag(ok); }); |
||||
} |
||||
|
||||
bool in_message_boundary() { return chunk_receiver_ == nullptr; } |
||||
|
||||
private: |
||||
struct ChunkReceiver { |
||||
size_t bytes_remaining; |
||||
SliceBuffer incoming; |
||||
}; |
||||
std::unique_ptr<ChunkReceiver> chunk_receiver_; |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_MESSAGE_REASSEMBLY_H
|
@ -0,0 +1,74 @@ |
||||
// 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_PENDING_CONNECTION_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_PENDING_CONNECTION_H |
||||
|
||||
#include <string> |
||||
|
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/lib/promise/promise.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/util/dual_ref_counted.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
// Essentially this is the promise of one endpoint in the future, with the
|
||||
// addition of an id used for handshaking so that can be communicated around as
|
||||
// necessary.
|
||||
class PendingConnection { |
||||
public: |
||||
explicit PendingConnection(absl::string_view id, |
||||
Promise<absl::StatusOr<PromiseEndpoint>> connector) |
||||
: id_(id), connector_(std::move(connector)) {} |
||||
|
||||
PendingConnection(const PendingConnection&) = delete; |
||||
PendingConnection& operator=(const PendingConnection&) = delete; |
||||
PendingConnection(PendingConnection&&) = default; |
||||
PendingConnection& operator=(PendingConnection&&) = default; |
||||
|
||||
absl::string_view id() const { return id_; } |
||||
auto Await() { return std::move(connector_); } |
||||
|
||||
private: |
||||
std::string id_; |
||||
Promise<absl::StatusOr<PromiseEndpoint>> connector_; |
||||
}; |
||||
|
||||
class ServerConnectionFactory : public DualRefCounted<ServerConnectionFactory> { |
||||
public: |
||||
using DualRefCounted::DualRefCounted; |
||||
virtual PendingConnection RequestDataConnection() = 0; |
||||
}; |
||||
|
||||
class ClientConnectionFactory : public DualRefCounted<ClientConnectionFactory> { |
||||
public: |
||||
using DualRefCounted::DualRefCounted; |
||||
virtual PendingConnection Connect(absl::string_view id) = 0; |
||||
}; |
||||
|
||||
// Helper: convert an already existing endpoint into a pending connection
|
||||
inline PendingConnection ImmediateConnection(absl::string_view id, |
||||
PromiseEndpoint endpoint) { |
||||
return PendingConnection( |
||||
id, |
||||
[endpoint = std::move(endpoint)]() mutable |
||||
-> absl::StatusOr<PromiseEndpoint> { return std::move(endpoint); }); |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_PENDING_CONNECTION_H
|
@ -0,0 +1,555 @@ |
||||
// 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.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/server/chaotic_good_server.h" |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/slice.h> |
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstdint> |
||||
#include <memory> |
||||
#include <random> |
||||
#include <string> |
||||
#include <utility> |
||||
#include <vector> |
||||
|
||||
#include "absl/log/check.h" |
||||
#include "absl/log/log.h" |
||||
#include "absl/random/bit_gen_ref.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/server_transport.h" |
||||
#include "src/core/handshaker/handshaker.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/event_engine/channel_args_endpoint_config.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/event_engine/extensions/chaotic_good_extension.h" |
||||
#include "src/core/lib/event_engine/query_extensions.h" |
||||
#include "src/core/lib/event_engine/resolved_address_internal.h" |
||||
#include "src/core/lib/event_engine/tcp_socket_utils.h" |
||||
#include "src/core/lib/event_engine/utils.h" |
||||
#include "src/core/lib/iomgr/error.h" |
||||
#include "src/core/lib/iomgr/event_engine_shims/endpoint.h" |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/context.h" |
||||
#include "src/core/lib/promise/event_engine_wakeup_scheduler.h" |
||||
#include "src/core/lib/promise/if.h" |
||||
#include "src/core/lib/promise/join.h" |
||||
#include "src/core/lib/promise/latch.h" |
||||
#include "src/core/lib/promise/race.h" |
||||
#include "src/core/lib/promise/sleep.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/transport/error_utils.h" |
||||
#include "src/core/lib/transport/metadata.h" |
||||
#include "src/core/lib/transport/metadata_batch.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/server/server.h" |
||||
#include "src/core/util/orphanable.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
#include "src/core/util/status_helper.h" |
||||
#include "src/core/util/sync.h" |
||||
#include "src/core/util/time.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
namespace { |
||||
const Duration kConnectionDeadline = Duration::Seconds(120); |
||||
} // namespace
|
||||
|
||||
using grpc_event_engine::experimental::EventEngine; |
||||
|
||||
ChaoticGoodServerListener::ChaoticGoodServerListener( |
||||
Server* server, const ChannelArgs& args, |
||||
absl::AnyInvocable<std::string()> connection_id_generator) |
||||
: server_(server), |
||||
args_(args), |
||||
event_engine_( |
||||
args.GetObjectRef<grpc_event_engine::experimental::EventEngine>()), |
||||
data_connection_listener_(MakeRefCounted<DataConnectionListener>( |
||||
std::move(connection_id_generator), |
||||
args.GetDurationFromIntMillis(GRPC_ARG_SERVER_HANDSHAKE_TIMEOUT_MS) |
||||
.value_or(kConnectionDeadline), |
||||
event_engine_)) {} |
||||
|
||||
ChaoticGoodServerListener::~ChaoticGoodServerListener() { |
||||
if (on_destroy_done_ != nullptr) { |
||||
event_engine_->Run([on_destroy_done = on_destroy_done_]() { |
||||
ExecCtx exec_ctx; |
||||
ExecCtx::Run(DEBUG_LOCATION, on_destroy_done, absl::OkStatus()); |
||||
}); |
||||
} |
||||
} |
||||
|
||||
absl::StatusOr<int> ChaoticGoodServerListener::Bind( |
||||
grpc_event_engine::experimental::EventEngine::ResolvedAddress addr) { |
||||
if (GRPC_TRACE_FLAG_ENABLED(chaotic_good)) { |
||||
auto str = grpc_event_engine::experimental::ResolvedAddressToString(addr); |
||||
LOG(INFO) << "CHAOTIC_GOOD: Listen on " |
||||
<< (str.ok() ? str->c_str() : str.status().ToString()); |
||||
} |
||||
EventEngine::Listener::AcceptCallback accept_cb = |
||||
[self = RefAsSubclass<ChaoticGoodServerListener>()]( |
||||
std::unique_ptr<EventEngine::Endpoint> ep, MemoryAllocator) { |
||||
ExecCtx exec_ctx; |
||||
MutexLock lock(&self->mu_); |
||||
if (self->shutdown_) return; |
||||
self->connection_list_.emplace( |
||||
MakeOrphanable<ActiveConnection>(self, std::move(ep))); |
||||
}; |
||||
auto shutdown_cb = [](absl::Status status) { |
||||
if (!status.ok()) { |
||||
LOG(ERROR) << "Server accept connection failed: " << status; |
||||
} |
||||
}; |
||||
CHECK_NE(event_engine_, nullptr); |
||||
auto ee_listener = event_engine_->CreateListener( |
||||
std::move(accept_cb), std::move(shutdown_cb), |
||||
grpc_event_engine::experimental::ChannelArgsEndpointConfig(args_), |
||||
std::make_unique<MemoryQuota>("chaotic_good_server_listener")); |
||||
if (!ee_listener.ok()) { |
||||
LOG(ERROR) << "Bind failed: " << ee_listener.status().ToString(); |
||||
return ee_listener.status(); |
||||
} |
||||
ee_listener_ = std::move(ee_listener.value()); |
||||
auto port_num = ee_listener_->Bind(addr); |
||||
if (!port_num.ok()) { |
||||
return port_num.status(); |
||||
} |
||||
return port_num; |
||||
} |
||||
|
||||
absl::Status ChaoticGoodServerListener::StartListening() { |
||||
CHECK(ee_listener_ != nullptr); |
||||
auto status = ee_listener_->Start(); |
||||
if (!status.ok()) { |
||||
LOG(ERROR) << "Start listening failed: " << status; |
||||
} else { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) << "CHAOTIC_GOOD: Started listening"; |
||||
} |
||||
return status; |
||||
} |
||||
|
||||
ChaoticGoodServerListener::ActiveConnection::ActiveConnection( |
||||
RefCountedPtr<ChaoticGoodServerListener> listener, |
||||
std::unique_ptr<EventEngine::Endpoint> endpoint) |
||||
: listener_(std::move(listener)) { |
||||
arena_->SetContext<grpc_event_engine::experimental::EventEngine>( |
||||
listener_->event_engine_.get()); |
||||
handshaking_state_ = MakeRefCounted<HandshakingState>(Ref()); |
||||
handshaking_state_->Start(std::move(endpoint)); |
||||
} |
||||
|
||||
ChaoticGoodServerListener::ActiveConnection::~ActiveConnection() { |
||||
if (receive_settings_activity_ != nullptr) receive_settings_activity_.reset(); |
||||
} |
||||
|
||||
void ChaoticGoodServerListener::ActiveConnection::Orphan() { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) << "ActiveConnection::Orphan() " << this; |
||||
if (handshaking_state_ != nullptr) { |
||||
handshaking_state_->Shutdown(); |
||||
handshaking_state_.reset(); |
||||
} |
||||
ActivityPtr activity; |
||||
{ |
||||
MutexLock lock(&mu_); |
||||
orphaned_ = true; |
||||
activity = std::move(receive_settings_activity_); |
||||
} |
||||
activity.reset(); |
||||
Unref(); |
||||
} |
||||
|
||||
ChaoticGoodServerListener::DataConnectionListener::DataConnectionListener( |
||||
absl::AnyInvocable<std::string()> connection_id_generator, |
||||
Duration connect_timeout, |
||||
std::shared_ptr<grpc_event_engine::experimental::EventEngine> event_engine) |
||||
: connection_id_generator_(std::move(connection_id_generator)), |
||||
event_engine_(std::move(event_engine)), |
||||
connect_timeout_(connect_timeout) {} |
||||
|
||||
PendingConnection |
||||
ChaoticGoodServerListener::DataConnectionListener::RequestDataConnection() { |
||||
MutexLock lock(&mu_); |
||||
std::string connection_id; |
||||
while (true) { |
||||
connection_id = connection_id_generator_(); |
||||
if (!pending_connections_.contains(connection_id)) break; |
||||
} |
||||
if (shutdown_) { |
||||
return PendingConnection(connection_id, []() { |
||||
return absl::UnavailableError("Server shutdown"); |
||||
}); |
||||
} |
||||
auto latch = std::make_shared<PromiseEndpointLatch>(); |
||||
auto timeout_task = event_engine_->RunAfter( |
||||
connect_timeout_, |
||||
[connection_id, self = WeakRefAsSubclass<DataConnectionListener>()]() { |
||||
self->ConnectionTimeout(connection_id); |
||||
}); |
||||
pending_connections_.emplace(connection_id, |
||||
PendingConnectionInfo{latch, timeout_task}); |
||||
return PendingConnection(connection_id, |
||||
Map(latch->Wait(), [latch](auto x) { return x; })); |
||||
} |
||||
|
||||
ChaoticGoodServerListener::DataConnectionListener::PromiseEndpointLatchPtr |
||||
ChaoticGoodServerListener::DataConnectionListener::Extract( |
||||
absl::string_view id) { |
||||
MutexLock lock(&mu_); |
||||
auto ex = pending_connections_.extract(id); |
||||
if (!ex.empty()) { |
||||
event_engine_->Cancel(ex.mapped().timeout); |
||||
return std::move(ex.mapped().latch); |
||||
} |
||||
return nullptr; |
||||
} |
||||
|
||||
void ChaoticGoodServerListener::DataConnectionListener::ConnectionTimeout( |
||||
absl::string_view id) { |
||||
auto latch = Extract(id); |
||||
if (latch != nullptr) { |
||||
latch->Set(absl::DeadlineExceededError("Connection timeout")); |
||||
} |
||||
} |
||||
|
||||
void ChaoticGoodServerListener::DataConnectionListener::FinishDataConnection( |
||||
absl::string_view id, PromiseEndpoint endpoint) { |
||||
auto latch = Extract(id); |
||||
if (latch != nullptr) { |
||||
latch->Set(std::move(endpoint)); |
||||
} |
||||
} |
||||
|
||||
void ChaoticGoodServerListener::DataConnectionListener::Orphaned() { |
||||
absl::flat_hash_map<std::string, PendingConnectionInfo> pending_connections; |
||||
{ |
||||
MutexLock lock(&mu_); |
||||
CHECK(!shutdown_); |
||||
pending_connections = std::move(pending_connections_); |
||||
pending_connections_.clear(); |
||||
shutdown_ = true; |
||||
} |
||||
for (const auto& conn : pending_connections) { |
||||
event_engine_->Cancel(conn.second.timeout); |
||||
conn.second.latch->Set(absl::UnavailableError("Server shutdown")); |
||||
} |
||||
} |
||||
|
||||
void ChaoticGoodServerListener::ActiveConnection::Done() { |
||||
// Can easily be holding various locks here: bounce through EE to ensure no
|
||||
// deadlocks.
|
||||
listener_->event_engine_->Run([self = Ref()]() { |
||||
ExecCtx exec_ctx; |
||||
OrphanablePtr<ActiveConnection> con; |
||||
MutexLock lock(&self->listener_->mu_); |
||||
auto v = self->listener_->connection_list_.extract(self.get()); |
||||
if (!v.empty()) con = std::move(v.value()); |
||||
}); |
||||
} |
||||
|
||||
ChaoticGoodServerListener::ActiveConnection::HandshakingState::HandshakingState( |
||||
RefCountedPtr<ActiveConnection> connection) |
||||
: connection_(std::move(connection)), |
||||
handshake_mgr_(MakeRefCounted<HandshakeManager>()) {} |
||||
|
||||
void ChaoticGoodServerListener::ActiveConnection::HandshakingState::Start( |
||||
std::unique_ptr<EventEngine::Endpoint> endpoint) { |
||||
handshake_mgr_->DoHandshake( |
||||
OrphanablePtr<grpc_endpoint>( |
||||
grpc_event_engine_endpoint_create(std::move(endpoint))), |
||||
connection_->args(), |
||||
Timestamp::Now() + connection_->listener_->data_connection_listener_ |
||||
->connection_timeout(), |
||||
nullptr, [self = Ref()](absl::StatusOr<HandshakerArgs*> result) { |
||||
self->OnHandshakeDone(std::move(result)); |
||||
}); |
||||
} |
||||
|
||||
auto ChaoticGoodServerListener::ActiveConnection::HandshakingState:: |
||||
EndpointReadSettingsFrame(RefCountedPtr<HandshakingState> self) { |
||||
return TrySeq( |
||||
self->connection_->endpoint_.ReadSlice(FrameHeader::kFrameHeaderSize), |
||||
[self](Slice slice) { |
||||
// Parse frame header
|
||||
auto frame_header = FrameHeader::Parse(reinterpret_cast<const uint8_t*>( |
||||
GRPC_SLICE_START_PTR(slice.c_slice()))); |
||||
if (frame_header.ok() && frame_header->type != FrameType::kSettings) { |
||||
frame_header = absl::InternalError("Not a settings frame"); |
||||
} |
||||
return If( |
||||
frame_header.ok(), |
||||
[self, &frame_header]() { |
||||
return TrySeq( |
||||
self->connection_->endpoint_.Read( |
||||
frame_header->payload_length), |
||||
[frame_header = *frame_header, |
||||
self](SliceBuffer buffer) -> absl::StatusOr<bool> { |
||||
// Read Setting frame.
|
||||
SettingsFrame frame; |
||||
// Deserialize frame from read buffer.
|
||||
auto status = |
||||
frame.Deserialize(frame_header, std::move(buffer)); |
||||
if (!status.ok()) return status; |
||||
if (frame.body.data_channel()) { |
||||
if (frame.body.connection_id().empty()) { |
||||
return absl::UnavailableError( |
||||
"no connection id in data endpoint settings frame"); |
||||
} |
||||
if (frame.body.connection_id().size() != 1) { |
||||
return absl::UnavailableError(absl::StrCat( |
||||
"Got ", frame.body.connection_id().size(), |
||||
" connection ids in data endpoint " |
||||
"settings frame (expect one)")); |
||||
} |
||||
self->data_.emplace<DataConnection>( |
||||
frame.body.connection_id()[0]); |
||||
} else { |
||||
Config config{self->connection_->args()}; |
||||
auto settings_status = |
||||
config.ReceiveClientIncomingSettings(frame.body); |
||||
if (!settings_status.ok()) return settings_status; |
||||
const int num_data_connections = |
||||
self->connection_->listener_->args() |
||||
.GetInt(GRPC_ARG_CHAOTIC_GOOD_DATA_CONNECTIONS) |
||||
.value_or(1); |
||||
auto& data_connection_listener = |
||||
*self->connection_->listener_ |
||||
->data_connection_listener_; |
||||
for (int i = 0; i < num_data_connections; i++) { |
||||
config.ServerAddPendingDataEndpoint( |
||||
data_connection_listener.RequestDataConnection()); |
||||
} |
||||
self->data_.emplace<ControlConnection>(std::move(config)); |
||||
} |
||||
return !frame.body.data_channel(); |
||||
}); |
||||
}, |
||||
[&frame_header]() { |
||||
return [r = frame_header.status()]() -> absl::StatusOr<bool> { |
||||
return r; |
||||
}; |
||||
}); |
||||
}); |
||||
} |
||||
|
||||
auto ChaoticGoodServerListener::ActiveConnection::HandshakingState:: |
||||
ControlEndpointWriteSettingsFrame(RefCountedPtr<HandshakingState> self) { |
||||
SettingsFrame frame; |
||||
frame.body.set_data_channel(false); |
||||
std::get<ControlConnection>(self->data_) |
||||
.config.PrepareServerOutgoingSettings(frame.body); |
||||
SliceBuffer write_buffer; |
||||
frame.MakeHeader().Serialize( |
||||
write_buffer.AddTiny(FrameHeader::kFrameHeaderSize)); |
||||
frame.SerializePayload(write_buffer); |
||||
return TrySeq( |
||||
self->connection_->endpoint_.Write(std::move(write_buffer)), [self]() { |
||||
return self->connection_->listener_->server_->SetupTransport( |
||||
new ChaoticGoodServerTransport( |
||||
self->connection_->args(), |
||||
std::move(self->connection_->endpoint_), |
||||
std::move(std::get<ControlConnection>(self->data_).config), |
||||
self->connection_->listener_->data_connection_listener_), |
||||
nullptr, self->connection_->args(), nullptr); |
||||
}); |
||||
} |
||||
|
||||
auto ChaoticGoodServerListener::ActiveConnection::HandshakingState:: |
||||
DataEndpointWriteSettingsFrame(RefCountedPtr<HandshakingState> self) { |
||||
// Send data endpoint setting frame
|
||||
SettingsFrame frame; |
||||
frame.body.set_data_channel(true); |
||||
SliceBuffer write_buffer; |
||||
frame.MakeHeader().Serialize( |
||||
write_buffer.AddTiny(FrameHeader::kFrameHeaderSize)); |
||||
frame.SerializePayload(write_buffer); |
||||
// ignore encoding errors: they will be logged separately already
|
||||
return TrySeq(self->connection_->endpoint_.Write(std::move(write_buffer)), |
||||
[self]() mutable { |
||||
self->connection_->listener_->data_connection_listener_ |
||||
->FinishDataConnection( |
||||
std::get<DataConnection>(self->data_).connection_id, |
||||
std::move(self->connection_->endpoint_)); |
||||
return absl::OkStatus(); |
||||
}); |
||||
} |
||||
|
||||
auto ChaoticGoodServerListener::ActiveConnection::HandshakingState:: |
||||
EndpointWriteSettingsFrame(RefCountedPtr<HandshakingState> self, |
||||
bool is_control_endpoint) { |
||||
return If( |
||||
is_control_endpoint, |
||||
[&self] { return ControlEndpointWriteSettingsFrame(self); }, |
||||
[&self] { return DataEndpointWriteSettingsFrame(self); }); |
||||
} |
||||
|
||||
void ChaoticGoodServerListener::ActiveConnection::HandshakingState:: |
||||
OnHandshakeDone(absl::StatusOr<HandshakerArgs*> result) { |
||||
if (!result.ok()) { |
||||
LOG_EVERY_N_SEC(ERROR, 5) << "Handshake failed: ", result.status(); |
||||
connection_->Done(); |
||||
return; |
||||
} |
||||
CHECK_NE(*result, nullptr); |
||||
if ((*result)->endpoint == nullptr) { |
||||
LOG_EVERY_N_SEC(ERROR, 5) |
||||
<< "Server handshake done but has empty endpoint."; |
||||
connection_->Done(); |
||||
return; |
||||
} |
||||
CHECK(grpc_event_engine::experimental::grpc_is_event_engine_endpoint( |
||||
(*result)->endpoint.get())); |
||||
auto ee_endpoint = |
||||
grpc_event_engine::experimental::grpc_take_wrapped_event_engine_endpoint( |
||||
(*result)->endpoint.release()); |
||||
auto* chaotic_good_ext = grpc_event_engine::experimental::QueryExtension< |
||||
grpc_event_engine::experimental::ChaoticGoodExtension>(ee_endpoint.get()); |
||||
connection_->endpoint_ = |
||||
PromiseEndpoint(std::move(ee_endpoint), SliceBuffer()); |
||||
auto activity = MakeActivity( |
||||
[self = Ref(), chaotic_good_ext]() { |
||||
return TrySeq( |
||||
Race(EndpointReadSettingsFrame(self), |
||||
TrySeq(Sleep(Timestamp::Now() + kConnectionDeadline), |
||||
[]() -> absl::StatusOr<bool> { |
||||
return absl::DeadlineExceededError( |
||||
"Waiting for initial settings frame"); |
||||
})), |
||||
[self, chaotic_good_ext](bool is_control_endpoint) { |
||||
if (chaotic_good_ext != nullptr) { |
||||
chaotic_good_ext->EnableStatsCollection(is_control_endpoint); |
||||
if (is_control_endpoint) { |
||||
// Control endpoint should use the default memory quota
|
||||
chaotic_good_ext->UseMemoryQuota( |
||||
ResourceQuota::Default()->memory_quota()); |
||||
} |
||||
} |
||||
return EndpointWriteSettingsFrame(self, is_control_endpoint); |
||||
}); |
||||
}, |
||||
EventEngineWakeupScheduler(connection_->listener_->event_engine_), |
||||
[self = Ref()](absl::Status status) { |
||||
if (!status.ok()) { |
||||
GRPC_TRACE_LOG(chaotic_good, ERROR) |
||||
<< "Server setting frame handling failed: " << status; |
||||
} |
||||
self->connection_->Done(); |
||||
}, |
||||
connection_->arena_.get()); |
||||
MutexLock lock(&connection_->mu_); |
||||
if (connection_->orphaned_) return; |
||||
connection_->receive_settings_activity_ = std::move(activity); |
||||
} |
||||
|
||||
void ChaoticGoodServerListener::Orphan() { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) << "ChaoticGoodServerListener::Orphan()"; |
||||
{ |
||||
absl::flat_hash_set<OrphanablePtr<ActiveConnection>> connection_list; |
||||
MutexLock lock(&mu_); |
||||
connection_list = std::move(connection_list_); |
||||
connection_list_.clear(); |
||||
shutdown_ = true; |
||||
} |
||||
ee_listener_.reset(); |
||||
Unref(); |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int grpc_server_add_chaotic_good_legacy_port(grpc_server* server, |
||||
const char* addr) { |
||||
using grpc_event_engine::experimental::EventEngine; |
||||
grpc_core::ExecCtx exec_ctx; |
||||
auto* const core_server = grpc_core::Server::FromC(server); |
||||
const std::string parsed_addr = grpc_core::URI::PercentDecode(addr); |
||||
absl::StatusOr<std::vector<EventEngine::ResolvedAddress>> results = |
||||
std::vector<EventEngine::ResolvedAddress>(); |
||||
if (grpc_core::IsEventEngineDnsNonClientChannelEnabled()) { |
||||
absl::StatusOr<std::unique_ptr<EventEngine::DNSResolver>> ee_resolver = |
||||
core_server->channel_args().GetObjectRef<EventEngine>()->GetDNSResolver( |
||||
EventEngine::DNSResolver::ResolverOptions()); |
||||
if (!ee_resolver.ok()) { |
||||
LOG(ERROR) << "Failed to resolve " << addr << ": " |
||||
<< ee_resolver.status().ToString(); |
||||
return 0; |
||||
} |
||||
results = grpc_event_engine::experimental::LookupHostnameBlocking( |
||||
ee_resolver->get(), parsed_addr, absl::StrCat(0xd20)); |
||||
if (!results.ok()) { |
||||
LOG(ERROR) << "Failed to resolve " << addr << ": " |
||||
<< results.status().ToString(); |
||||
return 0; |
||||
} |
||||
} else { |
||||
// TODO(yijiem): Remove this after event_engine_dns_non_client_channel
|
||||
// is fully enabled.
|
||||
const auto resolved_or = |
||||
grpc_core::GetDNSResolver()->LookupHostnameBlocking( |
||||
parsed_addr, absl::StrCat(0xd20)); |
||||
if (!resolved_or.ok()) { |
||||
LOG(ERROR) << "Failed to resolve " << addr << ": " |
||||
<< resolved_or.status().ToString(); |
||||
return 0; |
||||
} |
||||
for (const auto& addr : *resolved_or) { |
||||
results->push_back( |
||||
grpc_event_engine::experimental::CreateResolvedAddress(addr)); |
||||
} |
||||
} |
||||
int port_num = 0; |
||||
std::vector<std::pair<std::string, absl::Status>> error_list; |
||||
for (const auto& ee_addr : results.value()) { |
||||
auto listener = grpc_core::MakeOrphanable< |
||||
grpc_core::chaotic_good_legacy::ChaoticGoodServerListener>( |
||||
grpc_core::Server::FromC(server), core_server->channel_args()); |
||||
std::string addr_str = |
||||
*grpc_event_engine::experimental::ResolvedAddressToString(ee_addr); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) << "BIND: " << addr_str; |
||||
auto bind_result = listener->Bind(ee_addr); |
||||
if (!bind_result.ok()) { |
||||
error_list.push_back( |
||||
std::pair(std::move(addr_str), bind_result.status())); |
||||
continue; |
||||
} |
||||
if (port_num == 0) { |
||||
port_num = bind_result.value(); |
||||
} else { |
||||
CHECK(port_num == bind_result.value()); |
||||
} |
||||
core_server->AddListener(std::move(listener)); |
||||
} |
||||
if (error_list.size() == results->size()) { |
||||
LOG(ERROR) << "Failed to bind any address for " << addr; |
||||
for (const auto& error : error_list) { |
||||
LOG(ERROR) << " " << error.first << ": " << error.second; |
||||
} |
||||
} else if (!error_list.empty()) { |
||||
LOG(INFO) << "Failed to bind some addresses for " << addr; |
||||
for (const auto& error : error_list) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "Binding Failed: " << error.first << ": " << error.second; |
||||
} |
||||
} |
||||
return port_num; |
||||
} |
@ -0,0 +1,214 @@ |
||||
// Copyright 2022 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_SERVER_CHAOTIC_GOOD_SERVER_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_SERVER_CHAOTIC_GOOD_SERVER_H |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <cstddef> |
||||
#include <cstdint> |
||||
#include <memory> |
||||
#include <string> |
||||
#include <vector> |
||||
|
||||
#include "absl/container/flat_hash_map.h" |
||||
#include "absl/random/random.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/channelz/channelz.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/config.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/pending_connection.h" |
||||
#include "src/core/handshaker/handshaker.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/iomgr/closure.h" |
||||
#include "src/core/lib/iomgr/error.h" |
||||
#include "src/core/lib/iomgr/iomgr_fwd.h" |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/inter_activity_latch.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/server/server.h" |
||||
#include "src/core/util/sync.h" |
||||
#include "src/core/util/time.h" |
||||
|
||||
// Channel arg: integer number of data connections to specify
|
||||
// Defaults to 1 if not set
|
||||
#define GRPC_ARG_CHAOTIC_GOOD_DATA_CONNECTIONS \ |
||||
"grpc.chaotic_good.data_connections" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
class ChaoticGoodServerListener final : public Server::ListenerInterface { |
||||
public: |
||||
static absl::AnyInvocable<std::string()> DefaultConnectionIDGenerator() { |
||||
return [bitgen = absl::BitGen()]() mutable { |
||||
return absl::StrCat(absl::Hex(absl::Uniform<uint64_t>(bitgen))); |
||||
}; |
||||
} |
||||
|
||||
explicit ChaoticGoodServerListener( |
||||
Server* server, const ChannelArgs& args, |
||||
absl::AnyInvocable<std::string()> connection_id_generator = |
||||
DefaultConnectionIDGenerator()); |
||||
~ChaoticGoodServerListener() override; |
||||
// Bind address to EventEngine listener.
|
||||
absl::StatusOr<int> Bind( |
||||
grpc_event_engine::experimental::EventEngine::ResolvedAddress addr); |
||||
absl::Status StartListening(); |
||||
const ChannelArgs& args() const { return args_; } |
||||
void Orphan() override; |
||||
|
||||
class ActiveConnection : public InternallyRefCounted<ActiveConnection> { |
||||
public: |
||||
ActiveConnection( |
||||
RefCountedPtr<ChaoticGoodServerListener> listener, |
||||
std::unique_ptr<grpc_event_engine::experimental::EventEngine::Endpoint> |
||||
endpoint); |
||||
~ActiveConnection() override; |
||||
const ChannelArgs& args() const { return listener_->args(); } |
||||
|
||||
void Orphan() override; |
||||
|
||||
class HandshakingState : public RefCounted<HandshakingState> { |
||||
public: |
||||
explicit HandshakingState(RefCountedPtr<ActiveConnection> connection); |
||||
~HandshakingState() override {}; |
||||
void Start(std::unique_ptr< |
||||
grpc_event_engine::experimental::EventEngine::Endpoint> |
||||
endpoint); |
||||
|
||||
void Shutdown() { |
||||
handshake_mgr_->Shutdown(absl::CancelledError("Shutdown")); |
||||
} |
||||
|
||||
private: |
||||
struct DataConnection { |
||||
explicit DataConnection(std::string connection_id) |
||||
: connection_id(std::move(connection_id)) {} |
||||
std::string connection_id; |
||||
}; |
||||
struct ControlConnection { |
||||
explicit ControlConnection(Config config) : config(std::move(config)) {} |
||||
Config config; |
||||
}; |
||||
|
||||
static auto EndpointReadSettingsFrame( |
||||
RefCountedPtr<HandshakingState> self); |
||||
static auto EndpointWriteSettingsFrame( |
||||
RefCountedPtr<HandshakingState> self, bool is_control_endpoint); |
||||
static auto ControlEndpointWriteSettingsFrame( |
||||
RefCountedPtr<HandshakingState> self); |
||||
static auto DataEndpointWriteSettingsFrame( |
||||
RefCountedPtr<HandshakingState> self); |
||||
|
||||
void OnHandshakeDone(absl::StatusOr<HandshakerArgs*> result); |
||||
const RefCountedPtr<ActiveConnection> connection_; |
||||
const RefCountedPtr<HandshakeManager> handshake_mgr_; |
||||
std::variant<std::monostate, DataConnection, ControlConnection> data_; |
||||
}; |
||||
|
||||
private: |
||||
void Done(); |
||||
RefCountedPtr<Arena> arena_ = SimpleArenaAllocator()->MakeArena(); |
||||
const RefCountedPtr<ChaoticGoodServerListener> listener_; |
||||
RefCountedPtr<HandshakingState> handshaking_state_; |
||||
Mutex mu_; |
||||
ActivityPtr receive_settings_activity_ ABSL_GUARDED_BY(mu_); |
||||
bool orphaned_ ABSL_GUARDED_BY(mu_) = false; |
||||
PromiseEndpoint endpoint_; |
||||
absl::BitGen bitgen_; |
||||
}; |
||||
|
||||
class DataConnectionListener final : public ServerConnectionFactory { |
||||
public: |
||||
DataConnectionListener( |
||||
absl::AnyInvocable<std::string()> connection_id_generator, |
||||
Duration connect_timeout, |
||||
std::shared_ptr<grpc_event_engine::experimental::EventEngine> |
||||
event_engine); |
||||
~DataConnectionListener() override { CHECK(shutdown_); } |
||||
|
||||
void Orphaned() override; |
||||
|
||||
PendingConnection RequestDataConnection() override; |
||||
void FinishDataConnection(absl::string_view id, PromiseEndpoint endpoint); |
||||
Duration connection_timeout() const { return connect_timeout_; } |
||||
|
||||
private: |
||||
using PromiseEndpointLatch = |
||||
InterActivityLatch<absl::StatusOr<PromiseEndpoint>>; |
||||
using PromiseEndpointLatchPtr = std::shared_ptr<PromiseEndpointLatch>; |
||||
struct PendingConnectionInfo { |
||||
PromiseEndpointLatchPtr latch; |
||||
grpc_event_engine::experimental::EventEngine::TaskHandle timeout; |
||||
}; |
||||
|
||||
void ConnectionTimeout(absl::string_view id); |
||||
PromiseEndpointLatchPtr Extract(absl::string_view id); |
||||
|
||||
Mutex mu_; |
||||
absl::flat_hash_map<std::string, PendingConnectionInfo> pending_connections_ |
||||
ABSL_GUARDED_BY(mu_); |
||||
absl::AnyInvocable<std::string()> connection_id_generator_ |
||||
ABSL_GUARDED_BY(mu_); |
||||
const std::shared_ptr<grpc_event_engine::experimental::EventEngine> |
||||
event_engine_; |
||||
const Duration connect_timeout_; |
||||
bool shutdown_ ABSL_GUARDED_BY(mu_) = false; |
||||
}; |
||||
|
||||
void Start() override { StartListening().IgnoreError(); }; |
||||
|
||||
channelz::ListenSocketNode* channelz_listen_socket_node() const override { |
||||
return nullptr; |
||||
} |
||||
|
||||
void SetServerListenerState(RefCountedPtr<Server::ListenerState>) override {} |
||||
|
||||
const grpc_resolved_address* resolved_address() const override { |
||||
// chaotic good doesn't use the new ListenerState interface yet.
|
||||
Crash("Unimplemented"); |
||||
return nullptr; |
||||
} |
||||
|
||||
void SetOnDestroyDone(grpc_closure* closure) override { |
||||
MutexLock lock(&mu_); |
||||
on_destroy_done_ = closure; |
||||
}; |
||||
|
||||
private: |
||||
Server* const server_; |
||||
ChannelArgs args_; |
||||
std::shared_ptr<grpc_event_engine::experimental::EventEngine> event_engine_; |
||||
std::unique_ptr<grpc_event_engine::experimental::EventEngine::Listener> |
||||
ee_listener_; |
||||
Mutex mu_; |
||||
bool shutdown_ ABSL_GUARDED_BY(mu_) = false; |
||||
absl::flat_hash_set<OrphanablePtr<ActiveConnection>> connection_list_ |
||||
ABSL_GUARDED_BY(mu_); |
||||
grpc_closure* on_destroy_done_ ABSL_GUARDED_BY(mu_) = nullptr; |
||||
const RefCountedPtr<DataConnectionListener> data_connection_listener_; |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int grpc_server_add_chaotic_good_legacy_port(grpc_server* server, |
||||
const char* addr); |
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_SERVER_CHAOTIC_GOOD_SERVER_H
|
@ -0,0 +1,503 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/server_transport.h" |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/slice.h> |
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <memory> |
||||
#include <string> |
||||
#include <tuple> |
||||
|
||||
#include "absl/cleanup/cleanup.h" |
||||
#include "absl/log/check.h" |
||||
#include "absl/log/log.h" |
||||
#include "absl/random/bit_gen_ref.h" |
||||
#include "absl/random/random.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/chaotic_good_transport.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/iomgr/exec_ctx.h" |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/event_engine_wakeup_scheduler.h" |
||||
#include "src/core/lib/promise/for_each.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/switch.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
auto ChaoticGoodServerTransport::PushFrameIntoCall(RefCountedPtr<Stream> stream, |
||||
MessageFrame frame) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: PushFrameIntoCall: frame=" << frame.ToString(); |
||||
return stream->message_reassembly.PushFrameInto(std::move(frame), |
||||
stream->call); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::PushFrameIntoCall(RefCountedPtr<Stream> stream, |
||||
BeginMessageFrame frame) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: PushFrameIntoCall: frame=" << frame.ToString(); |
||||
return stream->message_reassembly.PushFrameInto(std::move(frame), |
||||
stream->call); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::PushFrameIntoCall(RefCountedPtr<Stream> stream, |
||||
MessageChunkFrame frame) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: PushFrameIntoCall: frame=" << frame.ToString(); |
||||
return stream->message_reassembly.PushFrameInto(std::move(frame), |
||||
stream->call); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::PushFrameIntoCall(RefCountedPtr<Stream> stream, |
||||
ClientEndOfStream) { |
||||
if (stream->message_reassembly.in_message_boundary()) { |
||||
stream->call.FinishSends(); |
||||
// Note that we cannot remove from the stream map yet, as we
|
||||
// may yet receive a cancellation.
|
||||
return Immediate(StatusFlag{Success{}}); |
||||
} else { |
||||
stream->message_reassembly.FailCall( |
||||
stream->call, "Received end of stream before end of chunked message"); |
||||
return Immediate(StatusFlag{Failure{}}); |
||||
} |
||||
} |
||||
|
||||
template <typename T> |
||||
auto ChaoticGoodServerTransport::DispatchFrame( |
||||
RefCountedPtr<ChaoticGoodTransport> transport, IncomingFrame frame) { |
||||
auto stream = LookupStream(frame.header().stream_id); |
||||
return If( |
||||
stream != nullptr, |
||||
[this, &stream, &frame, &transport]() { |
||||
// TODO(ctiller): instead of SpawnWaitable here we probably want a
|
||||
// small queue to push into, so that the call can proceed
|
||||
// asynchronously to other calls regardless of frame ordering.
|
||||
return stream->call.SpawnWaitable( |
||||
"push-frame", [this, stream, frame = std::move(frame), |
||||
transport = std::move(transport)]() mutable { |
||||
auto& call = stream->call; |
||||
return call.UntilCallCompletes(TrySeq( |
||||
frame.Payload(), |
||||
[transport = std::move(transport), |
||||
header = frame.header()](SliceBuffer payload) { |
||||
return transport->DeserializeFrame<T>(header, |
||||
std::move(payload)); |
||||
}, |
||||
[stream = std::move(stream), this](T frame) mutable { |
||||
auto& call = stream->call; |
||||
return Map(call.CancelIfFails(PushFrameIntoCall( |
||||
std::move(stream), std::move(frame))), |
||||
[](auto) { return absl::OkStatus(); }); |
||||
})); |
||||
}); |
||||
}, |
||||
[]() { return absl::OkStatus(); }); |
||||
} |
||||
|
||||
namespace { |
||||
auto BooleanSuccessToTransportErrorCapturingInitiator(CallInitiator initiator) { |
||||
return [initiator = std::move(initiator)](bool success) { |
||||
return success ? absl::OkStatus() |
||||
: absl::UnavailableError("Transport closed."); |
||||
}; |
||||
} |
||||
} // namespace
|
||||
|
||||
auto ChaoticGoodServerTransport::SendFrame( |
||||
ServerFrame frame, MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator) { |
||||
// Capture the call_initiator to ensure the underlying call spine is alive
|
||||
// until the outgoing_frames.Send promise completes.
|
||||
return Map(outgoing_frames.Send(std::move(frame)), |
||||
BooleanSuccessToTransportErrorCapturingInitiator( |
||||
std::move(call_initiator))); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::SendFrameAcked( |
||||
ServerFrame frame, MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator) { |
||||
// Capture the call_initiator to ensure the underlying call spine is alive
|
||||
// until the outgoing_frames.Send promise completes.
|
||||
return Map(outgoing_frames.SendAcked(std::move(frame)), |
||||
BooleanSuccessToTransportErrorCapturingInitiator( |
||||
std::move(call_initiator))); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::SendCallBody( |
||||
uint32_t stream_id, MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator) { |
||||
// Continuously send client frame with client to server messages.
|
||||
return ForEach(MessagesFrom(call_initiator), |
||||
[this, stream_id, outgoing_frames = std::move(outgoing_frames), |
||||
call_initiator](MessageHandle message) mutable { |
||||
return Map(message_chunker_.Send(std::move(message), |
||||
stream_id, outgoing_frames), |
||||
BooleanSuccessToTransportErrorCapturingInitiator( |
||||
std::move(call_initiator))); |
||||
}); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::SendCallInitialMetadataAndBody( |
||||
uint32_t stream_id, MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator) { |
||||
return TrySeq( |
||||
// Wait for initial metadata then send it out.
|
||||
call_initiator.PullServerInitialMetadata(), |
||||
[stream_id, outgoing_frames, call_initiator, |
||||
this](std::optional<ServerMetadataHandle> md) mutable { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: SendCallInitialMetadataAndBody: md=" |
||||
<< (md.has_value() ? (*md)->DebugString() : "null"); |
||||
return If( |
||||
md.has_value(), |
||||
[&md, stream_id, &outgoing_frames, &call_initiator, this]() { |
||||
ServerInitialMetadataFrame frame; |
||||
frame.body = ServerMetadataProtoFromGrpc(**md); |
||||
frame.stream_id = stream_id; |
||||
return TrySeq( |
||||
SendFrame(std::move(frame), outgoing_frames, call_initiator), |
||||
SendCallBody(stream_id, outgoing_frames, call_initiator)); |
||||
}, |
||||
[]() { return absl::OkStatus(); }); |
||||
}); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::CallOutboundLoop( |
||||
uint32_t stream_id, CallInitiator call_initiator) { |
||||
auto outgoing_frames = outgoing_frames_.MakeSender(); |
||||
return GRPC_LATENT_SEE_PROMISE( |
||||
"CallOutboundLoop", |
||||
Seq(Map(SendCallInitialMetadataAndBody(stream_id, outgoing_frames, |
||||
call_initiator), |
||||
[stream_id](absl::Status main_body_result) { |
||||
GRPC_TRACE_VLOG(chaotic_good, 2) |
||||
<< "CHAOTIC_GOOD: CallOutboundLoop: stream_id=" << stream_id |
||||
<< " main_body_result=" << main_body_result; |
||||
return Empty{}; |
||||
}), |
||||
call_initiator.PullServerTrailingMetadata(), |
||||
// Capture the call_initiator to ensure the underlying call_spine
|
||||
// is alive until the SendFragment promise completes.
|
||||
[this, stream_id, outgoing_frames, |
||||
call_initiator](ServerMetadataHandle md) mutable { |
||||
ServerTrailingMetadataFrame frame; |
||||
frame.body = ServerMetadataProtoFromGrpc(*md); |
||||
frame.stream_id = stream_id; |
||||
return SendFrame(std::move(frame), outgoing_frames, call_initiator); |
||||
})); |
||||
} |
||||
|
||||
absl::Status ChaoticGoodServerTransport::NewStream( |
||||
ChaoticGoodTransport& transport, const FrameHeader& header, |
||||
SliceBuffer payload) { |
||||
CHECK_EQ(header.payload_length, payload.Length()); |
||||
auto client_initial_metadata_frame = |
||||
transport.DeserializeFrame<ClientInitialMetadataFrame>( |
||||
header, std::move(payload)); |
||||
if (!client_initial_metadata_frame.ok()) { |
||||
return client_initial_metadata_frame.status(); |
||||
} |
||||
auto md = ClientMetadataGrpcFromProto(client_initial_metadata_frame->body); |
||||
if (!md.ok()) { |
||||
return md.status(); |
||||
} |
||||
RefCountedPtr<Arena> arena(call_arena_allocator_->MakeArena()); |
||||
arena->SetContext<grpc_event_engine::experimental::EventEngine>( |
||||
event_engine_.get()); |
||||
std::optional<CallInitiator> call_initiator; |
||||
auto call = MakeCallPair(std::move(*md), std::move(arena)); |
||||
call_initiator.emplace(std::move(call.initiator)); |
||||
const auto stream_id = client_initial_metadata_frame->stream_id; |
||||
auto add_result = NewStream(stream_id, *call_initiator); |
||||
if (!add_result.ok()) { |
||||
call_initiator.reset(); |
||||
return add_result; |
||||
} |
||||
call_initiator->SpawnGuarded( |
||||
"server-write", [this, stream_id, call_initiator = *call_initiator, |
||||
call_handler = std::move(call.handler)]() mutable { |
||||
call_destination_->StartCall(std::move(call_handler)); |
||||
return CallOutboundLoop(stream_id, call_initiator); |
||||
}); |
||||
return absl::OkStatus(); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::ReadOneFrame( |
||||
RefCountedPtr<ChaoticGoodTransport> transport) { |
||||
return GRPC_LATENT_SEE_PROMISE( |
||||
"ReadOneFrame", |
||||
TrySeq( |
||||
transport->ReadFrameBytes(), |
||||
[this, transport](IncomingFrame incoming_frame) mutable { |
||||
return Switch( |
||||
incoming_frame.header().type, |
||||
Case<FrameType::kClientInitialMetadata>([&, this]() { |
||||
return TrySeq(incoming_frame.Payload(), |
||||
[this, transport = std::move(transport), |
||||
header = incoming_frame.header()]( |
||||
SliceBuffer payload) mutable { |
||||
return NewStream(*transport, header, |
||||
std::move(payload)); |
||||
}); |
||||
}), |
||||
Case<FrameType::kMessage>([&, this]() mutable { |
||||
return DispatchFrame<MessageFrame>(std::move(transport), |
||||
std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kBeginMessage>([&, this]() mutable { |
||||
return DispatchFrame<BeginMessageFrame>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kMessageChunk>([&, this]() mutable { |
||||
return DispatchFrame<MessageChunkFrame>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kClientEndOfStream>([&, this]() mutable { |
||||
return DispatchFrame<ClientEndOfStream>( |
||||
std::move(transport), std::move(incoming_frame)); |
||||
}), |
||||
Case<FrameType::kCancel>([&, this]() { |
||||
auto stream = |
||||
ExtractStream(incoming_frame.header().stream_id); |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "Cancel stream " << incoming_frame.header().stream_id |
||||
<< (stream != nullptr ? " (active)" : " (not found)"); |
||||
return If( |
||||
stream != nullptr, |
||||
[&stream]() { |
||||
auto c = std::move(stream->call); |
||||
return c.SpawnWaitable("cancel", [c]() mutable { |
||||
c.Cancel(); |
||||
return absl::OkStatus(); |
||||
}); |
||||
}, |
||||
[]() -> absl::Status { return absl::OkStatus(); }); |
||||
}), |
||||
Default([&]() { |
||||
LOG_EVERY_N_SEC(INFO, 10) |
||||
<< "Bad frame type: " |
||||
<< incoming_frame.header().ToString(); |
||||
return ImmediateOkStatus(); |
||||
})); |
||||
}, |
||||
[]() -> LoopCtl<absl::Status> { return Continue{}; })); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::TransportReadLoop( |
||||
RefCountedPtr<ChaoticGoodTransport> transport) { |
||||
return Seq(got_acceptor_.Wait(), |
||||
Loop([this, transport = std::move(transport)]() mutable { |
||||
return ReadOneFrame(transport); |
||||
})); |
||||
} |
||||
|
||||
auto ChaoticGoodServerTransport::OnTransportActivityDone( |
||||
absl::string_view activity) { |
||||
return [self = RefAsSubclass<ChaoticGoodServerTransport>(), |
||||
activity](absl::Status status) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD: OnTransportActivityDone: activity=" << activity |
||||
<< " status=" << status; |
||||
self->AbortWithError(); |
||||
}; |
||||
} |
||||
|
||||
ChaoticGoodServerTransport::ChaoticGoodServerTransport( |
||||
const ChannelArgs& args, PromiseEndpoint control_endpoint, Config config, |
||||
RefCountedPtr<ServerConnectionFactory>) |
||||
: call_arena_allocator_(MakeRefCounted<CallArenaAllocator>( |
||||
args.GetObject<ResourceQuota>() |
||||
->memory_quota() |
||||
->CreateMemoryAllocator("chaotic-good"), |
||||
1024)), |
||||
event_engine_( |
||||
args.GetObjectRef<grpc_event_engine::experimental::EventEngine>()), |
||||
outgoing_frames_(4), |
||||
message_chunker_(config.MakeMessageChunker()) { |
||||
auto transport = MakeRefCounted<ChaoticGoodTransport>( |
||||
std::move(control_endpoint), config.TakePendingDataEndpoints(), |
||||
event_engine_, config.MakeTransportOptions(), false); |
||||
auto party_arena = SimpleArenaAllocator(0)->MakeArena(); |
||||
party_arena->SetContext<grpc_event_engine::experimental::EventEngine>( |
||||
event_engine_.get()); |
||||
party_ = Party::Make(std::move(party_arena)); |
||||
party_->Spawn( |
||||
"server-chaotic-writer", |
||||
GRPC_LATENT_SEE_PROMISE("ServerTransportWriteLoop", |
||||
transport->TransportWriteLoop(outgoing_frames_)), |
||||
OnTransportActivityDone("writer")); |
||||
party_->Spawn("server-chaotic-reader", |
||||
GRPC_LATENT_SEE_PROMISE("ServerTransportReadLoop", |
||||
TransportReadLoop(transport)), |
||||
OnTransportActivityDone("reader")); |
||||
} |
||||
|
||||
void ChaoticGoodServerTransport::SetCallDestination( |
||||
RefCountedPtr<UnstartedCallDestination> call_destination) { |
||||
CHECK(call_destination_ == nullptr); |
||||
CHECK(call_destination != nullptr); |
||||
call_destination_ = call_destination; |
||||
got_acceptor_.Set(); |
||||
} |
||||
|
||||
void ChaoticGoodServerTransport::Orphan() { |
||||
AbortWithError(); |
||||
RefCountedPtr<Party> party; |
||||
{ |
||||
MutexLock lock(&mu_); |
||||
party = std::move(party_); |
||||
} |
||||
party.reset(); |
||||
Unref(); |
||||
} |
||||
|
||||
void ChaoticGoodServerTransport::AbortWithError() { |
||||
// Mark transport as unavailable when the endpoint write/read failed.
|
||||
// Close all the available pipes.
|
||||
outgoing_frames_.MarkClosed(); |
||||
ReleasableMutexLock lock(&mu_); |
||||
aborted_with_error_ = true; |
||||
StreamMap stream_map = std::move(stream_map_); |
||||
stream_map_.clear(); |
||||
state_tracker_.SetState(GRPC_CHANNEL_SHUTDOWN, |
||||
absl::UnavailableError("transport closed"), |
||||
"transport closed"); |
||||
lock.Release(); |
||||
for (auto& pair : stream_map) { |
||||
auto stream = std::move(pair.second); |
||||
auto& call = stream->call; |
||||
call.SpawnInfallible("cancel", [stream = std::move(stream)]() mutable { |
||||
stream->call.Cancel(); |
||||
}); |
||||
} |
||||
} |
||||
|
||||
RefCountedPtr<ChaoticGoodServerTransport::Stream> |
||||
ChaoticGoodServerTransport::LookupStream(uint32_t stream_id) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD " << this << " LookupStream " << stream_id; |
||||
MutexLock lock(&mu_); |
||||
auto it = stream_map_.find(stream_id); |
||||
if (it == stream_map_.end()) return nullptr; |
||||
return it->second; |
||||
} |
||||
|
||||
RefCountedPtr<ChaoticGoodServerTransport::Stream> |
||||
ChaoticGoodServerTransport::ExtractStream(uint32_t stream_id) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD " << this << " ExtractStream " << stream_id; |
||||
MutexLock lock(&mu_); |
||||
auto it = stream_map_.find(stream_id); |
||||
if (it == stream_map_.end()) return nullptr; |
||||
auto r = std::move(it->second); |
||||
stream_map_.erase(it); |
||||
return r; |
||||
} |
||||
|
||||
absl::Status ChaoticGoodServerTransport::NewStream( |
||||
uint32_t stream_id, CallInitiator call_initiator) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD " << this << " NewStream " << stream_id; |
||||
MutexLock lock(&mu_); |
||||
if (aborted_with_error_) { |
||||
return absl::UnavailableError("Transport closed"); |
||||
} |
||||
auto it = stream_map_.find(stream_id); |
||||
if (it != stream_map_.end()) { |
||||
return absl::InternalError("Stream already exists"); |
||||
} |
||||
if (stream_id <= last_seen_new_stream_id_) { |
||||
return absl::InternalError("Stream id is not increasing"); |
||||
} |
||||
const bool on_done_added = call_initiator.OnDone( |
||||
[self = RefAsSubclass<ChaoticGoodServerTransport>(), stream_id](bool) { |
||||
GRPC_TRACE_LOG(chaotic_good, INFO) |
||||
<< "CHAOTIC_GOOD " << self.get() << " OnDone " << stream_id; |
||||
auto stream = self->ExtractStream(stream_id); |
||||
if (stream != nullptr) { |
||||
auto& call = stream->call; |
||||
call.SpawnInfallible("cancel", |
||||
[stream = std::move(stream)]() mutable { |
||||
stream->call.Cancel(); |
||||
}); |
||||
} |
||||
}); |
||||
if (!on_done_added) { |
||||
return absl::CancelledError(); |
||||
} |
||||
stream_map_.emplace(stream_id, |
||||
MakeRefCounted<Stream>(std::move(call_initiator))); |
||||
return absl::OkStatus(); |
||||
} |
||||
|
||||
void ChaoticGoodServerTransport::PerformOp(grpc_transport_op* op) { |
||||
RefCountedPtr<Party> cancelled_party; |
||||
bool close_outgoing_frames = false; |
||||
auto cleanup = absl::MakeCleanup([&close_outgoing_frames, this]() { |
||||
if (close_outgoing_frames) { |
||||
outgoing_frames_.MarkClosed(); |
||||
} |
||||
}); |
||||
MutexLock lock(&mu_); |
||||
bool did_stuff = false; |
||||
if (op->start_connectivity_watch != nullptr) { |
||||
state_tracker_.AddWatcher(op->start_connectivity_watch_state, |
||||
std::move(op->start_connectivity_watch)); |
||||
did_stuff = true; |
||||
} |
||||
if (op->stop_connectivity_watch != nullptr) { |
||||
state_tracker_.RemoveWatcher(op->stop_connectivity_watch); |
||||
did_stuff = true; |
||||
} |
||||
if (op->set_accept_stream) { |
||||
if (op->set_accept_stream_fn != nullptr) { |
||||
Crash(absl::StrCat( |
||||
"set_accept_stream not supported on chaotic good transports: ", |
||||
grpc_transport_op_string(op))); |
||||
} |
||||
did_stuff = true; |
||||
} |
||||
if (!op->goaway_error.ok() || !op->disconnect_with_error.ok()) { |
||||
cancelled_party = std::move(party_); |
||||
close_outgoing_frames = true; |
||||
state_tracker_.SetState(GRPC_CHANNEL_SHUTDOWN, |
||||
absl::UnavailableError("transport closed"), |
||||
"transport closed"); |
||||
did_stuff = true; |
||||
} |
||||
if (!did_stuff) { |
||||
Crash(absl::StrCat("unimplemented transport perform op: ", |
||||
grpc_transport_op_string(op))); |
||||
} |
||||
ExecCtx::Run(DEBUG_LOCATION, op->on_consumed, absl::OkStatus()); |
||||
} |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,158 @@ |
||||
// Copyright 2022 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_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_SERVER_TRANSPORT_H |
||||
#define GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_SERVER_TRANSPORT_H |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/event_engine/memory_allocator.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/slice.h> |
||||
#include <grpc/support/port_platform.h> |
||||
#include <stdint.h> |
||||
#include <stdio.h> |
||||
|
||||
#include <cstdint> |
||||
#include <initializer_list> // IWYU pragma: keep |
||||
#include <iostream> |
||||
#include <map> |
||||
#include <memory> |
||||
#include <optional> |
||||
#include <string> |
||||
#include <tuple> |
||||
#include <type_traits> |
||||
#include <utility> |
||||
#include <variant> |
||||
|
||||
#include "absl/base/thread_annotations.h" |
||||
#include "absl/container/flat_hash_map.h" |
||||
#include "absl/functional/any_invocable.h" |
||||
#include "absl/random/random.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/chaotic_good_transport.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/config.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/message_reassembly.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/pending_connection.h" |
||||
#include "src/core/lib/event_engine/default_event_engine.h" // IWYU pragma: keep |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/context.h" |
||||
#include "src/core/lib/promise/if.h" |
||||
#include "src/core/lib/promise/inter_activity_latch.h" |
||||
#include "src/core/lib/promise/inter_activity_pipe.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/mpsc.h" |
||||
#include "src/core/lib/promise/party.h" |
||||
#include "src/core/lib/promise/pipe.h" |
||||
#include "src/core/lib/promise/poll.h" |
||||
#include "src/core/lib/promise/seq.h" |
||||
#include "src/core/lib/promise/try_join.h" |
||||
#include "src/core/lib/promise/try_seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/slice/slice.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/slice/slice_internal.h" |
||||
#include "src/core/lib/transport/metadata_batch.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/lib/transport/transport.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
#include "src/core/util/sync.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
class ChaoticGoodServerTransport final : public ServerTransport { |
||||
public: |
||||
ChaoticGoodServerTransport(const ChannelArgs& args, |
||||
PromiseEndpoint control_endpoint, Config config, |
||||
RefCountedPtr<ServerConnectionFactory>); |
||||
|
||||
FilterStackTransport* filter_stack_transport() override { return nullptr; } |
||||
ClientTransport* client_transport() override { return nullptr; } |
||||
ServerTransport* server_transport() override { return this; } |
||||
absl::string_view GetTransportName() const override { return "chaotic_good"; } |
||||
void SetPollset(grpc_stream*, grpc_pollset*) override {} |
||||
void SetPollsetSet(grpc_stream*, grpc_pollset_set*) override {} |
||||
void PerformOp(grpc_transport_op*) override; |
||||
void Orphan() override; |
||||
|
||||
void SetCallDestination( |
||||
RefCountedPtr<UnstartedCallDestination> call_destination) override; |
||||
void AbortWithError(); |
||||
|
||||
private: |
||||
struct Stream : public RefCounted<Stream> { |
||||
explicit Stream(CallInitiator call) : call(std::move(call)) {} |
||||
CallInitiator call; |
||||
MessageReassembly message_reassembly; |
||||
}; |
||||
using StreamMap = absl::flat_hash_map<uint32_t, RefCountedPtr<Stream>>; |
||||
|
||||
absl::Status NewStream(uint32_t stream_id, CallInitiator call_initiator); |
||||
RefCountedPtr<Stream> LookupStream(uint32_t stream_id); |
||||
RefCountedPtr<Stream> ExtractStream(uint32_t stream_id); |
||||
auto SendCallInitialMetadataAndBody(uint32_t stream_id, |
||||
MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator); |
||||
auto SendCallBody(uint32_t stream_id, MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator); |
||||
auto CallOutboundLoop(uint32_t stream_id, CallInitiator call_initiator); |
||||
auto OnTransportActivityDone(absl::string_view activity); |
||||
auto TransportReadLoop(RefCountedPtr<ChaoticGoodTransport> transport); |
||||
auto ReadOneFrame(RefCountedPtr<ChaoticGoodTransport> transport); |
||||
// Read different parts of the server frame from control/data endpoints
|
||||
// based on frame header.
|
||||
// Resolves to a StatusOr<tuple<SliceBuffer, SliceBuffer>>
|
||||
auto ReadFrameBody(Slice read_buffer); |
||||
void SendCancel(uint32_t stream_id, absl::Status why); |
||||
absl::Status NewStream(ChaoticGoodTransport& transport, |
||||
const FrameHeader& header, |
||||
SliceBuffer initial_metadata_payload); |
||||
template <typename T> |
||||
auto DispatchFrame(RefCountedPtr<ChaoticGoodTransport> transport, |
||||
IncomingFrame frame); |
||||
auto PushFrameIntoCall(RefCountedPtr<Stream> stream, MessageFrame frame); |
||||
auto PushFrameIntoCall(RefCountedPtr<Stream> stream, ClientEndOfStream frame); |
||||
auto PushFrameIntoCall(RefCountedPtr<Stream> stream, BeginMessageFrame frame); |
||||
auto PushFrameIntoCall(RefCountedPtr<Stream> stream, MessageChunkFrame frame); |
||||
auto SendFrame(ServerFrame frame, MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator); |
||||
auto SendFrameAcked(ServerFrame frame, |
||||
MpscSender<ServerFrame> outgoing_frames, |
||||
CallInitiator call_initiator); |
||||
|
||||
RefCountedPtr<UnstartedCallDestination> call_destination_; |
||||
const RefCountedPtr<CallArenaAllocator> call_arena_allocator_; |
||||
const std::shared_ptr<grpc_event_engine::experimental::EventEngine> |
||||
event_engine_; |
||||
InterActivityLatch<void> got_acceptor_; |
||||
MpscReceiver<ServerFrame> outgoing_frames_; |
||||
Mutex mu_; |
||||
// Map of stream incoming server frames, key is stream_id.
|
||||
StreamMap stream_map_ ABSL_GUARDED_BY(mu_); |
||||
bool aborted_with_error_ ABSL_GUARDED_BY(mu_) = false; |
||||
uint32_t last_seen_new_stream_id_ = 0; |
||||
RefCountedPtr<Party> party_; |
||||
ConnectivityStateTracker state_tracker_ ABSL_GUARDED_BY(mu_){ |
||||
"chaotic_good_server", GRPC_CHANNEL_READY}; |
||||
MessageChunker message_chunker_; |
||||
}; |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_SRC_CORE_EXT_TRANSPORT_CHAOTIC_GOOD_LEGACY_SERVER_TRANSPORT_H
|
@ -0,0 +1,314 @@ |
||||
# Copyright 2021 gRPC authors. |
||||
# |
||||
# Licensed under the Apache License, Version 2.0 (the "License"); |
||||
# you may not use this file except in compliance with the License. |
||||
# You may obtain a copy of the License at |
||||
# |
||||
# http://www.apache.org/licenses/LICENSE-2.0 |
||||
# |
||||
# Unless required by applicable law or agreed to in writing, software |
||||
# distributed under the License is distributed on an "AS IS" BASIS, |
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
# See the License for the specific language governing permissions and |
||||
# limitations under the License. |
||||
|
||||
load("//bazel:grpc_build_system.bzl", "grpc_cc_library", "grpc_cc_proto_library", "grpc_cc_test", "grpc_internal_proto_library", "grpc_package") |
||||
load("//test/core/test_util:grpc_fuzzer.bzl", "grpc_fuzz_test") |
||||
|
||||
licenses(["notice"]) |
||||
|
||||
grpc_package( |
||||
name = "test/core/transport/chaotic_good_legacy", |
||||
visibility = "tests", |
||||
) |
||||
|
||||
grpc_cc_library( |
||||
name = "transport_test", |
||||
testonly = 1, |
||||
srcs = ["transport_test.cc"], |
||||
hdrs = ["transport_test.h"], |
||||
external_deps = ["gtest"], |
||||
deps = [ |
||||
"//:iomgr_timer", |
||||
"//src/core:chaotic_good_legacy_frame", |
||||
"//src/core:memory_quota", |
||||
"//src/core:resource_quota", |
||||
"//test/core/event_engine/fuzzing_event_engine", |
||||
"//test/core/event_engine/fuzzing_event_engine:fuzzing_event_engine_cc_proto", |
||||
], |
||||
) |
||||
|
||||
grpc_cc_test( |
||||
name = "frame_header_test", |
||||
srcs = ["frame_header_test.cc"], |
||||
external_deps = [ |
||||
"absl/status", |
||||
"gtest", |
||||
], |
||||
deps = ["//src/core:chaotic_good_legacy_frame_header"], |
||||
) |
||||
|
||||
grpc_fuzz_test( |
||||
name = "frame_header_fuzzer", |
||||
srcs = ["frame_header_fuzzer.cc"], |
||||
external_deps = [ |
||||
"absl/status:statusor", |
||||
"absl/strings", |
||||
"gtest", |
||||
"fuzztest", |
||||
"fuzztest_main", |
||||
], |
||||
tags = ["no_windows"], |
||||
deps = [ |
||||
"//:grpc", |
||||
"//src/core:chaotic_good_legacy_frame_header", |
||||
], |
||||
) |
||||
|
||||
grpc_cc_test( |
||||
name = "frame_test", |
||||
srcs = ["frame_test.cc"], |
||||
external_deps = [ |
||||
"absl/log:check", |
||||
"absl/random", |
||||
"absl/status", |
||||
"absl/status:statusor", |
||||
"gtest", |
||||
], |
||||
deps = [ |
||||
"//:channel_create", |
||||
"//src/core:chaotic_good_legacy_frame", |
||||
], |
||||
) |
||||
|
||||
grpc_internal_proto_library( |
||||
name = "frame_fuzzer_proto", |
||||
srcs = ["frame_fuzzer.proto"], |
||||
) |
||||
|
||||
grpc_cc_proto_library( |
||||
name = "frame_fuzzer_cc_proto", |
||||
deps = ["frame_fuzzer_proto"], |
||||
) |
||||
|
||||
grpc_fuzz_test( |
||||
name = "frame_fuzzer", |
||||
srcs = ["frame_fuzzer.cc"], |
||||
external_deps = [ |
||||
"absl/log:check", |
||||
"absl/log:log", |
||||
"absl/random:bit_gen_ref", |
||||
"absl/status:statusor", |
||||
"gtest", |
||||
"fuzztest", |
||||
"fuzztest_main", |
||||
], |
||||
tags = ["no_windows"], |
||||
deps = [ |
||||
"frame_fuzzer_cc_proto", |
||||
"//:exec_ctx", |
||||
"//:gpr", |
||||
"//:hpack_encoder", |
||||
"//:hpack_parser", |
||||
"//:ref_counted_ptr", |
||||
"//src/core:arena", |
||||
"//src/core:chaotic_good_legacy_frame", |
||||
"//src/core:chaotic_good_legacy_frame_header", |
||||
"//src/core:event_engine_memory_allocator", |
||||
"//src/core:memory_quota", |
||||
"//src/core:resource_quota", |
||||
"//src/core:slice", |
||||
"//src/core:slice_buffer", |
||||
"//test/core/promise:test_context", |
||||
], |
||||
) |
||||
|
||||
grpc_cc_test( |
||||
name = "client_transport_test", |
||||
srcs = ["client_transport_test.cc"], |
||||
external_deps = [ |
||||
"absl/functional:any_invocable", |
||||
"absl/status:statusor", |
||||
"absl/strings:str_format", |
||||
"gtest", |
||||
], |
||||
uses_event_engine = False, |
||||
uses_polling = False, |
||||
deps = [ |
||||
"transport_test", |
||||
"//:grpc", |
||||
"//:grpc_public_hdrs", |
||||
"//src/core:arena", |
||||
"//src/core:chaotic_good_legacy_client_transport", |
||||
"//src/core:if", |
||||
"//src/core:loop", |
||||
"//src/core:seq", |
||||
"//src/core:slice_buffer", |
||||
"//test/core/transport/util:mock_promise_endpoint", |
||||
], |
||||
) |
||||
|
||||
grpc_cc_test( |
||||
name = "client_transport_error_test", |
||||
srcs = ["client_transport_error_test.cc"], |
||||
external_deps = [ |
||||
"absl/functional:any_invocable", |
||||
"absl/status", |
||||
"absl/status:statusor", |
||||
"absl/strings:str_format", |
||||
"gtest", |
||||
], |
||||
uses_event_engine = False, |
||||
uses_polling = False, |
||||
deps = [ |
||||
"//:grpc_public_hdrs", |
||||
"//:grpc_unsecure", |
||||
"//:iomgr_timer", |
||||
"//:ref_counted_ptr", |
||||
"//src/core:activity", |
||||
"//src/core:arena", |
||||
"//src/core:chaotic_good_legacy_client_transport", |
||||
"//src/core:event_engine_wakeup_scheduler", |
||||
"//src/core:grpc_promise_endpoint", |
||||
"//src/core:if", |
||||
"//src/core:join", |
||||
"//src/core:loop", |
||||
"//src/core:memory_quota", |
||||
"//src/core:pipe", |
||||
"//src/core:resource_quota", |
||||
"//src/core:seq", |
||||
"//src/core:slice", |
||||
"//src/core:slice_buffer", |
||||
"//test/core/event_engine/fuzzing_event_engine", |
||||
"//test/core/event_engine/fuzzing_event_engine:fuzzing_event_engine_cc_proto", |
||||
], |
||||
) |
||||
|
||||
grpc_cc_test( |
||||
name = "server_transport_test", |
||||
srcs = ["server_transport_test.cc"], |
||||
external_deps = [ |
||||
"absl/functional:any_invocable", |
||||
"absl/status", |
||||
"absl/status:statusor", |
||||
"absl/strings:str_format", |
||||
"gtest", |
||||
], |
||||
uses_event_engine = False, |
||||
uses_polling = False, |
||||
deps = [ |
||||
"transport_test", |
||||
"//:grpc", |
||||
"//:grpc_public_hdrs", |
||||
"//:iomgr_timer", |
||||
"//:ref_counted_ptr", |
||||
"//src/core:arena", |
||||
"//src/core:chaotic_good_legacy_server_transport", |
||||
"//src/core:memory_quota", |
||||
"//src/core:resource_quota", |
||||
"//src/core:seq", |
||||
"//src/core:slice", |
||||
"//src/core:slice_buffer", |
||||
"//test/core/event_engine/fuzzing_event_engine", |
||||
"//test/core/event_engine/fuzzing_event_engine:fuzzing_event_engine_cc_proto", |
||||
"//test/core/transport/util:mock_promise_endpoint", |
||||
], |
||||
) |
||||
|
||||
grpc_cc_test( |
||||
name = "chaotic_good_server_test", |
||||
srcs = ["chaotic_good_server_test.cc"], |
||||
external_deps = [ |
||||
"absl/log:check", |
||||
"absl/log:log", |
||||
"absl/strings", |
||||
"absl/time", |
||||
"gtest", |
||||
], |
||||
tags = [ |
||||
"no_windows", |
||||
], |
||||
uses_event_engine = True, |
||||
uses_polling = False, |
||||
deps = [ |
||||
"//:grpc", |
||||
"//:grpc++", |
||||
"//:grpc_public_hdrs", |
||||
"//:parse_address", |
||||
"//:uri", |
||||
"//src/core:channel_args", |
||||
"//src/core:chaotic_good_legacy_connector", |
||||
"//src/core:chaotic_good_legacy_server", |
||||
"//src/core:notification", |
||||
"//src/core:resource_quota", |
||||
"//src/core:time", |
||||
"//test/core/event_engine:event_engine_test_utils", |
||||
"//test/core/test_util:grpc_test_util", |
||||
], |
||||
) |
||||
|
||||
grpc_fuzz_test( |
||||
name = "control_endpoint_test", |
||||
srcs = ["control_endpoint_test.cc"], |
||||
external_deps = [ |
||||
"fuzztest", |
||||
"fuzztest_main", |
||||
"gtest", |
||||
], |
||||
tags = [ |
||||
"no_windows", |
||||
], |
||||
deps = [ |
||||
"//src/core:chaotic_good_legacy_control_endpoint", |
||||
"//test/core/call/yodel:yodel_test", |
||||
"//test/core/transport/util:mock_promise_endpoint", |
||||
], |
||||
) |
||||
|
||||
grpc_fuzz_test( |
||||
name = "data_endpoints_test", |
||||
srcs = ["data_endpoints_test.cc"], |
||||
external_deps = [ |
||||
"fuzztest", |
||||
"fuzztest_main", |
||||
"gtest", |
||||
], |
||||
tags = [ |
||||
"no_windows", |
||||
], |
||||
deps = [ |
||||
"//src/core:chaotic_good_legacy_data_endpoints", |
||||
"//test/core/call/yodel:yodel_test", |
||||
"//test/core/transport/util:mock_promise_endpoint", |
||||
], |
||||
) |
||||
|
||||
grpc_fuzz_test( |
||||
name = "config_test", |
||||
srcs = ["config_test.cc"], |
||||
external_deps = [ |
||||
"fuzztest", |
||||
"fuzztest_main", |
||||
"gtest", |
||||
], |
||||
deps = [ |
||||
"//src/core:chaotic_good_frame_cc_proto", |
||||
"//src/core:chaotic_good_legacy_config", |
||||
], |
||||
) |
||||
|
||||
grpc_fuzz_test( |
||||
name = "message_chunker_test", |
||||
srcs = ["message_chunker_test.cc"], |
||||
external_deps = [ |
||||
"fuzztest", |
||||
"fuzztest_main", |
||||
"gtest", |
||||
], |
||||
deps = [ |
||||
"//src/core:chaotic_good_frame_cc_proto", |
||||
"//src/core:chaotic_good_legacy_message_chunker", |
||||
"//src/core:status_flag", |
||||
"//test/core/promise:poll_matcher", |
||||
], |
||||
) |
@ -0,0 +1,155 @@ |
||||
// 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.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/server/chaotic_good_server.h" |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/status.h> |
||||
#include <grpcpp/server.h> |
||||
|
||||
#include <memory> |
||||
#include <string> |
||||
#include <utility> |
||||
|
||||
#include "absl/log/check.h" |
||||
#include "absl/log/log.h" |
||||
#include "absl/strings/str_cat.h" |
||||
#include "absl/time/time.h" |
||||
#include "gmock/gmock.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/client/chaotic_good_connector.h" |
||||
#include "src/core/lib/address_utils/parse_address.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/event_engine/default_event_engine.h" |
||||
#include "src/core/lib/event_engine/tcp_socket_utils.h" |
||||
#include "src/core/server/server.h" |
||||
#include "src/core/util/notification.h" |
||||
#include "src/core/util/time.h" |
||||
#include "src/core/util/uri.h" |
||||
#include "test/core/event_engine/event_engine_test_utils.h" |
||||
#include "test/core/test_util/build.h" |
||||
#include "test/core/test_util/port.h" |
||||
#include "test/core/test_util/test_config.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace testing { |
||||
class ChaoticGoodServerTest : public ::testing::Test { |
||||
public: |
||||
ChaoticGoodServerTest() { |
||||
StartServer(); |
||||
ConstructConnector(); |
||||
} |
||||
|
||||
~ChaoticGoodServerTest() override { |
||||
{ |
||||
ExecCtx exec_ctx; |
||||
if (connecting_successful_) { |
||||
connecting_result_.transport->Orphan(); |
||||
} |
||||
if (connector_ != nullptr) connector_->Shutdown(absl::CancelledError()); |
||||
connector_.reset(); |
||||
} |
||||
args_.channel_args = ChannelArgs(); |
||||
auto* shutdown_cq = grpc_completion_queue_create_for_pluck(nullptr); |
||||
grpc_server_shutdown_and_notify(server_, shutdown_cq, nullptr); |
||||
auto ev = grpc_completion_queue_pluck( |
||||
shutdown_cq, nullptr, grpc_timeout_milliseconds_to_deadline(15000), |
||||
nullptr); |
||||
if (ev.type == GRPC_QUEUE_TIMEOUT) { |
||||
AsanAssertNoLeaks(); |
||||
} |
||||
CHECK_EQ(ev.type, GRPC_OP_COMPLETE); |
||||
CHECK_EQ(ev.tag, nullptr); |
||||
grpc_completion_queue_destroy(shutdown_cq); |
||||
grpc_server_destroy(server_); |
||||
} |
||||
|
||||
void StartServer() { |
||||
port_ = grpc_pick_unused_port_or_die(); |
||||
addr_ = absl::StrCat("[::1]:", port_); |
||||
server_ = grpc_server_create(nullptr, nullptr); |
||||
grpc_server_add_chaotic_good_legacy_port(server_, addr_.c_str()); |
||||
grpc_server_start(server_); |
||||
} |
||||
|
||||
void ConstructConnector() { |
||||
auto uri = URI::Parse("ipv6:" + addr_); |
||||
CHECK_OK(uri); |
||||
CHECK(grpc_parse_uri(*uri, &resolved_addr_)); |
||||
args_.address = &resolved_addr_; |
||||
args_.deadline = Timestamp::Now() + Duration::Seconds(5); |
||||
args_.channel_args = channel_args(); |
||||
connector_ = MakeRefCounted<ChaoticGoodConnector>(); |
||||
} |
||||
|
||||
protected: |
||||
static void OnConnectingFinished(void* arg, grpc_error_handle error) { |
||||
LOG(ERROR) << "OnConnectingFinished: " << arg << " " << error.ToString(); |
||||
ChaoticGoodServerTest* test = static_cast<ChaoticGoodServerTest*>(arg); |
||||
test->connecting_successful_ = error.ok(); |
||||
test->connect_finished_.Notify(); |
||||
} |
||||
|
||||
ChannelArgs channel_args() { |
||||
return CoreConfiguration::Get() |
||||
.channel_args_preconditioning() |
||||
.PreconditionChannelArgs(nullptr); |
||||
} |
||||
|
||||
grpc_server* server_; |
||||
Server* core_server_; |
||||
ChaoticGoodConnector::Args args_; |
||||
ChaoticGoodConnector::Result connecting_result_; |
||||
bool connecting_successful_ = false; |
||||
grpc_closure on_connecting_finished_; |
||||
Notification connect_finished_; |
||||
int port_; |
||||
std::string addr_; |
||||
grpc_resolved_address resolved_addr_; |
||||
RefCountedPtr<ChaoticGoodConnector> connector_; |
||||
}; |
||||
|
||||
TEST_F(ChaoticGoodServerTest, Connect) { |
||||
GRPC_CLOSURE_INIT(&on_connecting_finished_, OnConnectingFinished, this, |
||||
grpc_schedule_on_exec_ctx); |
||||
connector_->Connect(args_, &connecting_result_, &on_connecting_finished_); |
||||
connect_finished_.WaitForNotification(); |
||||
} |
||||
|
||||
TEST_F(ChaoticGoodServerTest, ConnectAndShutdown) { |
||||
Notification connect_finished; |
||||
GRPC_CLOSURE_INIT(&on_connecting_finished_, OnConnectingFinished, this, |
||||
grpc_schedule_on_exec_ctx); |
||||
{ |
||||
ExecCtx exec_ctx; |
||||
connector_->Connect(args_, &connecting_result_, &on_connecting_finished_); |
||||
connector_->Shutdown(absl::InternalError("shutdown")); |
||||
} |
||||
connect_finished_.WaitForNotification(); |
||||
} |
||||
|
||||
} // namespace testing
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
// Must call to create default EventEngine.
|
||||
grpc_init(); |
||||
int ret = RUN_ALL_TESTS(); |
||||
grpc_shutdown(); |
||||
return ret; |
||||
} |
@ -0,0 +1,440 @@ |
||||
// Copyright 2023 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/event_engine/memory_allocator.h> |
||||
#include <grpc/event_engine/slice.h> |
||||
#include <grpc/event_engine/slice_buffer.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/status.h> |
||||
#include <stddef.h> |
||||
|
||||
#include <algorithm> |
||||
#include <memory> |
||||
#include <optional> |
||||
#include <string> |
||||
#include <tuple> |
||||
#include <utility> |
||||
#include <vector> |
||||
|
||||
#include "absl/functional/any_invocable.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "absl/strings/str_format.h" |
||||
#include "gmock/gmock.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/config/core_configuration.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/client_transport.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/iomgr/timer_manager.h" |
||||
#include "src/core/lib/promise/activity.h" |
||||
#include "src/core/lib/promise/event_engine_wakeup_scheduler.h" |
||||
#include "src/core/lib/promise/if.h" |
||||
#include "src/core/lib/promise/join.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/pipe.h" |
||||
#include "src/core/lib/promise/seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/slice/slice_internal.h" |
||||
#include "src/core/lib/transport/metadata_batch.h" |
||||
#include "src/core/lib/transport/promise_endpoint.h" |
||||
#include "src/core/lib/transport/transport.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
#include "test/core/event_engine/fuzzing_event_engine/fuzzing_event_engine.h" |
||||
#include "test/core/event_engine/fuzzing_event_engine/fuzzing_event_engine.pb.h" |
||||
|
||||
using testing::AtMost; |
||||
using testing::MockFunction; |
||||
using testing::Return; |
||||
using testing::StrictMock; |
||||
using testing::WithArgs; |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace testing { |
||||
|
||||
class MockEndpoint |
||||
: public grpc_event_engine::experimental::EventEngine::Endpoint { |
||||
public: |
||||
MOCK_METHOD( |
||||
bool, Read, |
||||
(absl::AnyInvocable<void(absl::Status)> on_read, |
||||
grpc_event_engine::experimental::SliceBuffer* buffer, |
||||
const grpc_event_engine::experimental::EventEngine::Endpoint::ReadArgs* |
||||
args), |
||||
(override)); |
||||
|
||||
MOCK_METHOD( |
||||
bool, Write, |
||||
(absl::AnyInvocable<void(absl::Status)> on_writable, |
||||
grpc_event_engine::experimental::SliceBuffer* data, |
||||
const grpc_event_engine::experimental::EventEngine::Endpoint::WriteArgs* |
||||
args), |
||||
(override)); |
||||
|
||||
MOCK_METHOD( |
||||
const grpc_event_engine::experimental::EventEngine::ResolvedAddress&, |
||||
GetPeerAddress, (), (const, override)); |
||||
MOCK_METHOD( |
||||
const grpc_event_engine::experimental::EventEngine::ResolvedAddress&, |
||||
GetLocalAddress, (), (const, override)); |
||||
}; |
||||
|
||||
class MockClientConnectionFactory : public ClientConnectionFactory { |
||||
public: |
||||
MOCK_METHOD(PendingConnection, Connect, (absl::string_view), (override)); |
||||
void Orphaned() final {} |
||||
}; |
||||
|
||||
struct MockPromiseEndpoint { |
||||
StrictMock<MockEndpoint>* endpoint = new StrictMock<MockEndpoint>(); |
||||
PromiseEndpoint promise_endpoint{ |
||||
std::unique_ptr<StrictMock<MockEndpoint>>(endpoint), SliceBuffer()}; |
||||
}; |
||||
|
||||
// Send messages from client to server.
|
||||
auto SendClientToServerMessages(CallInitiator initiator, int num_messages) { |
||||
return Loop([initiator, num_messages]() mutable { |
||||
bool has_message = (num_messages > 0); |
||||
return If( |
||||
has_message, |
||||
[initiator, &num_messages]() mutable { |
||||
return Seq(initiator.PushMessage(Arena::MakePooled<Message>()), |
||||
[&num_messages]() -> LoopCtl<absl::Status> { |
||||
--num_messages; |
||||
return Continue(); |
||||
}); |
||||
}, |
||||
[initiator]() mutable -> LoopCtl<absl::Status> { |
||||
initiator.FinishSends(); |
||||
return absl::OkStatus(); |
||||
}); |
||||
}); |
||||
} |
||||
|
||||
ClientMetadataHandle TestInitialMetadata() { |
||||
auto md = Arena::MakePooledForOverwrite<ClientMetadata>(); |
||||
md->Set(HttpPathMetadata(), Slice::FromStaticString("/test")); |
||||
return md; |
||||
} |
||||
|
||||
class ClientTransportTest : public ::testing::Test { |
||||
protected: |
||||
const std::shared_ptr<grpc_event_engine::experimental::FuzzingEventEngine>& |
||||
event_engine() { |
||||
return event_engine_; |
||||
} |
||||
|
||||
ChannelArgs MakeChannelArgs() { |
||||
return CoreConfiguration::Get() |
||||
.channel_args_preconditioning() |
||||
.PreconditionChannelArgs(nullptr); |
||||
} |
||||
|
||||
template <typename... PromiseEndpoints> |
||||
Config MakeConfig(PromiseEndpoints... promise_endpoints) { |
||||
Config config(MakeChannelArgs()); |
||||
auto name_endpoint = [i = 0]() mutable { return absl::StrCat(++i); }; |
||||
std::vector<int> this_is_only_here_to_unpack_the_following_statement{ |
||||
(config.ServerAddPendingDataEndpoint(ImmediateConnection( |
||||
name_endpoint(), std::move(promise_endpoints))), |
||||
0)...}; |
||||
return config; |
||||
} |
||||
|
||||
auto MakeCall(ClientMetadataHandle client_initial_metadata) { |
||||
auto arena = call_arena_allocator_->MakeArena(); |
||||
arena->SetContext<grpc_event_engine::experimental::EventEngine>( |
||||
event_engine_.get()); |
||||
return MakeCallPair(std::move(client_initial_metadata), std::move(arena)); |
||||
} |
||||
|
||||
private: |
||||
std::shared_ptr<grpc_event_engine::experimental::FuzzingEventEngine> |
||||
event_engine_{ |
||||
std::make_shared<grpc_event_engine::experimental::FuzzingEventEngine>( |
||||
[]() { |
||||
grpc_timer_manager_set_threading(false); |
||||
grpc_event_engine::experimental::FuzzingEventEngine::Options |
||||
options; |
||||
return options; |
||||
}(), |
||||
fuzzing_event_engine::Actions())}; |
||||
RefCountedPtr<CallArenaAllocator> call_arena_allocator_{ |
||||
MakeRefCounted<CallArenaAllocator>( |
||||
MakeResourceQuota("test-quota") |
||||
->memory_quota() |
||||
->CreateMemoryAllocator("test-allocator"), |
||||
1024)}; |
||||
}; |
||||
|
||||
TEST_F(ClientTransportTest, AddOneStreamWithWriteFailed) { |
||||
MockPromiseEndpoint control_endpoint; |
||||
MockPromiseEndpoint data_endpoint; |
||||
auto client_connection_factory = |
||||
MakeRefCounted<StrictMock<MockClientConnectionFactory>>(); |
||||
// Mock write failed and read is pending.
|
||||
EXPECT_CALL(*control_endpoint.endpoint, Write) |
||||
.Times(AtMost(1)) |
||||
.WillOnce( |
||||
WithArgs<0>([](absl::AnyInvocable<void(absl::Status)> on_write) { |
||||
on_write(absl::InternalError("control endpoint write failed.")); |
||||
return false; |
||||
})); |
||||
EXPECT_CALL(*data_endpoint.endpoint, Write) |
||||
.Times(AtMost(1)) |
||||
.WillOnce( |
||||
WithArgs<0>([](absl::AnyInvocable<void(absl::Status)> on_write) { |
||||
on_write(absl::InternalError("data endpoint write failed.")); |
||||
return false; |
||||
})); |
||||
EXPECT_CALL(*control_endpoint.endpoint, Read).WillOnce(Return(false)); |
||||
auto transport = MakeOrphanable<ChaoticGoodClientTransport>( |
||||
MakeChannelArgs(), std::move(control_endpoint.promise_endpoint), |
||||
MakeConfig(std::move(data_endpoint.promise_endpoint)), |
||||
client_connection_factory); |
||||
auto call = MakeCall(TestInitialMetadata()); |
||||
transport->StartCall(call.handler.StartCall()); |
||||
call.initiator.SpawnGuarded("test-send", |
||||
[initiator = call.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 1); |
||||
}); |
||||
StrictMock<MockFunction<void()>> on_done; |
||||
EXPECT_CALL(on_done, Call()); |
||||
call.initiator.SpawnInfallible( |
||||
"test-read", [&on_done, initiator = call.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
}, |
||||
initiator.PullServerTrailingMetadata(), |
||||
[&on_done](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), |
||||
GRPC_STATUS_UNAVAILABLE); |
||||
on_done.Call(); |
||||
}); |
||||
}); |
||||
// Wait until ClientTransport's internal activities to finish.
|
||||
event_engine()->TickUntilIdle(); |
||||
transport.reset(); |
||||
event_engine()->TickUntilIdle(); |
||||
event_engine()->UnsetGlobalHooks(); |
||||
} |
||||
|
||||
TEST_F(ClientTransportTest, AddOneStreamWithReadFailed) { |
||||
MockPromiseEndpoint control_endpoint; |
||||
MockPromiseEndpoint data_endpoint; |
||||
auto client_connection_factory = |
||||
MakeRefCounted<StrictMock<MockClientConnectionFactory>>(); |
||||
// Mock read failed.
|
||||
EXPECT_CALL(*control_endpoint.endpoint, Read) |
||||
.WillOnce(WithArgs<0>( |
||||
[](absl::AnyInvocable<void(absl::Status)> on_read) mutable { |
||||
on_read(absl::InternalError("control endpoint read failed.")); |
||||
// Return false to mock EventEngine read not finish.
|
||||
return false; |
||||
})); |
||||
auto transport = MakeOrphanable<ChaoticGoodClientTransport>( |
||||
MakeChannelArgs(), std::move(control_endpoint.promise_endpoint), |
||||
MakeConfig(std::move(data_endpoint.promise_endpoint)), |
||||
client_connection_factory); |
||||
auto call = MakeCall(TestInitialMetadata()); |
||||
transport->StartCall(call.handler.StartCall()); |
||||
call.initiator.SpawnGuarded("test-send", |
||||
[initiator = call.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 1); |
||||
}); |
||||
StrictMock<MockFunction<void()>> on_done; |
||||
EXPECT_CALL(on_done, Call()); |
||||
call.initiator.SpawnInfallible( |
||||
"test-read", [&on_done, initiator = call.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
}, |
||||
initiator.PullServerTrailingMetadata(), |
||||
[&on_done](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), |
||||
GRPC_STATUS_UNAVAILABLE); |
||||
on_done.Call(); |
||||
}); |
||||
}); |
||||
// Wait until ClientTransport's internal activities to finish.
|
||||
event_engine()->TickUntilIdle(); |
||||
transport.reset(); |
||||
event_engine()->TickUntilIdle(); |
||||
event_engine()->UnsetGlobalHooks(); |
||||
} |
||||
|
||||
TEST_F(ClientTransportTest, AddMultipleStreamWithWriteFailed) { |
||||
// Mock write failed at first stream and second stream's write will fail too.
|
||||
MockPromiseEndpoint control_endpoint; |
||||
MockPromiseEndpoint data_endpoint; |
||||
auto client_connection_factory = |
||||
MakeRefCounted<StrictMock<MockClientConnectionFactory>>(); |
||||
EXPECT_CALL(*control_endpoint.endpoint, Write) |
||||
.Times(AtMost(1)) |
||||
.WillRepeatedly( |
||||
WithArgs<0>([](absl::AnyInvocable<void(absl::Status)> on_write) { |
||||
on_write(absl::InternalError("control endpoint write failed.")); |
||||
return false; |
||||
})); |
||||
EXPECT_CALL(*data_endpoint.endpoint, Write) |
||||
.Times(AtMost(1)) |
||||
.WillRepeatedly( |
||||
WithArgs<0>([](absl::AnyInvocable<void(absl::Status)> on_write) { |
||||
on_write(absl::InternalError("data endpoint write failed.")); |
||||
return false; |
||||
})); |
||||
EXPECT_CALL(*control_endpoint.endpoint, Read).WillOnce(Return(false)); |
||||
auto transport = MakeOrphanable<ChaoticGoodClientTransport>( |
||||
MakeChannelArgs(), std::move(control_endpoint.promise_endpoint), |
||||
MakeConfig(std::move(data_endpoint.promise_endpoint)), |
||||
client_connection_factory); |
||||
auto call1 = MakeCall(TestInitialMetadata()); |
||||
transport->StartCall(call1.handler.StartCall()); |
||||
auto call2 = MakeCall(TestInitialMetadata()); |
||||
transport->StartCall(call2.handler.StartCall()); |
||||
call1.initiator.SpawnGuarded( |
||||
"test-send-1", [initiator = call1.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 1); |
||||
}); |
||||
call2.initiator.SpawnGuarded( |
||||
"test-send-2", [initiator = call2.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 1); |
||||
}); |
||||
StrictMock<MockFunction<void()>> on_done1; |
||||
EXPECT_CALL(on_done1, Call()); |
||||
StrictMock<MockFunction<void()>> on_done2; |
||||
EXPECT_CALL(on_done2, Call()); |
||||
call1.initiator.SpawnInfallible( |
||||
"test-read-1", [&on_done1, initiator = call1.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
}, |
||||
initiator.PullServerTrailingMetadata(), |
||||
[&on_done1](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), |
||||
GRPC_STATUS_UNAVAILABLE); |
||||
on_done1.Call(); |
||||
}); |
||||
}); |
||||
call2.initiator.SpawnInfallible( |
||||
"test-read-2", [&on_done2, initiator = call2.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
}, |
||||
initiator.PullServerTrailingMetadata(), |
||||
[&on_done2](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), |
||||
GRPC_STATUS_UNAVAILABLE); |
||||
on_done2.Call(); |
||||
}); |
||||
}); |
||||
// Wait until ClientTransport's internal activities to finish.
|
||||
event_engine()->TickUntilIdle(); |
||||
transport.reset(); |
||||
event_engine()->TickUntilIdle(); |
||||
event_engine()->UnsetGlobalHooks(); |
||||
} |
||||
|
||||
TEST_F(ClientTransportTest, AddMultipleStreamWithReadFailed) { |
||||
MockPromiseEndpoint control_endpoint; |
||||
MockPromiseEndpoint data_endpoint; |
||||
auto client_connection_factory = |
||||
MakeRefCounted<StrictMock<MockClientConnectionFactory>>(); |
||||
// Mock read failed at first stream, and second stream's write will fail too.
|
||||
EXPECT_CALL(*control_endpoint.endpoint, Read) |
||||
.WillOnce(WithArgs<0>( |
||||
[](absl::AnyInvocable<void(absl::Status)> on_read) mutable { |
||||
on_read(absl::InternalError("control endpoint read failed.")); |
||||
// Return false to mock EventEngine read not finish.
|
||||
return false; |
||||
})); |
||||
auto transport = MakeOrphanable<ChaoticGoodClientTransport>( |
||||
MakeChannelArgs(), std::move(control_endpoint.promise_endpoint), |
||||
MakeConfig(std::move(data_endpoint.promise_endpoint)), |
||||
client_connection_factory); |
||||
auto call1 = MakeCall(TestInitialMetadata()); |
||||
transport->StartCall(call1.handler.StartCall()); |
||||
auto call2 = MakeCall(TestInitialMetadata()); |
||||
transport->StartCall(call2.handler.StartCall()); |
||||
call1.initiator.SpawnGuarded( |
||||
"test-send", [initiator = call1.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 1); |
||||
}); |
||||
call2.initiator.SpawnGuarded( |
||||
"test-send", [initiator = call2.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 1); |
||||
}); |
||||
StrictMock<MockFunction<void()>> on_done1; |
||||
EXPECT_CALL(on_done1, Call()); |
||||
StrictMock<MockFunction<void()>> on_done2; |
||||
EXPECT_CALL(on_done2, Call()); |
||||
call1.initiator.SpawnInfallible( |
||||
"test-read", [&on_done1, initiator = call1.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
}, |
||||
initiator.PullServerTrailingMetadata(), |
||||
[&on_done1](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), |
||||
GRPC_STATUS_UNAVAILABLE); |
||||
on_done1.Call(); |
||||
}); |
||||
}); |
||||
call2.initiator.SpawnInfallible( |
||||
"test-read", [&on_done2, initiator = call2.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
}, |
||||
initiator.PullServerTrailingMetadata(), |
||||
[&on_done2](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), |
||||
GRPC_STATUS_UNAVAILABLE); |
||||
on_done2.Call(); |
||||
}); |
||||
}); |
||||
// Wait until ClientTransport's internal activities to finish.
|
||||
event_engine()->TickUntilIdle(); |
||||
transport.reset(); |
||||
event_engine()->TickUntilIdle(); |
||||
event_engine()->UnsetGlobalHooks(); |
||||
} |
||||
|
||||
} // namespace testing
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
// Must call to create default EventEngine.
|
||||
grpc_init(); |
||||
int ret = RUN_ALL_TESTS(); |
||||
grpc_shutdown(); |
||||
return ret; |
||||
} |
@ -0,0 +1,304 @@ |
||||
// Copyright 2023 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/client_transport.h" |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/event_engine/memory_allocator.h> |
||||
#include <grpc/event_engine/slice.h> |
||||
#include <grpc/event_engine/slice_buffer.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/status.h> |
||||
|
||||
#include <algorithm> |
||||
#include <cstddef> |
||||
#include <cstdlib> |
||||
#include <initializer_list> |
||||
#include <memory> |
||||
#include <optional> |
||||
#include <string> |
||||
#include <tuple> |
||||
#include <vector> |
||||
|
||||
#include "absl/functional/any_invocable.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "absl/strings/str_format.h" |
||||
#include "gmock/gmock.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/config/core_configuration.h" |
||||
#include "src/core/ext/transport/chaotic_good/chaotic_good_frame.pb.h" |
||||
#include "src/core/lib/promise/if.h" |
||||
#include "src/core/lib/promise/loop.h" |
||||
#include "src/core/lib/promise/seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/transport/metadata_batch.h" |
||||
#include "test/core/transport/chaotic_good_legacy/transport_test.h" |
||||
#include "test/core/transport/util/mock_promise_endpoint.h" |
||||
|
||||
using testing::MockFunction; |
||||
using testing::Return; |
||||
using testing::StrictMock; |
||||
|
||||
using EventEngineSlice = grpc_event_engine::experimental::Slice; |
||||
|
||||
using grpc_core::chaotic_good::testing::MockPromiseEndpoint; |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace testing { |
||||
|
||||
class MockClientConnectionFactory : public ClientConnectionFactory { |
||||
public: |
||||
MOCK_METHOD(PendingConnection, Connect, (absl::string_view), (override)); |
||||
void Orphaned() final {} |
||||
}; |
||||
|
||||
ClientMetadataHandle TestInitialMetadata() { |
||||
auto md = Arena::MakePooledForOverwrite<ClientMetadata>(); |
||||
md->Set(HttpPathMetadata(), Slice::FromStaticString("/demo.Service/Step")); |
||||
return md; |
||||
} |
||||
|
||||
// Send messages from client to server.
|
||||
auto SendClientToServerMessages(CallInitiator initiator, int num_messages) { |
||||
return Loop([initiator, num_messages, i = 0]() mutable { |
||||
bool has_message = (i < num_messages); |
||||
return If( |
||||
has_message, |
||||
[initiator, &i]() mutable { |
||||
return Seq( |
||||
initiator.PushMessage(Arena::MakePooled<Message>( |
||||
SliceBuffer(Slice::FromCopiedString(std::to_string(i))), 0)), |
||||
[&i]() -> LoopCtl<absl::Status> { |
||||
++i; |
||||
return Continue(); |
||||
}); |
||||
}, |
||||
[initiator]() mutable -> LoopCtl<absl::Status> { |
||||
initiator.FinishSends(); |
||||
return absl::OkStatus(); |
||||
}); |
||||
}); |
||||
} |
||||
|
||||
ChannelArgs MakeChannelArgs( |
||||
std::shared_ptr<grpc_event_engine::experimental::EventEngine> |
||||
event_engine) { |
||||
return CoreConfiguration::Get() |
||||
.channel_args_preconditioning() |
||||
.PreconditionChannelArgs(nullptr) |
||||
.SetObject<grpc_event_engine::experimental::EventEngine>( |
||||
std::move(event_engine)); |
||||
} |
||||
|
||||
template <typename... PromiseEndpoints> |
||||
Config MakeConfig(const ChannelArgs& channel_args, |
||||
PromiseEndpoints... promise_endpoints) { |
||||
Config config(channel_args); |
||||
auto name_endpoint = [i = 0]() mutable { return absl::StrCat(++i); }; |
||||
std::vector<int> this_is_only_here_to_unpack_the_following_statement{ |
||||
(config.ServerAddPendingDataEndpoint( |
||||
ImmediateConnection(name_endpoint(), std::move(promise_endpoints))), |
||||
0)...}; |
||||
return config; |
||||
} |
||||
|
||||
TEST_F(TransportTest, AddOneStream) { |
||||
MockPromiseEndpoint control_endpoint(1000); |
||||
MockPromiseEndpoint data_endpoint(1001); |
||||
auto client_connection_factory = |
||||
MakeRefCounted<StrictMock<MockClientConnectionFactory>>(); |
||||
static const std::string many_as(1024 * 1024, 'a'); |
||||
const auto server_initial_metadata = |
||||
EncodeProto<chaotic_good_frame::ServerMetadata>("message: 'hello'"); |
||||
const auto server_trailing_metadata = |
||||
EncodeProto<chaotic_good_frame::ServerMetadata>("status: 0"); |
||||
const auto client_initial_metadata = |
||||
EncodeProto<chaotic_good_frame::ClientMetadata>( |
||||
"path: '/demo.Service/Step'"); |
||||
control_endpoint.ExpectRead( |
||||
{SerializedFrameHeader(FrameType::kServerInitialMetadata, 0, 1, |
||||
server_initial_metadata.length()), |
||||
server_initial_metadata.Copy(), |
||||
SerializedFrameHeader(FrameType::kMessage, 1, 1, many_as.length()), |
||||
SerializedFrameHeader(FrameType::kServerTrailingMetadata, 0, 1, |
||||
server_trailing_metadata.length()), |
||||
server_trailing_metadata.Copy()}, |
||||
event_engine().get()); |
||||
data_endpoint.ExpectRead( |
||||
{EventEngineSlice::FromCopiedString(many_as), Zeros(56)}, nullptr); |
||||
EXPECT_CALL(*control_endpoint.endpoint, Read) |
||||
.InSequence(control_endpoint.read_sequence) |
||||
.WillOnce(Return(false)); |
||||
auto channel_args = MakeChannelArgs(event_engine()); |
||||
auto transport = MakeOrphanable<ChaoticGoodClientTransport>( |
||||
channel_args, std::move(control_endpoint.promise_endpoint), |
||||
MakeConfig(channel_args, std::move(data_endpoint.promise_endpoint)), |
||||
client_connection_factory); |
||||
auto call = MakeCall(TestInitialMetadata()); |
||||
StrictMock<MockFunction<void()>> on_done; |
||||
EXPECT_CALL(on_done, Call()); |
||||
control_endpoint.ExpectWrite( |
||||
{SerializedFrameHeader(FrameType::kClientInitialMetadata, 0, 1, |
||||
client_initial_metadata.length()), |
||||
client_initial_metadata.Copy()}, |
||||
nullptr); |
||||
control_endpoint.ExpectWrite( |
||||
{SerializedFrameHeader(FrameType::kMessage, 0, 1, 1), |
||||
EventEngineSlice::FromCopiedString("0"), |
||||
SerializedFrameHeader(FrameType::kClientEndOfStream, 0, 1, 0)}, |
||||
nullptr); |
||||
transport->StartCall(call.handler.StartCall()); |
||||
call.initiator.SpawnGuarded("test-send", |
||||
[initiator = call.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 1); |
||||
}); |
||||
call.initiator.SpawnInfallible( |
||||
"test-read", [&on_done, initiator = call.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
EXPECT_TRUE(md.value().has_value()); |
||||
EXPECT_EQ(md.value() |
||||
.value() |
||||
->get_pointer(GrpcMessageMetadata()) |
||||
->as_string_view(), |
||||
"hello"); |
||||
}, |
||||
[initiator]() mutable { return initiator.PullMessage(); }, |
||||
[](ServerToClientNextMessage msg) { |
||||
EXPECT_TRUE(msg.ok()); |
||||
EXPECT_TRUE(msg.has_value()); |
||||
EXPECT_EQ(msg.value().payload()->JoinIntoString(), many_as); |
||||
}, |
||||
[initiator]() mutable { return initiator.PullMessage(); }, |
||||
[](ServerToClientNextMessage msg) { |
||||
EXPECT_TRUE(msg.ok()); |
||||
EXPECT_FALSE(msg.has_value()); |
||||
}, |
||||
[initiator]() mutable { |
||||
return initiator.PullServerTrailingMetadata(); |
||||
}, |
||||
[&on_done](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), GRPC_STATUS_OK); |
||||
on_done.Call(); |
||||
}); |
||||
}); |
||||
// Wait until ClientTransport's internal activities to finish.
|
||||
event_engine()->TickUntilIdle(); |
||||
event_engine()->UnsetGlobalHooks(); |
||||
} |
||||
|
||||
TEST_F(TransportTest, AddOneStreamMultipleMessages) { |
||||
MockPromiseEndpoint control_endpoint(1000); |
||||
MockPromiseEndpoint data_endpoint(1001); |
||||
auto client_connection_factory = |
||||
MakeRefCounted<StrictMock<MockClientConnectionFactory>>(); |
||||
const auto server_initial_metadata = |
||||
EncodeProto<chaotic_good_frame::ServerMetadata>(""); |
||||
const auto server_trailing_metadata = |
||||
EncodeProto<chaotic_good_frame::ServerMetadata>("status: 0"); |
||||
const auto client_initial_metadata = |
||||
EncodeProto<chaotic_good_frame::ClientMetadata>( |
||||
"path: '/demo.Service/Step'"); |
||||
control_endpoint.ExpectRead( |
||||
{SerializedFrameHeader(FrameType::kServerInitialMetadata, 0, 1, |
||||
server_initial_metadata.length()), |
||||
server_initial_metadata.Copy(), |
||||
SerializedFrameHeader(FrameType::kMessage, 0, 1, 8), |
||||
EventEngineSlice::FromCopiedString("12345678"), |
||||
SerializedFrameHeader(FrameType::kMessage, 0, 1, 8), |
||||
EventEngineSlice::FromCopiedString("87654321"), |
||||
SerializedFrameHeader(FrameType::kServerTrailingMetadata, 0, 1, |
||||
server_trailing_metadata.length()), |
||||
server_trailing_metadata.Copy()}, |
||||
event_engine().get()); |
||||
EXPECT_CALL(*control_endpoint.endpoint, Read) |
||||
.InSequence(control_endpoint.read_sequence) |
||||
.WillOnce(Return(false)); |
||||
auto channel_args = MakeChannelArgs(event_engine()); |
||||
auto transport = MakeOrphanable<ChaoticGoodClientTransport>( |
||||
channel_args, std::move(control_endpoint.promise_endpoint), |
||||
MakeConfig(channel_args, std::move(data_endpoint.promise_endpoint)), |
||||
client_connection_factory); |
||||
auto call = MakeCall(TestInitialMetadata()); |
||||
StrictMock<MockFunction<void()>> on_done; |
||||
EXPECT_CALL(on_done, Call()); |
||||
control_endpoint.ExpectWrite( |
||||
{SerializedFrameHeader(FrameType::kClientInitialMetadata, 0, 1, |
||||
client_initial_metadata.length()), |
||||
client_initial_metadata.Copy()}, |
||||
nullptr); |
||||
control_endpoint.ExpectWrite( |
||||
{SerializedFrameHeader(FrameType::kMessage, 0, 1, 1), |
||||
EventEngineSlice::FromCopiedString("0"), |
||||
SerializedFrameHeader(FrameType::kMessage, 0, 1, 1), |
||||
EventEngineSlice::FromCopiedString("1"), |
||||
SerializedFrameHeader(FrameType::kClientEndOfStream, 0, 1, 0)}, |
||||
nullptr); |
||||
transport->StartCall(call.handler.StartCall()); |
||||
call.initiator.SpawnGuarded("test-send", |
||||
[initiator = call.initiator]() mutable { |
||||
return SendClientToServerMessages(initiator, 2); |
||||
}); |
||||
call.initiator.SpawnInfallible( |
||||
"test-read", [&on_done, initiator = call.initiator]() mutable { |
||||
return Seq( |
||||
initiator.PullServerInitialMetadata(), |
||||
[](ValueOrFailure<std::optional<ServerMetadataHandle>> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
EXPECT_TRUE(md.value().has_value()); |
||||
}, |
||||
initiator.PullMessage(), |
||||
[](ServerToClientNextMessage msg) { |
||||
EXPECT_TRUE(msg.ok()); |
||||
EXPECT_TRUE(msg.has_value()); |
||||
EXPECT_EQ(msg.value().payload()->JoinIntoString(), "12345678"); |
||||
}, |
||||
initiator.PullMessage(), |
||||
[](ServerToClientNextMessage msg) { |
||||
EXPECT_TRUE(msg.ok()); |
||||
EXPECT_TRUE(msg.has_value()); |
||||
EXPECT_EQ(msg.value().payload()->JoinIntoString(), "87654321"); |
||||
}, |
||||
initiator.PullMessage(), |
||||
[](ServerToClientNextMessage msg) { |
||||
EXPECT_TRUE(msg.ok()); |
||||
EXPECT_FALSE(msg.has_value()); |
||||
}, |
||||
initiator.PullServerTrailingMetadata(), |
||||
[&on_done](ServerMetadataHandle md) { |
||||
EXPECT_EQ(md->get(GrpcStatusMetadata()).value(), GRPC_STATUS_OK); |
||||
on_done.Call(); |
||||
}); |
||||
}); |
||||
// Wait until ClientTransport's internal activities to finish.
|
||||
event_engine()->TickUntilIdle(); |
||||
event_engine()->UnsetGlobalHooks(); |
||||
} |
||||
|
||||
} // namespace testing
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
// Must call to create default EventEngine.
|
||||
grpc_init(); |
||||
int ret = RUN_ALL_TESTS(); |
||||
grpc_shutdown(); |
||||
return ret; |
||||
} |
@ -0,0 +1,126 @@ |
||||
// Copyright 2023 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/config.h" |
||||
|
||||
#include <vector> |
||||
|
||||
#include "fuzztest/fuzztest.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/ext/transport/chaotic_good/chaotic_good_frame.pb.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace { |
||||
|
||||
// Arguments to a config class - these represent channel args, but in a format
|
||||
// that the fuzzer can search really quickly.
|
||||
// We then convert this struct to channel args, and feed it into the config to
|
||||
// keep things as they would be in production code.
|
||||
struct FuzzerChannelArgs { |
||||
std::optional<int> alignment; |
||||
std::optional<int> max_recv_chunk_size; |
||||
std::optional<int> max_send_chunk_size; |
||||
std::optional<int> inlined_payload_size_threshold; |
||||
std::optional<bool> tracing_enabled; |
||||
|
||||
ChannelArgs MakeChannelArgs() { |
||||
ChannelArgs out; |
||||
auto transfer = [&out](auto value, const char* name) { |
||||
if (!value.has_value()) return; |
||||
out = out.Set(name, *value); |
||||
}; |
||||
transfer(alignment, GRPC_ARG_CHAOTIC_GOOD_ALIGNMENT); |
||||
transfer(max_recv_chunk_size, GRPC_ARG_CHAOTIC_GOOD_MAX_RECV_CHUNK_SIZE); |
||||
transfer(max_send_chunk_size, GRPC_ARG_CHAOTIC_GOOD_MAX_SEND_CHUNK_SIZE); |
||||
transfer(inlined_payload_size_threshold, |
||||
GRPC_ARG_CHAOTIC_GOOD_INLINED_PAYLOAD_SIZE_THRESHOLD); |
||||
transfer(tracing_enabled, GRPC_ARG_TCP_TRACING_ENABLED); |
||||
return out; |
||||
} |
||||
}; |
||||
|
||||
class FakeClientConnectionFactory |
||||
: public chaotic_good_legacy::ClientConnectionFactory { |
||||
public: |
||||
chaotic_good_legacy::PendingConnection Connect( |
||||
absl::string_view id) override { |
||||
Crash("Connect not implemented"); |
||||
} |
||||
void Orphaned() override {} |
||||
}; |
||||
|
||||
void ConfigTest(FuzzerChannelArgs client_args_input, |
||||
FuzzerChannelArgs server_args_input) { |
||||
// Create channel args
|
||||
const auto client_args = client_args_input.MakeChannelArgs(); |
||||
const auto server_args = server_args_input.MakeChannelArgs(); |
||||
// Initialize configs
|
||||
chaotic_good_legacy::Config client_config(client_args); |
||||
chaotic_good_legacy::Config server_config(server_args); |
||||
VLOG(2) << "client_config: " << client_config; |
||||
VLOG(2) << "server_config: " << server_config; |
||||
// Perform handshake
|
||||
chaotic_good_frame::Settings client_settings; |
||||
client_config.PrepareClientOutgoingSettings(client_settings); |
||||
VLOG(2) << "client settings: " << client_settings.ShortDebugString(); |
||||
CHECK_OK(server_config.ReceiveClientIncomingSettings(client_settings)); |
||||
VLOG(2) << "server_config': " << server_config; |
||||
chaotic_good_frame::Settings server_settings; |
||||
server_config.PrepareServerOutgoingSettings(server_settings); |
||||
VLOG(2) << "server settings: " << server_settings.ShortDebugString(); |
||||
FakeClientConnectionFactory fake_factory; |
||||
CHECK_OK(client_config.ReceiveServerIncomingSettings(server_settings, |
||||
fake_factory)); |
||||
VLOG(2) << "client_config': " << client_config; |
||||
// Generate results
|
||||
const chaotic_good_legacy::ChaoticGoodTransport::Options client_options = |
||||
client_config.MakeTransportOptions(); |
||||
const chaotic_good_legacy::ChaoticGoodTransport::Options server_options = |
||||
server_config.MakeTransportOptions(); |
||||
const chaotic_good_legacy::MessageChunker client_chunker = |
||||
client_config.MakeMessageChunker(); |
||||
const chaotic_good_legacy::MessageChunker server_chunker = |
||||
server_config.MakeMessageChunker(); |
||||
// Validate results
|
||||
EXPECT_EQ(client_options.encode_alignment, server_options.decode_alignment); |
||||
EXPECT_EQ(client_options.decode_alignment, server_options.encode_alignment); |
||||
EXPECT_EQ(client_chunker.alignment(), client_options.encode_alignment); |
||||
EXPECT_EQ(server_chunker.alignment(), server_options.encode_alignment); |
||||
EXPECT_GE(server_config.max_recv_chunk_size(), |
||||
client_config.max_send_chunk_size()); |
||||
EXPECT_GE(client_config.max_recv_chunk_size(), |
||||
server_config.max_send_chunk_size()); |
||||
if (auto a = client_args.GetInt(GRPC_ARG_CHAOTIC_GOOD_ALIGNMENT); |
||||
a.has_value() && *a > 0) { |
||||
EXPECT_EQ(client_options.decode_alignment, *a); |
||||
} |
||||
if (auto a = server_args.GetInt(GRPC_ARG_CHAOTIC_GOOD_ALIGNMENT); |
||||
a.has_value() && *a > 0) { |
||||
EXPECT_EQ(server_options.decode_alignment, *a); |
||||
} |
||||
if (auto a = client_args.GetInt( |
||||
GRPC_ARG_CHAOTIC_GOOD_INLINED_PAYLOAD_SIZE_THRESHOLD); |
||||
a.has_value() && *a > 0) { |
||||
EXPECT_EQ(client_options.inlined_payload_size_threshold, *a); |
||||
} |
||||
if (auto a = server_args.GetInt( |
||||
GRPC_ARG_CHAOTIC_GOOD_INLINED_PAYLOAD_SIZE_THRESHOLD); |
||||
a.has_value() && *a > 0) { |
||||
EXPECT_EQ(server_options.inlined_payload_size_threshold, *a); |
||||
} |
||||
} |
||||
FUZZ_TEST(MyTestSuite, ConfigTest); |
||||
|
||||
} // namespace
|
||||
} // namespace grpc_core
|
@ -0,0 +1,45 @@ |
||||
// 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.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/control_endpoint.h" |
||||
|
||||
#include <grpc/grpc.h> |
||||
|
||||
#include "gtest/gtest.h" |
||||
#include "test/core/call/yodel/yodel_test.h" |
||||
#include "test/core/transport/util/mock_promise_endpoint.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
class ControlEndpointTest : public YodelTest { |
||||
protected: |
||||
using YodelTest::YodelTest; |
||||
}; |
||||
|
||||
#define CONTROL_ENDPOINT_TEST(name) YODEL_TEST(ControlEndpointTest, name) |
||||
|
||||
CONTROL_ENDPOINT_TEST(CanWrite) { |
||||
chaotic_good::testing::MockPromiseEndpoint ep(1234); |
||||
chaotic_good_legacy::ControlEndpoint control_endpoint( |
||||
std::move(ep.promise_endpoint), event_engine().get()); |
||||
ep.ExpectWrite( |
||||
{grpc_event_engine::experimental::Slice::FromCopiedString("hello")}, |
||||
nullptr); |
||||
SpawnTestSeqWithoutContext( |
||||
"write", |
||||
control_endpoint.Write(SliceBuffer(Slice::FromCopiedString("hello")))); |
||||
WaitForAllPendingWork(); |
||||
} |
||||
|
||||
} // namespace grpc_core
|
@ -0,0 +1,150 @@ |
||||
// 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.
|
||||
|
||||
// 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.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/data_endpoints.h" |
||||
|
||||
#include <google/protobuf/text_format.h> |
||||
#include <grpc/grpc.h> |
||||
|
||||
#include "gtest/gtest.h" |
||||
#include "test/core/call/yodel/yodel_test.h" |
||||
#include "test/core/transport/util/mock_promise_endpoint.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
class DataEndpointsTest : public YodelTest { |
||||
protected: |
||||
using YodelTest::YodelTest; |
||||
}; |
||||
|
||||
#define DATA_ENDPOINTS_TEST(name) YODEL_TEST(DataEndpointsTest, name) |
||||
|
||||
template <typename... Args> |
||||
std::vector<chaotic_good_legacy::PendingConnection> Endpoints(Args... args) { |
||||
std::vector<chaotic_good_legacy::PendingConnection> connections; |
||||
std::vector<int> this_is_just_here_to_get_the_statements_to_unpack = { |
||||
(connections.emplace_back( |
||||
chaotic_good_legacy::ImmediateConnection("foo", std::move(args))), |
||||
0)...}; |
||||
return connections; |
||||
} |
||||
|
||||
DATA_ENDPOINTS_TEST(CanWrite) { |
||||
chaotic_good::testing::MockPromiseEndpoint ep(1234); |
||||
chaotic_good_legacy::DataEndpoints data_endpoints( |
||||
Endpoints(std::move(ep.promise_endpoint)), event_engine().get(), false); |
||||
ep.ExpectWrite( |
||||
{grpc_event_engine::experimental::Slice::FromCopiedString("hello")}, |
||||
event_engine().get()); |
||||
SpawnTestSeqWithoutContext( |
||||
"write", |
||||
data_endpoints.Write(SliceBuffer(Slice::FromCopiedString("hello"))), |
||||
[](uint32_t id) { EXPECT_EQ(id, 0); }); |
||||
WaitForAllPendingWork(); |
||||
} |
||||
|
||||
DATA_ENDPOINTS_TEST(CanMultiWrite) { |
||||
chaotic_good::testing::MockPromiseEndpoint ep1(1234); |
||||
chaotic_good::testing::MockPromiseEndpoint ep2(1235); |
||||
chaotic_good_legacy::DataEndpoints data_endpoints( |
||||
Endpoints(std::move(ep1.promise_endpoint), |
||||
std::move(ep2.promise_endpoint)), |
||||
event_engine().get(), false); |
||||
SliceBuffer writes1; |
||||
SliceBuffer writes2; |
||||
ep1.CaptureWrites(writes1, event_engine().get()); |
||||
ep2.CaptureWrites(writes2, event_engine().get()); |
||||
uint32_t write1_ep = 42; |
||||
uint32_t write2_ep = 42; |
||||
SpawnTestSeqWithoutContext( |
||||
"write", |
||||
data_endpoints.Write(SliceBuffer(Slice::FromCopiedString("hello"))), |
||||
[&write1_ep](uint32_t id) { write1_ep = id; }, |
||||
data_endpoints.Write(SliceBuffer(Slice::FromCopiedString("world"))), |
||||
[&write2_ep](uint32_t id) { write2_ep = id; }); |
||||
TickUntilTrue([&]() { return writes1.Length() + writes2.Length() == 10; }); |
||||
WaitForAllPendingWork(); |
||||
EXPECT_THAT(write1_ep, ::testing::AnyOf(0, 1)); |
||||
EXPECT_THAT(write2_ep, ::testing::AnyOf(0, 1)); |
||||
std::string expect[2]; |
||||
expect[write1_ep] += "hello"; |
||||
expect[write2_ep] += "world"; |
||||
LOG(INFO) << GRPC_DUMP_ARGS(write1_ep, write2_ep); |
||||
EXPECT_EQ(writes1.JoinIntoString(), expect[0]); |
||||
EXPECT_EQ(writes2.JoinIntoString(), expect[1]); |
||||
} |
||||
|
||||
DATA_ENDPOINTS_TEST(CanRead) { |
||||
chaotic_good::testing::MockPromiseEndpoint ep(1234); |
||||
chaotic_good_legacy::DataEndpoints data_endpoints( |
||||
Endpoints(std::move(ep.promise_endpoint)), event_engine().get(), false); |
||||
ep.ExpectRead( |
||||
{grpc_event_engine::experimental::Slice::FromCopiedString("hello")}, |
||||
event_engine().get()); |
||||
SpawnTestSeqWithoutContext("read", data_endpoints.Read(0, 5).Await(), |
||||
[](absl::StatusOr<SliceBuffer> result) { |
||||
EXPECT_TRUE(result.ok()); |
||||
EXPECT_EQ(result->JoinIntoString(), "hello"); |
||||
}); |
||||
WaitForAllPendingWork(); |
||||
} |
||||
|
||||
namespace { |
||||
yodel::Msg ParseTestProto(const std::string& text) { |
||||
yodel::Msg msg; |
||||
CHECK(google::protobuf::TextFormat::ParseFromString(text, &msg)); |
||||
return msg; |
||||
} |
||||
} // namespace
|
||||
|
||||
TEST(DataEndpointsTest, CanMultiWriteRegression) { |
||||
CanMultiWrite(ParseTestProto( |
||||
R"pb(event_engine_actions { |
||||
run_delay: 9223372036854775807 |
||||
run_delay: 9223372036854775807 |
||||
run_delay: 9223372036854775801 |
||||
run_delay: 0 |
||||
run_delay: 5807413915228537483 |
||||
assign_ports: 3508738622 |
||||
assign_ports: 4238198998 |
||||
assign_ports: 857428670 |
||||
assign_ports: 0 |
||||
assign_ports: 4227858431 |
||||
assign_ports: 2863084513 |
||||
assign_ports: 1868867780 |
||||
assign_ports: 0 |
||||
connections { write_size: 2147483647 write_size: 4294705148 } |
||||
connections { write_size: 1 } |
||||
} |
||||
rng: 1 |
||||
rng: 14109448502428080414 |
||||
rng: 18446744073709551615 |
||||
rng: 13568317980260708783)pb")); |
||||
} |
||||
|
||||
} // namespace grpc_core
|
@ -0,0 +1,109 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include <grpc/event_engine/memory_allocator.h> |
||||
#include <stddef.h> |
||||
#include <stdint.h> |
||||
|
||||
#include <limits> |
||||
#include <memory> |
||||
|
||||
#include "absl/log/check.h" |
||||
#include "absl/log/log.h" |
||||
#include "absl/random/bit_gen_ref.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "fuzztest/fuzztest.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
#include "src/core/lib/iomgr/exec_ctx.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
#include "test/core/promise/test_context.h" |
||||
#include "test/core/transport/chaotic_good_legacy/frame_fuzzer.pb.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
|
||||
struct DeterministicBitGen : public std::numeric_limits<uint64_t> { |
||||
using result_type = uint64_t; |
||||
uint64_t operator()() { return 42; } |
||||
}; |
||||
|
||||
template <typename T> |
||||
void AssertRoundTrips(const T& input, FrameType expected_frame_type) { |
||||
FrameHeader hdr = input.MakeHeader(); |
||||
CHECK_EQ(hdr.type, expected_frame_type); |
||||
CHECK_EQ(hdr.payload_connection_id, 0); |
||||
SliceBuffer payload; |
||||
input.SerializePayload(payload); |
||||
CHECK_GE(hdr.payload_length, payload.Length()); |
||||
T output; |
||||
auto deser = output.Deserialize(hdr, std::move(payload)); |
||||
CHECK_OK(deser); |
||||
CHECK_EQ(input.ToString(), output.ToString()); |
||||
} |
||||
|
||||
template <typename T> |
||||
void FinishParseAndChecks(const FrameHeader& header, SliceBuffer payload) { |
||||
T parsed; |
||||
ExecCtx exec_ctx; // Initialized to get this_cpu() info in global_stat().
|
||||
auto deser = parsed.Deserialize(header, std::move(payload)); |
||||
if (!deser.ok()) return; |
||||
AssertRoundTrips(parsed, header.type); |
||||
} |
||||
|
||||
void Run(const frame_fuzzer::Test& test) { |
||||
if (test.header().size() != FrameHeader::kFrameHeaderSize) return; |
||||
auto r = FrameHeader::Parse( |
||||
reinterpret_cast<const uint8_t*>(test.header().data())); |
||||
if (!r.ok()) return; |
||||
if (test.payload().size() != r->payload_length) return; |
||||
auto arena = SimpleArenaAllocator()->MakeArena(); |
||||
TestContext<Arena> ctx(arena.get()); |
||||
SliceBuffer payload( |
||||
Slice::FromCopiedBuffer(test.payload().data(), test.payload().size())); |
||||
switch (r->type) { |
||||
default: |
||||
return; // We don't know how to parse this frame type.
|
||||
case FrameType::kSettings: |
||||
FinishParseAndChecks<SettingsFrame>(*r, std::move(payload)); |
||||
break; |
||||
case FrameType::kClientInitialMetadata: |
||||
FinishParseAndChecks<ClientInitialMetadataFrame>(*r, std::move(payload)); |
||||
break; |
||||
case FrameType::kClientEndOfStream: |
||||
FinishParseAndChecks<ClientEndOfStream>(*r, std::move(payload)); |
||||
break; |
||||
case FrameType::kServerInitialMetadata: |
||||
FinishParseAndChecks<ServerInitialMetadataFrame>(*r, std::move(payload)); |
||||
break; |
||||
case FrameType::kServerTrailingMetadata: |
||||
FinishParseAndChecks<ServerTrailingMetadataFrame>(*r, std::move(payload)); |
||||
break; |
||||
case FrameType::kMessage: |
||||
FinishParseAndChecks<MessageFrame>(*r, std::move(payload)); |
||||
break; |
||||
case FrameType::kCancel: |
||||
FinishParseAndChecks<CancelFrame>(*r, std::move(payload)); |
||||
break; |
||||
} |
||||
} |
||||
FUZZ_TEST(FrameFuzzer, Run); |
||||
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,22 @@ |
||||
// Copyright 2021 gRPC authors. |
||||
// |
||||
// Licensed under the Apache License, Version 2.0 (the "License"); |
||||
// you may not use this file except in compliance with the License. |
||||
// You may obtain a copy of the License at |
||||
// |
||||
// http://www.apache.org/licenses/LICENSE-2.0 |
||||
// |
||||
// Unless required by applicable law or agreed to in writing, software |
||||
// distributed under the License is distributed on an "AS IS" BASIS, |
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
// See the License for the specific language governing permissions and |
||||
// limitations under the License. |
||||
|
||||
syntax = "proto3"; |
||||
|
||||
package frame_fuzzer; |
||||
|
||||
message Test { |
||||
bytes header = 1; |
||||
bytes payload = 2; |
||||
} |
@ -0,0 +1,35 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include <stdint.h> |
||||
#include <stdlib.h> |
||||
#include <string.h> |
||||
|
||||
#include "absl/status/statusor.h" |
||||
#include "absl/strings/escaping.h" |
||||
#include "fuzztest/fuzztest.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
|
||||
using grpc_core::chaotic_good_legacy::FrameHeader; |
||||
using HeaderBuffer = std::array<uint8_t, FrameHeader::kFrameHeaderSize>; |
||||
|
||||
void RoundTrips(HeaderBuffer buffer) { |
||||
auto r = FrameHeader::Parse(buffer.data()); |
||||
if (!r.ok()) return; |
||||
HeaderBuffer reserialized; |
||||
r->Serialize(reserialized.data()); |
||||
EXPECT_EQ(buffer, reserialized); |
||||
} |
||||
FUZZ_TEST(FrameHeaderTest, RoundTrips); |
@ -0,0 +1,67 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame_header.h" |
||||
|
||||
#include <cstdint> |
||||
#include <vector> |
||||
|
||||
#include "absl/status/status.h" |
||||
#include "gtest/gtest.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace { |
||||
|
||||
std::vector<uint8_t> Serialize(FrameHeader h) { |
||||
uint8_t buffer[FrameHeader::kFrameHeaderSize]; |
||||
h.Serialize(buffer); |
||||
return std::vector<uint8_t>(buffer, buffer + FrameHeader::kFrameHeaderSize); |
||||
} |
||||
|
||||
absl::StatusOr<FrameHeader> Deserialize(std::vector<uint8_t> data) { |
||||
if (data.size() != FrameHeader::kFrameHeaderSize) { |
||||
return absl::InvalidArgumentError("bad length"); |
||||
} |
||||
return FrameHeader::Parse(data.data()); |
||||
} |
||||
|
||||
TEST(FrameHeaderTest, SimpleSerialize) { |
||||
EXPECT_EQ( |
||||
Serialize(FrameHeader{FrameType::kCancel, 1, 0x01020304, 0x05060708}), |
||||
std::vector<uint8_t>({ |
||||
1, 0, 0xff, 0, // type, payload_connection_id
|
||||
0x04, 0x03, 0x02, 0x01, // stream_id
|
||||
0x08, 0x07, 0x06, 0x05, // payload_length
|
||||
})); |
||||
} |
||||
|
||||
TEST(FrameHeaderTest, SimpleDeserialize) { |
||||
EXPECT_EQ(Deserialize(std::vector<uint8_t>({ |
||||
1, 0, 0xff, 0, // type, payload_connection_id
|
||||
0x04, 0x03, 0x02, 0x01, // stream_id
|
||||
0x08, 0x07, 0x06, 0x05, // payload_length
|
||||
})), |
||||
absl::StatusOr<FrameHeader>( |
||||
FrameHeader{FrameType::kCancel, 1, 0x01020304, 0x05060708})); |
||||
} |
||||
|
||||
} // namespace
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
return RUN_ALL_TESTS(); |
||||
} |
@ -0,0 +1,59 @@ |
||||
// Copyright 2022 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
|
||||
#include <cstdint> |
||||
|
||||
#include "absl/log/check.h" |
||||
#include "absl/random/random.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace { |
||||
|
||||
template <typename T> |
||||
void AssertRoundTrips(const T& input, FrameType expected_frame_type) { |
||||
const auto hdr = input.MakeHeader(); |
||||
EXPECT_EQ(hdr.type, expected_frame_type); |
||||
// Frames should always set connection id 0, though the transport may adjust
|
||||
// it.
|
||||
EXPECT_EQ(hdr.payload_connection_id, 0); |
||||
SliceBuffer output_buffer; |
||||
input.SerializePayload(output_buffer); |
||||
EXPECT_EQ(hdr.payload_length, output_buffer.Length()); |
||||
T output; |
||||
auto deser = output.Deserialize(hdr, std::move(output_buffer)); |
||||
CHECK_OK(deser); |
||||
CHECK_EQ(output.ToString(), input.ToString()); |
||||
} |
||||
|
||||
TEST(FrameTest, SettingsFrameRoundTrips) { |
||||
AssertRoundTrips(SettingsFrame{}, FrameType::kSettings); |
||||
} |
||||
|
||||
} // namespace
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
int r = RUN_ALL_TESTS(); |
||||
return r; |
||||
} |
@ -0,0 +1,95 @@ |
||||
// Copyright 2023 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/message_chunker.h" |
||||
|
||||
#include <vector> |
||||
|
||||
#include "fuzztest/fuzztest.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/lib/promise/status_flag.h" |
||||
#include "test/core/promise/poll_matcher.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace { |
||||
|
||||
// One frame for this test is one of the message carrying frame types.
|
||||
using Frame = std::variant<chaotic_good_legacy::BeginMessageFrame, |
||||
chaotic_good_legacy::MessageChunkFrame, |
||||
chaotic_good_legacy::MessageFrame>; |
||||
|
||||
// This type looks like an mpsc for sending frames, but simply accumulates
|
||||
// frames so we can look at them at the end of the test and ensure they're
|
||||
// correct.
|
||||
struct Sender { |
||||
std::vector<Frame> frames; |
||||
Sender() = default; |
||||
Sender(const Sender&) = delete; |
||||
Sender(Sender&&) = delete; |
||||
Sender& operator=(const Sender&) = delete; |
||||
Sender& operator=(Sender&&) = delete; |
||||
auto Send(Frame frame) { |
||||
frames.emplace_back(std::move(frame)); |
||||
return []() -> Poll<bool> { return true; }; |
||||
} |
||||
}; |
||||
|
||||
void MessageChunkerTest(uint32_t max_chunk_size, uint32_t alignment, |
||||
uint32_t stream_id, uint32_t message_flags, |
||||
std::string payload) { |
||||
chaotic_good_legacy::MessageChunker chunker(max_chunk_size, alignment); |
||||
Sender sender; |
||||
EXPECT_THAT(chunker.Send(Arena::MakePooled<Message>( |
||||
SliceBuffer(Slice::FromCopiedString(payload)), |
||||
message_flags), |
||||
stream_id, sender)(), |
||||
IsReady(true)); |
||||
if (max_chunk_size == 0) { |
||||
// No chunking ==> one frame with just a message.
|
||||
EXPECT_EQ(sender.frames.size(), 1); |
||||
auto& f = std::get<chaotic_good_legacy::MessageFrame>(sender.frames[0]); |
||||
EXPECT_EQ(f.message->payload()->JoinIntoString(), payload); |
||||
EXPECT_EQ(f.stream_id, stream_id); |
||||
} else { |
||||
// Chunking ==> we'd better get at least one frame.
|
||||
ASSERT_GE(sender.frames.size(), 1); |
||||
if (sender.frames.size() == 1) { |
||||
// If just one frame, it'd better be one of the old-style message frames.
|
||||
EXPECT_LE(payload.length(), max_chunk_size); |
||||
auto& f = std::get<chaotic_good_legacy::MessageFrame>(sender.frames[0]); |
||||
EXPECT_EQ(f.message->payload()->JoinIntoString(), payload); |
||||
EXPECT_EQ(f.stream_id, stream_id); |
||||
} else { |
||||
// Otherwise we should get a BeginMessage frame followed by a sequence of
|
||||
// MessageChunk frames, in payload order.
|
||||
auto& f0 = |
||||
std::get<chaotic_good_legacy::BeginMessageFrame>(sender.frames[0]); |
||||
EXPECT_EQ(f0.stream_id, stream_id); |
||||
EXPECT_EQ(f0.body.length(), payload.length()); |
||||
std::string received_payload; |
||||
for (size_t i = 1; i < sender.frames.size(); i++) { |
||||
auto& f = |
||||
std::get<chaotic_good_legacy::MessageChunkFrame>(sender.frames[i]); |
||||
EXPECT_LE(f.payload.Length(), max_chunk_size); |
||||
EXPECT_EQ(f.stream_id, stream_id); |
||||
received_payload.append(f.payload.JoinIntoString()); |
||||
} |
||||
EXPECT_EQ(received_payload, payload); |
||||
} |
||||
} |
||||
} |
||||
FUZZ_TEST(MyTestSuite, MessageChunkerTest); |
||||
|
||||
} // namespace
|
||||
} // namespace grpc_core
|
@ -0,0 +1,228 @@ |
||||
// Copyright 2023 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "src/core/ext/transport/chaotic_good_legacy/server_transport.h" |
||||
|
||||
#include <grpc/event_engine/event_engine.h> |
||||
#include <grpc/event_engine/memory_allocator.h> |
||||
#include <grpc/event_engine/slice.h> |
||||
#include <grpc/event_engine/slice_buffer.h> |
||||
#include <grpc/grpc.h> |
||||
#include <grpc/status.h> |
||||
|
||||
#include <algorithm> |
||||
#include <memory> |
||||
#include <optional> |
||||
#include <string> |
||||
#include <vector> |
||||
|
||||
#include "absl/functional/any_invocable.h" |
||||
#include "absl/status/status.h" |
||||
#include "absl/status/statusor.h" |
||||
#include "absl/strings/str_format.h" |
||||
#include "gmock/gmock.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/ext/transport/chaotic_good/chaotic_good_frame.pb.h" |
||||
#include "src/core/lib/iomgr/timer_manager.h" |
||||
#include "src/core/lib/promise/seq.h" |
||||
#include "src/core/lib/resource_quota/arena.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/slice/slice_buffer.h" |
||||
#include "src/core/lib/slice/slice_internal.h" |
||||
#include "src/core/lib/transport/metadata_batch.h" |
||||
#include "src/core/util/ref_counted_ptr.h" |
||||
#include "test/core/event_engine/fuzzing_event_engine/fuzzing_event_engine.h" |
||||
#include "test/core/event_engine/fuzzing_event_engine/fuzzing_event_engine.pb.h" |
||||
#include "test/core/transport/chaotic_good_legacy/transport_test.h" |
||||
#include "test/core/transport/util/mock_promise_endpoint.h" |
||||
|
||||
using testing::_; |
||||
using testing::MockFunction; |
||||
using testing::Return; |
||||
using testing::StrictMock; |
||||
using testing::WithArgs; |
||||
|
||||
using EventEngineSlice = grpc_event_engine::experimental::Slice; |
||||
|
||||
using grpc_core::chaotic_good::testing::MockPromiseEndpoint; |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace testing { |
||||
|
||||
ServerMetadataHandle TestInitialMetadata() { |
||||
auto md = Arena::MakePooledForOverwrite<ServerMetadata>(); |
||||
md->Set(GrpcMessageMetadata(), Slice::FromStaticString("hello")); |
||||
return md; |
||||
} |
||||
|
||||
ServerMetadataHandle TestTrailingMetadata() { |
||||
auto md = Arena::MakePooledForOverwrite<ServerMetadata>(); |
||||
md->Set(GrpcStatusMetadata(), GRPC_STATUS_OK); |
||||
return md; |
||||
} |
||||
|
||||
ChannelArgs MakeChannelArgs( |
||||
std::shared_ptr<grpc_event_engine::experimental::EventEngine> |
||||
event_engine) { |
||||
return CoreConfiguration::Get() |
||||
.channel_args_preconditioning() |
||||
.PreconditionChannelArgs(nullptr) |
||||
.SetObject<grpc_event_engine::experimental::EventEngine>( |
||||
std::move(event_engine)); |
||||
} |
||||
|
||||
template <typename... PromiseEndpoints> |
||||
Config MakeConfig(const ChannelArgs& channel_args, |
||||
PromiseEndpoints... promise_endpoints) { |
||||
Config config(channel_args); |
||||
auto name_endpoint = [i = 0]() mutable { return absl::StrCat(++i); }; |
||||
std::vector<int> this_is_only_here_to_unpack_the_following_statement{ |
||||
(config.ServerAddPendingDataEndpoint( |
||||
ImmediateConnection(name_endpoint(), std::move(promise_endpoints))), |
||||
0)...}; |
||||
return config; |
||||
} |
||||
|
||||
class MockCallDestination : public UnstartedCallDestination { |
||||
public: |
||||
~MockCallDestination() override = default; |
||||
MOCK_METHOD(void, Orphaned, (), (override)); |
||||
MOCK_METHOD(void, StartCall, (UnstartedCallHandler unstarted_call_handler), |
||||
(override)); |
||||
}; |
||||
|
||||
class MockServerConnectionFactory : public ServerConnectionFactory { |
||||
public: |
||||
MOCK_METHOD(PendingConnection, RequestDataConnection, (), (override)); |
||||
void Orphaned() final {} |
||||
}; |
||||
|
||||
TEST_F(TransportTest, ReadAndWriteOneMessage) { |
||||
MockPromiseEndpoint control_endpoint(1); |
||||
MockPromiseEndpoint data_endpoint(2); |
||||
auto server_connection_factory = |
||||
MakeRefCounted<StrictMock<MockServerConnectionFactory>>(); |
||||
auto call_destination = MakeRefCounted<StrictMock<MockCallDestination>>(); |
||||
EXPECT_CALL(*call_destination, Orphaned()).Times(1); |
||||
auto channel_args = MakeChannelArgs(event_engine()); |
||||
auto transport = MakeOrphanable<ChaoticGoodServerTransport>( |
||||
channel_args, std::move(control_endpoint.promise_endpoint), |
||||
MakeConfig(channel_args, std::move(data_endpoint.promise_endpoint)), |
||||
server_connection_factory); |
||||
const auto server_initial_metadata = |
||||
EncodeProto<chaotic_good_frame::ServerMetadata>("message: 'hello'"); |
||||
const auto server_trailing_metadata = |
||||
EncodeProto<chaotic_good_frame::ServerMetadata>("status: 0"); |
||||
const auto client_initial_metadata = |
||||
EncodeProto<chaotic_good_frame::ClientMetadata>( |
||||
"path: '/demo.Service/Step'"); |
||||
// Once we set the acceptor, expect to read some frames.
|
||||
// We'll return a new request with a payload of "12345678".
|
||||
control_endpoint.ExpectRead( |
||||
{SerializedFrameHeader(FrameType::kClientInitialMetadata, 0, 1, |
||||
client_initial_metadata.length()), |
||||
client_initial_metadata.Copy(), |
||||
SerializedFrameHeader(FrameType::kMessage, 0, 1, 8), |
||||
EventEngineSlice::FromCopiedString("12345678"), |
||||
SerializedFrameHeader(FrameType::kClientEndOfStream, 0, 1, 0)}, |
||||
event_engine().get()); |
||||
// Once that's read we'll create a new call
|
||||
StrictMock<MockFunction<void()>> on_done; |
||||
auto control_address = |
||||
grpc_event_engine::experimental::URIToResolvedAddress("ipv4:1.2.3.4:5678") |
||||
.value(); |
||||
EXPECT_CALL(*control_endpoint.endpoint, GetPeerAddress) |
||||
.WillRepeatedly( |
||||
[&control_address]() -> const grpc_event_engine::experimental:: |
||||
EventEngine::ResolvedAddress& { |
||||
return control_address; |
||||
}); |
||||
EXPECT_CALL(*call_destination, StartCall(_)) |
||||
.WillOnce(WithArgs<0>([&on_done]( |
||||
UnstartedCallHandler unstarted_call_handler) { |
||||
EXPECT_EQ(unstarted_call_handler.UnprocessedClientInitialMetadata() |
||||
.get_pointer(HttpPathMetadata()) |
||||
->as_string_view(), |
||||
"/demo.Service/Step"); |
||||
auto handler = unstarted_call_handler.StartCall(); |
||||
handler.SpawnInfallible("test-io", [&on_done, handler]() mutable { |
||||
return Seq( |
||||
handler.PullClientInitialMetadata(), |
||||
[](ValueOrFailure<ClientMetadataHandle> md) { |
||||
EXPECT_TRUE(md.ok()); |
||||
EXPECT_EQ(md.value() |
||||
->get_pointer(HttpPathMetadata()) |
||||
->as_string_view(), |
||||
"/demo.Service/Step"); |
||||
}, |
||||
[handler]() mutable { return handler.PullMessage(); }, |
||||
[](ClientToServerNextMessage msg) { |
||||
EXPECT_TRUE(msg.ok()); |
||||
EXPECT_TRUE(msg.has_value()); |
||||
EXPECT_EQ(msg.value().payload()->JoinIntoString(), "12345678"); |
||||
}, |
||||
[handler]() mutable { return handler.PullMessage(); }, |
||||
[](ClientToServerNextMessage msg) { |
||||
EXPECT_TRUE(msg.ok()); |
||||
EXPECT_FALSE(msg.has_value()); |
||||
}, |
||||
[handler]() mutable { |
||||
return handler.PushServerInitialMetadata(TestInitialMetadata()); |
||||
}, |
||||
[handler]() mutable { |
||||
return handler.PushMessage(Arena::MakePooled<Message>( |
||||
SliceBuffer(Slice::FromCopiedString("87654321")), 0)); |
||||
}, |
||||
[handler, &on_done]() mutable { |
||||
handler.PushServerTrailingMetadata(TestTrailingMetadata()); |
||||
on_done.Call(); |
||||
}); |
||||
}); |
||||
})); |
||||
transport->SetCallDestination(call_destination); |
||||
EXPECT_CALL(on_done, Call()); |
||||
EXPECT_CALL(*control_endpoint.endpoint, Read) |
||||
.InSequence(control_endpoint.read_sequence) |
||||
.WillOnce(Return(false)); |
||||
control_endpoint.ExpectWrite( |
||||
{SerializedFrameHeader(FrameType::kServerInitialMetadata, 0, 1, |
||||
server_initial_metadata.length()), |
||||
server_initial_metadata.Copy(), |
||||
SerializedFrameHeader(FrameType::kMessage, 0, 1, 8), |
||||
EventEngineSlice::FromCopiedString("87654321"), |
||||
SerializedFrameHeader(FrameType::kServerTrailingMetadata, 0, 1, |
||||
server_trailing_metadata.length()), |
||||
server_trailing_metadata.Copy()}, |
||||
nullptr); |
||||
// Wait until ClientTransport's internal activities to finish.
|
||||
event_engine()->TickUntilIdle(); |
||||
::testing::Mock::VerifyAndClearExpectations(control_endpoint.endpoint); |
||||
::testing::Mock::VerifyAndClearExpectations(data_endpoint.endpoint); |
||||
event_engine()->UnsetGlobalHooks(); |
||||
} |
||||
|
||||
} // namespace testing
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
// Must call to create default EventEngine.
|
||||
grpc_init(); |
||||
int ret = RUN_ALL_TESTS(); |
||||
grpc_shutdown(); |
||||
return ret; |
||||
} |
@ -0,0 +1,50 @@ |
||||
// Copyright 2023 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "test/core/transport/chaotic_good_legacy/transport_test.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace testing { |
||||
|
||||
grpc_event_engine::experimental::Slice SerializedFrameHeader( |
||||
FrameType type, uint16_t payload_connection_id, uint32_t stream_id, |
||||
uint32_t payload_length) { |
||||
uint8_t buffer[FrameHeader::kFrameHeaderSize] = { |
||||
static_cast<uint8_t>(payload_connection_id), |
||||
static_cast<uint8_t>(payload_connection_id >> 16), |
||||
static_cast<uint8_t>(type), |
||||
0, |
||||
static_cast<uint8_t>(stream_id), |
||||
static_cast<uint8_t>(stream_id >> 8), |
||||
static_cast<uint8_t>(stream_id >> 16), |
||||
static_cast<uint8_t>(stream_id >> 24), |
||||
static_cast<uint8_t>(payload_length), |
||||
static_cast<uint8_t>(payload_length >> 8), |
||||
static_cast<uint8_t>(payload_length >> 16), |
||||
static_cast<uint8_t>(payload_length >> 24), |
||||
}; |
||||
return grpc_event_engine::experimental::Slice::FromCopiedBuffer( |
||||
buffer, sizeof(buffer)); |
||||
} |
||||
|
||||
grpc_event_engine::experimental::Slice Zeros(uint32_t length) { |
||||
std::string zeros(length, 0); |
||||
return grpc_event_engine::experimental::Slice::FromCopiedBuffer(zeros.data(), |
||||
length); |
||||
} |
||||
|
||||
} // namespace testing
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
@ -0,0 +1,96 @@ |
||||
// Copyright 2023 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_TEST_CORE_TRANSPORT_CHAOTIC_GOOD_LEGACY_TRANSPORT_TEST_H |
||||
#define GRPC_TEST_CORE_TRANSPORT_CHAOTIC_GOOD_LEGACY_TRANSPORT_TEST_H |
||||
|
||||
#include <google/protobuf/text_format.h> |
||||
|
||||
#include "gmock/gmock.h" |
||||
#include "gtest/gtest.h" |
||||
#include "src/core/ext/transport/chaotic_good_legacy/frame.h" |
||||
#include "src/core/lib/event_engine/event_engine_context.h" |
||||
#include "src/core/lib/iomgr/timer_manager.h" |
||||
#include "src/core/lib/resource_quota/memory_quota.h" |
||||
#include "src/core/lib/resource_quota/resource_quota.h" |
||||
#include "src/core/lib/transport/call_arena_allocator.h" |
||||
#include "src/core/lib/transport/call_spine.h" |
||||
#include "test/core/event_engine/fuzzing_event_engine/fuzzing_event_engine.h" |
||||
#include "test/core/event_engine/fuzzing_event_engine/fuzzing_event_engine.pb.h" |
||||
|
||||
namespace grpc_core { |
||||
namespace chaotic_good_legacy { |
||||
namespace testing { |
||||
|
||||
class TransportTest : public ::testing::Test { |
||||
protected: |
||||
const std::shared_ptr<grpc_event_engine::experimental::FuzzingEventEngine>& |
||||
event_engine() { |
||||
return event_engine_; |
||||
} |
||||
|
||||
RefCountedPtr<Arena> MakeArena() { |
||||
auto arena = call_arena_allocator_->MakeArena(); |
||||
arena->SetContext<grpc_event_engine::experimental::EventEngine>( |
||||
event_engine_.get()); |
||||
return arena; |
||||
} |
||||
|
||||
RefCountedPtr<CallArenaAllocator> call_arena_allocator() { |
||||
return call_arena_allocator_; |
||||
} |
||||
|
||||
auto MakeCall(ClientMetadataHandle client_initial_metadata) { |
||||
return MakeCallPair(std::move(client_initial_metadata), MakeArena()); |
||||
} |
||||
|
||||
private: |
||||
std::shared_ptr<grpc_event_engine::experimental::FuzzingEventEngine> |
||||
event_engine_{ |
||||
std::make_shared<grpc_event_engine::experimental::FuzzingEventEngine>( |
||||
[]() { |
||||
grpc_timer_manager_set_threading(false); |
||||
grpc_event_engine::experimental::FuzzingEventEngine::Options |
||||
options; |
||||
return options; |
||||
}(), |
||||
fuzzing_event_engine::Actions())}; |
||||
RefCountedPtr<CallArenaAllocator> call_arena_allocator_{ |
||||
MakeRefCounted<CallArenaAllocator>( |
||||
MakeResourceQuota("test-quota") |
||||
->memory_quota() |
||||
->CreateMemoryAllocator("test-allocator"), |
||||
1024)}; |
||||
}; |
||||
|
||||
grpc_event_engine::experimental::Slice SerializedFrameHeader( |
||||
FrameType type, uint16_t payload_connection_id, uint32_t stream_id, |
||||
uint32_t payload_length); |
||||
|
||||
grpc_event_engine::experimental::Slice Zeros(uint32_t length); |
||||
|
||||
template <typename T> |
||||
grpc_event_engine::experimental::Slice EncodeProto(const std::string& fields) { |
||||
T msg; |
||||
CHECK(google::protobuf::TextFormat::ParseFromString(fields, &msg)); |
||||
std::string out; |
||||
CHECK(msg.SerializeToString(&out)); |
||||
return grpc_event_engine::experimental::Slice::FromCopiedString(out); |
||||
} |
||||
|
||||
} // namespace testing
|
||||
} // namespace chaotic_good_legacy
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_TEST_CORE_TRANSPORT_CHAOTIC_GOOD_LEGACY_TRANSPORT_TEST_H
|
Loading…
Reference in new issue