"try_seq",
"useful",
":chttp2_flow_control",
+ ":slice_buffer",
"//:chttp2_frame",
"//:gpr_platform",
"//:promise",
#include "src/core/ext/transport/chttp2/transport/frame.h"
#include "src/core/ext/transport/chttp2/transport/header_assembler.h"
#include "src/core/ext/transport/chttp2/transport/http2_settings.h"
-#include "src/core/ext/transport/chttp2/transport/http2_settings_manager.h"
#include "src/core/ext/transport/chttp2/transport/http2_status.h"
#include "src/core/ext/transport/chttp2/transport/http2_transport.h"
#include "src/core/ext/transport/chttp2/transport/http2_ztrace_collector.h"
}
auto Http2ClientTransport::WaitForSettingsTimeoutOnDone() {
- // TODO(tjagtap) : [PH2][P1][Settings] : Handle Transport Close case.
- // TODO(tjagtap) : [PH2][P1][Settings] Move this out of the transport into a
- // Settings class.
return [self = RefAsSubclass<Http2ClientTransport>()](absl::Status status) {
if (!status.ok()) {
GRPC_UNUSED absl::Status result = self->HandleError(
#include "src/core/ext/transport/chttp2/transport/goaway.h"
#include "src/core/ext/transport/chttp2/transport/hpack_encoder.h"
#include "src/core/ext/transport/chttp2/transport/hpack_parser.h"
-#include "src/core/ext/transport/chttp2/transport/http2_settings_manager.h"
#include "src/core/ext/transport/chttp2/transport/http2_settings_promises.h"
#include "src/core/ext/transport/chttp2/transport/http2_status.h"
#include "src/core/ext/transport/chttp2/transport/http2_transport.h"
return frame;
}
-uint32_t Http2SettingsManager::MaybeSendAck() {
- return std::exchange(num_acks_to_send_, 0);
-}
-
bool Http2SettingsManager::AckLastSend() {
if (update_state_ != UpdateState::kSending) return false;
update_state_ = UpdateState::kIdle;
// This function is not idempotent.
std::optional<Http2SettingsFrame> MaybeSendUpdate();
- // Returns 0 if we don't need to send a SETTINGS ACK frame to the peer.
- // Returns n>0 if we need to send n SETTINGS ACK frames to the peer.
- // Transport MUST send one SETTINGS ACK frame for each count returned by this
- // function to the peer.
- // This function is not idempotent.
- uint32_t MaybeSendAck();
- void OnSettingsReceived() { ++num_acks_to_send_; }
-
// To be called from a promise based HTTP2 transport only
http2::Http2ErrorCode ApplyIncomingSettings(
const std::vector<Http2SettingsFrame::Setting>& settings) {
Http2Settings local_;
Http2Settings sent_;
Http2Settings acked_;
-
- // Number of incoming SETTINGS frames that we have received but not ACKed yet.
- uint32_t num_acks_to_send_ = 0;
};
} // namespace grpc_core
#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/slice/slice_buffer.h"
#include "src/core/util/grpc_check.h"
#include "src/core/util/ref_counted.h"
#include "src/core/util/time.h"
// receive and process the SETTINGS ACK.
class SettingsPromiseManager : public RefCounted<SettingsPromiseManager> {
// TODO(tjagtap) [PH2][P1][Settings] : Add new DCHECKs
- // TODO(tjagtap) [PH2][P1][Settings] : Refactor full class
public:
SettingsPromiseManager() = default;
// Not copyable, movable or assignable.
// Buffered to apply settings at start of next write cycle, only after
// SETTINGS ACK is written to the endpoint.
void BufferPeerSettings(std::vector<Http2SettingsFrame::Setting>&& settings) {
- settings_.OnSettingsReceived();
+ ++num_acks_to_send_;
pending_peer_settings_.reserve(pending_peer_settings_.size() +
settings.size());
pending_peer_settings_.insert(pending_peer_settings_.end(),
WillSendSettings();
}
}
- const uint32_t num_acks = settings_.MaybeSendAck();
- if (num_acks > 0) {
- std::vector<Http2Frame> ack_frames(num_acks);
- for (uint32_t i = 0; i < num_acks; ++i) {
+ if (num_acks_to_send_ > 0) {
+ GRPC_SETTINGS_TIMEOUT_DLOG << "Sending " << num_acks_to_send_
+ << " settings ACK frames";
+ std::vector<Http2Frame> ack_frames(num_acks_to_send_);
+ for (uint32_t i = 0; i < num_acks_to_send_; ++i) {
ack_frames[i] = Http2SettingsFrame{true, {}};
}
Serialize(absl::MakeSpan(ack_frames), output_buf);
- GRPC_SETTINGS_TIMEOUT_DLOG << "Sending " << num_acks
- << " settings ACK frames";
+ num_acks_to_send_ = 0;
}
}
//////////////////////////////////////////////////////////////////////////////
// Data Members for SETTINGS being received from the peer.
std::vector<Http2SettingsFrame::Setting> pending_peer_settings_;
+ // Number of incoming SETTINGS frames that we have received but not ACKed yet.
+ uint32_t num_acks_to_send_ = 0;
};
} // namespace grpc_core
"absl/log:log",
"absl/status",
"absl/strings",
+ "absl/types:span",
"gtest",
],
uses_polling = False,
"//:gpr",
"//:grpc",
"//:orphanable",
+ "//:ref_counted_ptr",
"//src/core:1999",
"//src/core:arena",
"//src/core:call_spine",
"//src/core:channel_args",
+ "//src/core:chttp2_flow_control",
"//src/core:default_event_engine",
"//src/core:http2_client_transport",
"//src/core:http2_settings",
"//src/core:notification",
"//src/core:poll",
"//src/core:sleep",
+ "//src/core:slice",
+ "//src/core:slice_buffer",
"//src/core:time",
"//src/core:transport_common",
"//src/core:try_join",
Http2ErrorCode::kConnectError);
}
-TEST(Http2SettingsManagerTest, NoAckNeededInitially) {
- // No ACK should be sent initially.
- Http2SettingsManager settings_manager;
- EXPECT_EQ(settings_manager.MaybeSendAck(), 0u);
-}
-
-TEST(Http2SettingsManagerTest, AckNeededAfterEmptySettings) {
- // If we receive an empty SETTINGS frame, we should send an ACK.
- Http2SettingsManager settings_manager;
- settings_manager.OnSettingsReceived();
- EXPECT_EQ(settings_manager.MaybeSendAck(), 1u);
- EXPECT_EQ(settings_manager.MaybeSendAck(), 0u);
-}
-
-TEST(Http2SettingsManagerTest, AckNeededAfterValidSettings) {
- // If we receive a valid SETTINGS frame, we should send an ACK.
- Http2SettingsManager settings_manager;
- std::vector<Http2SettingsFrame::Setting> settings = {
- {Http2Settings::kHeaderTableSizeWireId, 1000},
- {Http2Settings::kMaxConcurrentStreamsWireId, 200}};
- settings_manager.OnSettingsReceived();
- EXPECT_EQ(settings_manager.MaybeSendAck(), 1u);
- EXPECT_EQ(settings_manager.MaybeSendAck(), 0u);
-}
-
-TEST(Http2SettingsManagerTest, MultipleAcksNeeded) {
- // If we receive multiple SETTINGS frames before sending an ACK,
- // we should send an ACK for each.
- Http2SettingsManager settings_manager;
- settings_manager.OnSettingsReceived();
- settings_manager.OnSettingsReceived();
- EXPECT_EQ(settings_manager.MaybeSendAck(), 2u);
- EXPECT_EQ(settings_manager.MaybeSendAck(), 0u);
-}
-
} // namespace grpc_core
int main(int argc, char** argv) {
EXPECT_EQ(settings2.allow_security_frame(), false);
}
-TEST(SettingsPromiseManagerTest1, MaybeGetSettingsAndSettingsAckFramesIdle) {
- // Tests that in idle state, first call to
- // MaybeGetSettingsAndSettingsAckFrames sends initial settings, and second
- // call does nothing.
- chttp2::TransportFlowControl transport_flow_control(
- /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
- /*memory_owner=*/nullptr);
- RefCountedPtr<SettingsPromiseManager> timeout_manager =
- MakeRefCounted<SettingsPromiseManager>();
- SliceBuffer output_buf;
- // We add "hello" to output_buf to ensure that
- // MaybeGetSettingsAndSettingsAckFrames appends to it and does not overwrite
- // it, i.e. the original contents of output_buf are not erased.
- output_buf.Append(Slice::FromCopiedString("hello"));
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- timeout_manager->TestOnlyTimeoutWaiterSpawned();
- ASSERT_THAT(output_buf.JoinIntoString(), ::testing::StartsWith("hello"));
- EXPECT_GT(output_buf.Length(), 5);
- output_buf.Clear();
- output_buf.Append(Slice::FromCopiedString("hello"));
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- EXPECT_EQ(output_buf.Length(), 5);
- EXPECT_EQ(output_buf.JoinIntoString(), "hello");
-}
-
-TEST(SettingsPromiseManagerTest1,
- MaybeGetSettingsAndSettingsAckFramesMultipleAcks) {
- // If multiple settings frames are received then multiple ACKs should be sent.
- chttp2::TransportFlowControl transport_flow_control(
- /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
- /*memory_owner=*/nullptr);
- RefCountedPtr<SettingsPromiseManager> timeout_manager =
- MakeRefCounted<SettingsPromiseManager>();
- SliceBuffer output_buf;
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- timeout_manager->TestOnlyTimeoutWaiterSpawned();
- output_buf.Clear();
- output_buf.Append(Slice::FromCopiedString("hello"));
- for (int i = 0; i < 5; ++i) {
- timeout_manager->BufferPeerSettings(
- {{Http2Settings::kMaxConcurrentStreamsWireId, 100}});
- }
-
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
-
- SliceBuffer expected_buf;
- expected_buf.Append(Slice::FromCopiedString("hello"));
- for (int i = 0; i < 5; ++i) {
- Http2SettingsFrame settings;
- settings.ack = true;
- Http2Frame frame(settings);
- Serialize(absl::Span<Http2Frame>(&frame, 1), expected_buf);
- }
- EXPECT_EQ(output_buf.Length(), expected_buf.Length());
- EXPECT_EQ(output_buf.JoinIntoString(), expected_buf.JoinIntoString());
-}
-
-TEST(SettingsPromiseManagerTest1,
- MaybeGetSettingsAndSettingsAckFramesAfterAckAndChange) {
- // Tests that after initial settings are sent and ACKed, no frame is sent. If
- // settings are changed, a new SETTINGS frame with diff is sent.
- chttp2::TransportFlowControl transport_flow_control(
- /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
- /*memory_owner=*/nullptr);
- RefCountedPtr<SettingsPromiseManager> timeout_manager =
- MakeRefCounted<SettingsPromiseManager>();
- const uint32_t kSetMaxFrameSize = 16385;
- SliceBuffer output_buf;
- // We add "hello" to output_buf to ensure that
- // MaybeGetSettingsAndSettingsAckFrames appends to it and does not overwrite
- // it, i.e. the original contents of output_buf are not erased.
- output_buf.Append(Slice::FromCopiedString("hello"));
- // Initial settings
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- timeout_manager->TestOnlyTimeoutWaiterSpawned();
- ASSERT_THAT(output_buf.JoinIntoString(), ::testing::StartsWith("hello"));
- EXPECT_GT(output_buf.Length(), 5);
- // Ack settings
- EXPECT_TRUE(timeout_manager->OnSettingsAckReceived());
- output_buf.Clear();
- output_buf.Append(Slice::FromCopiedString("hello"));
- // No changes - no frames
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- EXPECT_EQ(output_buf.Length(), 5);
- EXPECT_EQ(output_buf.JoinIntoString(), "hello");
- output_buf.Clear();
- // Change settings
- timeout_manager->mutable_local().SetMaxFrameSize(kSetMaxFrameSize);
- output_buf.Append(Slice::FromCopiedString("hello"));
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- timeout_manager->TestOnlyTimeoutWaiterSpawned();
- // Check frame
- Http2SettingsFrame expected_settings;
- expected_settings.ack = false;
- expected_settings.settings.push_back(
- {Http2Settings::kMaxFrameSizeWireId, kSetMaxFrameSize});
- Http2Frame expected_frame(expected_settings);
- SliceBuffer expected_buf;
- expected_buf.Append(Slice::FromCopiedString("hello"));
- Serialize(absl::Span<Http2Frame>(&expected_frame, 1), expected_buf);
- EXPECT_EQ(output_buf.Length(), expected_buf.Length());
- EXPECT_EQ(output_buf.JoinIntoString(), expected_buf.JoinIntoString());
-
- // We set SetMaxFrameSize to the same value as previous value.
- // The Diff will be zero, in this case a new SETTINGS frame must not be sent.
- timeout_manager->mutable_local().SetMaxFrameSize(kSetMaxFrameSize);
- output_buf.Clear();
- output_buf.Append(Slice::FromCopiedString("hello"));
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- EXPECT_EQ(output_buf.Length(), 5);
- EXPECT_EQ(output_buf.JoinIntoString(), "hello");
-}
-
-TEST(SettingsPromiseManagerTest1, MaybeGetSettingsAndSettingsAckFramesWithAck) {
- // Tests that if we need to send initial settings and also ACK received
- // settings, both frames are sent.
- chttp2::TransportFlowControl transport_flow_control(
- /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
- /*memory_owner=*/nullptr);
- RefCountedPtr<SettingsPromiseManager> timeout_manager =
- MakeRefCounted<SettingsPromiseManager>();
- SliceBuffer output_buf;
- // We add "hello" to output_buf to ensure that
- // MaybeGetSettingsAndSettingsAckFrames appends to it and does not overwrite
- // it, i.e. the original contents of output_buf are not erased.
- output_buf.Append(Slice::FromCopiedString("hello"));
- timeout_manager->BufferPeerSettings(
- {{Http2Settings::kMaxConcurrentStreamsWireId, 100}});
- timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
- output_buf);
- EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
- timeout_manager->TestOnlyTimeoutWaiterSpawned();
- Http2SettingsFrame expected_settings;
- expected_settings.ack = false;
- timeout_manager->mutable_local().Diff(
- true, Http2Settings(), [&](uint16_t key, uint32_t value) {
- expected_settings.settings.push_back({key, value});
- });
- Http2SettingsFrame expected_settings_ack;
- expected_settings_ack.ack = true;
- SliceBuffer expected_buf;
- expected_buf.Append(Slice::FromCopiedString("hello"));
- std::vector<Http2Frame> frames;
- frames.emplace_back(expected_settings);
- frames.emplace_back(expected_settings_ack);
- Serialize(absl::MakeSpan(frames), expected_buf);
- EXPECT_EQ(output_buf.Length(), expected_buf.Length());
- EXPECT_EQ(output_buf.JoinIntoString(), expected_buf.JoinIntoString());
-}
-
///////////////////////////////////////////////////////////////////////////////
// Flow control helpers tests
#include <memory>
#include <tuple>
#include <utility>
+#include <vector>
+#include "src/core/ext/transport/chttp2/transport/flow_control.h"
#include "src/core/ext/transport/chttp2/transport/frame.h"
+#include "src/core/ext/transport/chttp2/transport/http2_settings.h"
#include "src/core/ext/transport/chttp2/transport/http2_settings_promises.h"
#include "src/core/lib/channel/channel_args.h"
#include "src/core/lib/event_engine/default_event_engine.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/slice/slice.h"
+#include "src/core/lib/slice/slice_buffer.h"
#include "src/core/util/notification.h"
#include "src/core/util/orphanable.h"
+#include "src/core/util/ref_counted_ptr.h"
#include "src/core/util/time.h"
+#include "gmock/gmock.h"
#include "gtest/gtest.h"
#include "absl/log/log.h"
#include "absl/status/status.h"
#include "absl/strings/string_view.h"
+#include "absl/types/span.h"
namespace grpc_core {
namespace http2 {
notification2.WaitForNotification();
}
+TEST(SettingsPromiseManagerTest1, MaybeGetSettingsAndSettingsAckFramesIdle) {
+ // Tests that in idle state, first call to
+ // MaybeGetSettingsAndSettingsAckFrames sends initial settings, and second
+ // call does nothing.
+ chttp2::TransportFlowControl transport_flow_control(
+ /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
+ /*memory_owner=*/nullptr);
+ RefCountedPtr<SettingsPromiseManager> timeout_manager =
+ MakeRefCounted<SettingsPromiseManager>();
+ SliceBuffer output_buf;
+ // We add "hello" to output_buf to ensure that
+ // MaybeGetSettingsAndSettingsAckFrames appends to it and does not overwrite
+ // it, i.e. the original contents of output_buf are not erased.
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ timeout_manager->TestOnlyTimeoutWaiterSpawned();
+ ASSERT_THAT(output_buf.JoinIntoString(), ::testing::StartsWith("hello"));
+ EXPECT_GT(output_buf.Length(), 5);
+ output_buf.Clear();
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ EXPECT_EQ(output_buf.Length(), 5);
+ EXPECT_EQ(output_buf.JoinIntoString(), "hello");
+}
+
+TEST(SettingsPromiseManagerTest1,
+ MaybeGetSettingsAndSettingsAckFramesMultipleAcks) {
+ // If multiple settings frames are received then multiple ACKs should be sent.
+ chttp2::TransportFlowControl transport_flow_control(
+ /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
+ /*memory_owner=*/nullptr);
+ RefCountedPtr<SettingsPromiseManager> timeout_manager =
+ MakeRefCounted<SettingsPromiseManager>();
+ SliceBuffer output_buf;
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ timeout_manager->TestOnlyTimeoutWaiterSpawned();
+ output_buf.Clear();
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ for (int i = 0; i < 5; ++i) {
+ timeout_manager->BufferPeerSettings(
+ {{Http2Settings::kMaxConcurrentStreamsWireId, 100}});
+ }
+
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+
+ SliceBuffer expected_buf;
+ expected_buf.Append(Slice::FromCopiedString("hello"));
+ for (int i = 0; i < 5; ++i) {
+ Http2SettingsFrame settings;
+ settings.ack = true;
+ Http2Frame frame(settings);
+ Serialize(absl::Span<Http2Frame>(&frame, 1), expected_buf);
+ }
+ EXPECT_EQ(output_buf.Length(), expected_buf.Length());
+ EXPECT_EQ(output_buf.JoinIntoString(), expected_buf.JoinIntoString());
+}
+
+TEST(SettingsPromiseManagerTest1,
+ MaybeGetSettingsAndSettingsAckFramesAfterAckAndChange) {
+ // Tests that after initial settings are sent and ACKed, no frame is sent. If
+ // settings are changed, a new SETTINGS frame with diff is sent.
+ chttp2::TransportFlowControl transport_flow_control(
+ /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
+ /*memory_owner=*/nullptr);
+ RefCountedPtr<SettingsPromiseManager> timeout_manager =
+ MakeRefCounted<SettingsPromiseManager>();
+ const uint32_t kSetMaxFrameSize = 16385;
+ SliceBuffer output_buf;
+ // We add "hello" to output_buf to ensure that
+ // MaybeGetSettingsAndSettingsAckFrames appends to it and does not overwrite
+ // it, i.e. the original contents of output_buf are not erased.
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ // Initial settings
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ timeout_manager->TestOnlyTimeoutWaiterSpawned();
+ ASSERT_THAT(output_buf.JoinIntoString(), ::testing::StartsWith("hello"));
+ EXPECT_GT(output_buf.Length(), 5);
+ // Ack settings
+ EXPECT_TRUE(timeout_manager->OnSettingsAckReceived());
+ output_buf.Clear();
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ // No changes - no frames
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ EXPECT_EQ(output_buf.Length(), 5);
+ EXPECT_EQ(output_buf.JoinIntoString(), "hello");
+ output_buf.Clear();
+ // Change settings
+ timeout_manager->mutable_local().SetMaxFrameSize(kSetMaxFrameSize);
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ timeout_manager->TestOnlyTimeoutWaiterSpawned();
+ // Check frame
+ Http2SettingsFrame expected_settings;
+ expected_settings.ack = false;
+ expected_settings.settings.push_back(
+ {Http2Settings::kMaxFrameSizeWireId, kSetMaxFrameSize});
+ Http2Frame expected_frame(expected_settings);
+ SliceBuffer expected_buf;
+ expected_buf.Append(Slice::FromCopiedString("hello"));
+ Serialize(absl::Span<Http2Frame>(&expected_frame, 1), expected_buf);
+ EXPECT_EQ(output_buf.Length(), expected_buf.Length());
+ EXPECT_EQ(output_buf.JoinIntoString(), expected_buf.JoinIntoString());
+
+ // We set SetMaxFrameSize to the same value as previous value.
+ // The Diff will be zero, in this case a new SETTINGS frame must not be sent.
+ timeout_manager->mutable_local().SetMaxFrameSize(kSetMaxFrameSize);
+ output_buf.Clear();
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_FALSE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ EXPECT_EQ(output_buf.Length(), 5);
+ EXPECT_EQ(output_buf.JoinIntoString(), "hello");
+}
+
+TEST(SettingsPromiseManagerTest1, MaybeGetSettingsAndSettingsAckFramesWithAck) {
+ // Tests that if we need to send initial settings and also ACK received
+ // settings, both frames are sent.
+ chttp2::TransportFlowControl transport_flow_control(
+ /*name=*/"TestFlowControl", /*enable_bdp_probe=*/false,
+ /*memory_owner=*/nullptr);
+ RefCountedPtr<SettingsPromiseManager> timeout_manager =
+ MakeRefCounted<SettingsPromiseManager>();
+ SliceBuffer output_buf;
+ // We add "hello" to output_buf to ensure that
+ // MaybeGetSettingsAndSettingsAckFrames appends to it and does not overwrite
+ // it, i.e. the original contents of output_buf are not erased.
+ output_buf.Append(Slice::FromCopiedString("hello"));
+ timeout_manager->BufferPeerSettings(
+ {{Http2Settings::kMaxConcurrentStreamsWireId, 100}});
+ timeout_manager->MaybeGetSettingsAndSettingsAckFrames(transport_flow_control,
+ output_buf);
+ EXPECT_TRUE(timeout_manager->ShouldSpawnWaitForSettingsTimeout());
+ timeout_manager->TestOnlyTimeoutWaiterSpawned();
+ Http2SettingsFrame expected_settings;
+ expected_settings.ack = false;
+ timeout_manager->mutable_local().Diff(
+ true, Http2Settings(), [&](uint16_t key, uint32_t value) {
+ expected_settings.settings.push_back({key, value});
+ });
+ Http2SettingsFrame expected_settings_ack;
+ expected_settings_ack.ack = true;
+ SliceBuffer expected_buf;
+ expected_buf.Append(Slice::FromCopiedString("hello"));
+ std::vector<Http2Frame> frames;
+ frames.emplace_back(expected_settings);
+ frames.emplace_back(expected_settings_ack);
+ Serialize(absl::MakeSpan(frames), expected_buf);
+ EXPECT_EQ(output_buf.Length(), expected_buf.Length());
+ EXPECT_EQ(output_buf.JoinIntoString(), expected_buf.JoinIntoString());
+}
+
} // namespace testing
} // namespace http2