diff --git a/api/envoy/config/core/v3/protocol.proto b/api/envoy/config/core/v3/protocol.proto index 74fe641fe3a24..a4d4dec56f02c 100644 --- a/api/envoy/config/core/v3/protocol.proto +++ b/api/envoy/config/core/v3/protocol.proto @@ -473,7 +473,7 @@ message KeepaliveSettings { [(validate.rules).duration = {gte {nanos: 1000000}}]; } -// [#next-free-field: 18] +// [#next-free-field: 19] message Http2ProtocolOptions { option (udpa.annotations.versioning).previous_message_type = "envoy.api.v2.core.Http2ProtocolOptions"; @@ -662,6 +662,14 @@ message Http2ProtocolOptions { // Configure the maximum amount of metadata than can be handled per stream. Defaults to 1 MB. google.protobuf.UInt64Value max_metadata_size = 17; + + // Timeout for graceful HTTP/2 GOAWAY shutdown sequence. + // When graceful shutdown is enabled, Envoy will send an initial GOAWAY frame with + // last_stream_id set to 2^31-1, wait for this timeout period, then send a final + // GOAWAY frame with the actual highest received stream ID. This allows in-flight + // requests to complete gracefully. If set to 0, graceful shutdown is disabled + // and the standard immediate GOAWAY behavior is used. Defaults to 1000ms (1 second). + google.protobuf.Duration graceful_goaway_timeout = 18; } // [#not-implemented-hide:] diff --git a/changelogs/current.yaml b/changelogs/current.yaml index 28f6da57776f9..ba90aa8e114b6 100644 --- a/changelogs/current.yaml +++ b/changelogs/current.yaml @@ -548,5 +548,14 @@ new_features: change: | Added ``setUpstreamOverrideHost`` method to AsyncClient StreamOptions to enable direct host routing that bypasses load balancer selection. +- area: http2 + change: | + Enabled graceful HTTP/2 GOAWAY shutdown by default with a 1-second timeout. The + :ref:`graceful_goaway_timeout ` + field now defaults to 1000ms (1 second) instead of 0 (disabled). This allows in-flight + requests to complete gracefully during connection shutdown by sending an initial GOAWAY + with max stream ID, waiting for the timeout, then sending a final GOAWAY. Set to 0 to + disable graceful shutdown. This feature can be disabled using the runtime guard + ``envoy.reloadable_features.http2_graceful_goaway``. deprecated: diff --git a/envoy/http/codec.h b/envoy/http/codec.h index bc7c45cacf0c3..2bc3cdc6850aa 100644 --- a/envoy/http/codec.h +++ b/envoy/http/codec.h @@ -586,6 +586,12 @@ class Connection { */ virtual void goAway() PURE; + /** + * Indicate graceful "go away" to the remote. For HTTP/2, this implements RFC-compliant + * graceful shutdown. For other protocols, this falls back to regular goAway(). + */ + virtual void goAwayGraceful() { goAway(); } + /** * @return the protocol backing the connection. This can change if for example an HTTP/1.1 * connection gets an HTTP/1.0 request on it. diff --git a/source/common/http/codec_client.h b/source/common/http/codec_client.h index 4638c839d5ea5..0f398ad13fea6 100644 --- a/source/common/http/codec_client.h +++ b/source/common/http/codec_client.h @@ -89,6 +89,12 @@ class CodecClient : protected Logger::Loggable, */ void goAway() { codec_->goAway(); } + /** + * Send a graceful codec level go away indication to the peer. + * For HTTP/2, this implements RFC-compliant graceful shutdown. + */ + void goAwayGraceful() { codec_->goAwayGraceful(); } + /** * @return the underlying connection ID. */ diff --git a/source/common/http/conn_manager_impl.cc b/source/common/http/conn_manager_impl.cc index 4e12fff583b7e..679de65a06099 100644 --- a/source/common/http/conn_manager_impl.cc +++ b/source/common/http/conn_manager_impl.cc @@ -786,7 +786,15 @@ void ConnectionManagerImpl::sendGoAwayAndClose() { if (go_away_sent_) { return; } - codec_->goAway(); + + // Try graceful GOAWAY for HTTP/2 connections, fallback to regular GOAWAY + auto* http2_conn = dynamic_cast(codec_.get()); + if (http2_conn != nullptr) { + http2_conn->goAwayGraceful(); + } else { + codec_->goAway(); + } + go_away_sent_ = true; doConnectionClose(Network::ConnectionCloseType::FlushWriteAndDelay, absl::nullopt, "forced_goaway"); diff --git a/source/common/http/http2/codec_impl.cc b/source/common/http/http2/codec_impl.cc index b3dd8917337d6..1d87dcdf9c3ba 100644 --- a/source/common/http/http2/codec_impl.cc +++ b/source/common/http/http2/codec_impl.cc @@ -22,6 +22,7 @@ #include "source/common/http/headers.h" #include "source/common/http/http2/codec_stats.h" #include "source/common/http/utility.h" +#include "source/common/protobuf/utility.h" #include "source/common/runtime/runtime_features.h" #include "absl/cleanup/cleanup.h" @@ -865,7 +866,7 @@ ConnectionImpl::ConnectionImpl(Network::Connection& connection, CodecStats& stat stream_error_on_invalid_http_messaging_( http2_options.override_stream_error_on_invalid_http_message().value()), protocol_constraints_(stats, http2_options), dispatching_(false), raised_goaway_(false), - random_(random_generator), + graceful_goaway_in_progress_(false), random_(random_generator), last_received_data_time_(connection_.dispatcher().timeSource().monotonicTime()) { if (http2_options.has_use_oghttp2_codec()) { use_oghttp2_library_ = http2_options.use_oghttp2_codec().value(); @@ -890,6 +891,14 @@ ConnectionImpl::ConnectionImpl(Network::Connection& connection, CodecStats& stat // This call schedules the initial interval, with jitter. onKeepaliveResponse(); } + + // Initialize graceful GOAWAY timeout + graceful_goaway_timeout_ = std::chrono::milliseconds( + PROTOBUF_GET_MS_OR_DEFAULT(http2_options, graceful_goaway_timeout, 1000)); + if (graceful_goaway_timeout_.count() > 0) { + graceful_goaway_timer_ = + connection.dispatcher().createTimer([this]() { onGracefulGoAwayTimeout(); }); + } } ConnectionImpl::~ConnectionImpl() { @@ -943,6 +952,18 @@ void ConnectionImpl::onKeepaliveResponseTimeout() { StreamInfo::LocalCloseReasons::get().Http2PingTimeout); } +void ConnectionImpl::onGracefulGoAwayTimeout() { + // Send final GOAWAY with actual highest received stream ID + if (graceful_goaway_in_progress_) { + ENVOY_CONN_LOG(debug, "Graceful GOAWAY timeout reached, sending final GOAWAY", connection_); + graceful_goaway_in_progress_ = false; + adapter_->SubmitGoAway(adapter_->GetHighestReceivedStreamId(), + http2::adapter::Http2ErrorCode::HTTP2_NO_ERROR, ""); + stats_.goaway_sent_.inc(); + sendPendingFramesAndHandleError(); + } +} + bool ConnectionImpl::slowContainsStreamId(int32_t stream_id) const { for (const auto& stream : active_streams_) { if (stream->stream_id_ == stream_id) { @@ -1046,6 +1067,35 @@ void ConnectionImpl::goAway() { } } +void ConnectionImpl::goAwayGraceful() { + // If graceful GOAWAY is disabled by runtime guard, fallback to immediate GOAWAY + if (!Runtime::runtimeFeatureEnabled("envoy.reloadable_features.http2_graceful_goaway")) { + goAway(); + return; + } + + // If graceful GOAWAY is not configured or already in progress, fallback to immediate GOAWAY + if (graceful_goaway_timeout_.count() == 0 || graceful_goaway_in_progress_) { + goAway(); + return; + } + + // Send initial GOAWAY with max stream ID (2^31-1) to signal graceful shutdown + graceful_goaway_in_progress_ = true; + adapter_->SubmitGoAway(0x7FFFFFFF, http2::adapter::Http2ErrorCode::HTTP2_NO_ERROR, ""); + stats_.goaway_graceful_sent_.inc(); + + if (sendPendingFramesAndHandleError()) { + // Intended to check through coverage that this error case is tested + return; + } + + // Start timer for final GOAWAY + if (graceful_goaway_timer_) { + graceful_goaway_timer_->enableTimer(graceful_goaway_timeout_); + } +} + void ConnectionImpl::shutdownNotice() { adapter_->SubmitShutdownNotice(); @@ -2030,7 +2080,8 @@ void ConnectionImpl::dumpState(std::ostream& os, int indent_level) const { << DUMP_MEMBER(max_headers_count_) << DUMP_MEMBER(per_stream_buffer_limit_) << DUMP_MEMBER(allow_metadata_) << DUMP_MEMBER(stream_error_on_invalid_http_messaging_) << DUMP_MEMBER(is_outbound_flood_monitored_control_frame_) << DUMP_MEMBER(dispatching_) - << DUMP_MEMBER(raised_goaway_) << DUMP_MEMBER(pending_deferred_reset_streams_.size()) << '\n'; + << DUMP_MEMBER(raised_goaway_) << DUMP_MEMBER(graceful_goaway_in_progress_) + << DUMP_MEMBER(pending_deferred_reset_streams_.size()) << '\n'; // Dump the protocol constraints DUMP_DETAILS(&protocol_constraints_); @@ -2295,14 +2346,14 @@ Http::Status ServerConnectionImpl::dispatch(Buffer::Instance& data) { RETURN_IF_ERROR(protocol_constraints_.checkOutboundFrameLimits()); if (should_send_go_away_and_close_on_dispatch_ != nullptr && should_send_go_away_and_close_on_dispatch_->shouldShedLoad()) { - ConnectionImpl::goAway(); + ConnectionImpl::goAwayGraceful(); sent_go_away_on_dispatch_ = true; return envoyOverloadError( "Load shed point http2_server_go_away_and_close_on_dispatch triggered"); } if (should_send_go_away_on_dispatch_ != nullptr && !sent_go_away_on_dispatch_ && should_send_go_away_on_dispatch_->shouldShedLoad()) { - ConnectionImpl::goAway(); + ConnectionImpl::goAwayGraceful(); sent_go_away_on_dispatch_ = true; } return ConnectionImpl::dispatch(data); diff --git a/source/common/http/http2/codec_impl.h b/source/common/http/http2/codec_impl.h index ae0757576489d..b22a0d8634128 100644 --- a/source/common/http/http2/codec_impl.h +++ b/source/common/http/http2/codec_impl.h @@ -159,6 +159,7 @@ class ConnectionImpl : public virtual Connection, // NOTE: the `dispatch` method is also overridden in the ServerConnectionImpl class Http::Status dispatch(Buffer::Instance& data) override; void goAway() override; + void goAwayGraceful() override; Protocol protocol() override { return Protocol::Http2; } void shutdownNotice() override; Status protocolErrorForTest(); // Used in tests to simulate errors. @@ -786,6 +787,7 @@ class ConnectionImpl : public virtual Connection, uint32_t padding_length); void onKeepaliveResponse(); void onKeepaliveResponseTimeout(); + void onGracefulGoAwayTimeout(); bool slowContainsStreamId(int32_t stream_id) const; virtual StreamResetReason getMessagingErrorResetReason() const PURE; @@ -798,13 +800,16 @@ class ConnectionImpl : public virtual Connection, std::map pending_deferred_reset_streams_; bool dispatching_ : 1; bool raised_goaway_ : 1; + bool graceful_goaway_in_progress_ : 1; Event::SchedulableCallbackPtr protocol_constraint_violation_callback_; Random::RandomGenerator& random_; MonotonicTime last_received_data_time_{}; Event::TimerPtr keepalive_send_timer_; Event::TimerPtr keepalive_timeout_timer_; + Event::TimerPtr graceful_goaway_timer_; std::chrono::milliseconds keepalive_interval_; std::chrono::milliseconds keepalive_timeout_; + std::chrono::milliseconds graceful_goaway_timeout_; uint32_t keepalive_interval_jitter_percent_; }; diff --git a/source/common/http/http2/codec_stats.h b/source/common/http/http2/codec_stats.h index 2912cd05b1b5c..421b0b0a4add9 100644 --- a/source/common/http/http2/codec_stats.h +++ b/source/common/http/http2/codec_stats.h @@ -16,6 +16,7 @@ namespace Http2 { #define ALL_HTTP2_CODEC_STATS(COUNTER, GAUGE) \ COUNTER(dropped_headers_with_underscores) \ COUNTER(goaway_sent) \ + COUNTER(goaway_graceful_sent) \ COUNTER(header_overflow) \ COUNTER(headers_cb_no_stream) \ COUNTER(inbound_empty_frames_flood) \ diff --git a/source/common/runtime/runtime_features.cc b/source/common/runtime/runtime_features.cc index 132f335726905..bb7f7fed6dea1 100644 --- a/source/common/runtime/runtime_features.cc +++ b/source/common/runtime/runtime_features.cc @@ -48,6 +48,7 @@ RUNTIME_GUARD(envoy_reloadable_features_http1_balsa_allow_cr_or_lf_at_request_st RUNTIME_GUARD(envoy_reloadable_features_http1_balsa_delay_reset); RUNTIME_GUARD(envoy_reloadable_features_http1_balsa_disallow_lone_cr_in_chunk_extension); RUNTIME_GUARD(envoy_reloadable_features_http2_discard_host_header); +RUNTIME_GUARD(envoy_reloadable_features_http2_graceful_goaway); RUNTIME_GUARD(envoy_reloadable_features_http2_propagate_reset_events); RUNTIME_GUARD(envoy_reloadable_features_http2_use_oghttp2); RUNTIME_GUARD(envoy_reloadable_features_http3_remove_empty_cookie); diff --git a/test/common/http/http2/codec_impl_test.cc b/test/common/http/http2/codec_impl_test.cc index a8eecaae1ab15..f8dfdfb10f562 100644 --- a/test/common/http/http2/codec_impl_test.cc +++ b/test/common/http/http2/codec_impl_test.cc @@ -187,9 +187,9 @@ class Http2CodecImplTestFixture { Http2Impl http2_implementation) : client_settings_(client_settings), server_settings_(server_settings), http2_implementation_(http2_implementation) { - // Make sure we explicitly test for stream flush timer creation. - EXPECT_CALL(client_connection_.dispatcher_, createTimer_(_)).Times(0); - EXPECT_CALL(server_connection_.dispatcher_, createTimer_(_)).Times(0); + // Allow graceful GOAWAY timer creation (default 1000ms timeout creates timer) + EXPECT_CALL(client_connection_.dispatcher_, createTimer_(_)).Times(testing::AtMost(1)); + EXPECT_CALL(server_connection_.dispatcher_, createTimer_(_)).Times(testing::AtMost(1)); } virtual ~Http2CodecImplTestFixture() { client_connection_.dispatcher_.clearDeferredDeleteList(); @@ -655,6 +655,175 @@ TEST_P(Http2CodecImplTest, ShutdownNotice) { driveToCompletion(); } +TEST_P(Http2CodecImplTest, GracefulGoAwayBasicFunctionality) { + // Configure graceful GOAWAY timeout of 100ms + server_http2_options_.mutable_graceful_goaway_timeout()->set_seconds(0); + server_http2_options_.mutable_graceful_goaway_timeout()->set_nanos(100 * 1000 * 1000); // 100ms + + auto graceful_goaway_timer = new NiceMock(&server_connection_.dispatcher_); + EXPECT_CALL(*graceful_goaway_timer, enableTimer(std::chrono::milliseconds(100), _)); + + initialize(); + + // Test basic functionality - just verify the method exists and stats work + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_sent").value()); + + // Allow GOAWAY callbacks to be called (these happen automatically when frames are processed) + EXPECT_CALL(client_callbacks_, onGoAway(_)).Times(AtLeast(1)); + + server_->goAwayGraceful(); + driveToCompletion(); + + // Verify graceful GOAWAY was sent (stats should increment) + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + EXPECT_EQ(0, server_stats_store_.counter("http2.goaway_sent").value()); + + // Trigger timeout and final GOAWAY + graceful_goaway_timer->invokeCallback(); + driveToCompletion(); + + // Verify final GOAWAY was sent + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_sent").value()); +} + +TEST_P(Http2CodecImplTest, GracefulGoAwayFallbackWhenTimeoutZero) { + // Configure graceful GOAWAY timeout of 0 (disabled) + server_http2_options_.mutable_graceful_goaway_timeout()->set_seconds(0); + server_http2_options_.mutable_graceful_goaway_timeout()->set_nanos(0); + initialize(); + + // Should fallback to immediate GOAWAY + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_sent").value()); + + EXPECT_CALL(client_callbacks_, onGoAway(_)).Times(AtLeast(1)); + server_->goAwayGraceful(); + driveToCompletion(); + + // Verify immediate GOAWAY was sent (no graceful GOAWAY) + EXPECT_EQ(0, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_sent").value()); +} + +TEST_P(Http2CodecImplTest, GracefulGoAwayAlreadyInProgress) { + // Configure graceful GOAWAY timeout of 100ms + server_http2_options_.mutable_graceful_goaway_timeout()->set_seconds(0); + server_http2_options_.mutable_graceful_goaway_timeout()->set_nanos(100 * 1000 * 1000); // 100ms + + auto graceful_goaway_timer = new NiceMock(&server_connection_.dispatcher_); + EXPECT_CALL(*graceful_goaway_timer, enableTimer(std::chrono::milliseconds(100), _)); + + initialize(); + + // First graceful GOAWAY call + EXPECT_CALL(client_callbacks_, onGoAway(_)).Times(AtLeast(1)); + server_->goAwayGraceful(); + driveToCompletion(); + + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + EXPECT_EQ(0, server_stats_store_.counter("http2.goaway_sent").value()); + + // Second graceful GOAWAY call should fallback to immediate GOAWAY + server_->goAwayGraceful(); + driveToCompletion(); + + // Should have sent immediate GOAWAY + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_sent").value()); +} + +TEST_P(Http2CodecImplTest, GracefulGoAwayCausesOutboundFlood) { + // Configure graceful GOAWAY timeout of 100ms + server_http2_options_.mutable_graceful_goaway_timeout()->set_seconds(0); + server_http2_options_.mutable_graceful_goaway_timeout()->set_nanos(100 * 1000 * 1000); // 100ms + + initialize(); + + TestRequestHeaderMapImpl request_headers; + HttpTestUtility::addDefaultHeaders(request_headers); + EXPECT_CALL(request_decoder_, decodeHeaders_(_, false)); + EXPECT_TRUE(request_encoder_->encodeHeaders(request_headers, false).ok()); + driveToCompletion(); + + int frame_count = 0; + Buffer::OwnedImpl buffer; + ON_CALL(server_connection_, write(_, _)) + .WillByDefault(Invoke([&buffer, &frame_count](Buffer::Instance& frame, bool) { + ++frame_count; + buffer.move(frame); + })); + + auto* violation_callback = + new NiceMock(&server_connection_.dispatcher_); + + TestResponseHeaderMapImpl response_headers{{":status", "200"}}; + response_encoder_->encodeHeaders(response_headers, false); + // Account for the single HEADERS frame above + for (uint32_t i = 0; i < CommonUtility::OptionsLimits::DEFAULT_MAX_OUTBOUND_FRAMES - 1; ++i) { + Buffer::OwnedImpl data("0"); + EXPECT_NO_THROW(response_encoder_->encodeData(data, false)); + } + EXPECT_NO_THROW(driveToCompletion()); + + EXPECT_FALSE(violation_callback->enabled_); + + // Test graceful goaway with outbound flood - should trigger error path + server_->goAwayGraceful(); + driveToCompletion(); + + EXPECT_TRUE(violation_callback->enabled_); + EXPECT_CALL(server_connection_, close(Envoy::Network::ConnectionCloseType::NoFlush, _)); + violation_callback->invokeCallback(); + + EXPECT_EQ(frame_count, CommonUtility::OptionsLimits::DEFAULT_MAX_OUTBOUND_FRAMES + 1); + EXPECT_EQ(1, server_stats_store_.counter("http2.outbound_flood").value()); + // Verify graceful GOAWAY stat was incremented before the error + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_graceful_sent").value()); +} + +TEST_P(Http2CodecImplTest, GracefulGoAwayRuntimeGuardDisabled) { + // Configure graceful GOAWAY timeout of 100ms + server_http2_options_.mutable_graceful_goaway_timeout()->set_seconds(0); + server_http2_options_.mutable_graceful_goaway_timeout()->set_nanos(100 * 1000 * 1000); // 100ms + + // Disable the runtime feature + TestScopedRuntime scoped_runtime; + scoped_runtime.mergeValues({{"envoy.reloadable_features.http2_graceful_goaway", "false"}}); + + initialize(); + + // Should fallback to immediate GOAWAY when runtime feature is disabled + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_sent").value()); + + EXPECT_CALL(client_callbacks_, onGoAway(_)).Times(AtLeast(1)); + server_->goAwayGraceful(); + driveToCompletion(); + + // Verify immediate GOAWAY was sent (no graceful GOAWAY due to runtime guard) + EXPECT_EQ(0, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_sent").value()); +} + +TEST_P(Http2CodecImplTest, ShutdownNoticeDoesNotIncrementGracefulCounter) { + initialize(); + + // Test that shutdownNotice does NOT increment the graceful GOAWAY counter + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_graceful_sent").value()); + ASSERT_EQ(0, server_stats_store_.counter("http2.goaway_sent").value()); + + // shutdownNotice should send initial graceful GOAWAY via adapter + EXPECT_CALL(client_callbacks_, onGoAway(_)); + server_->shutdownNotice(); + driveToCompletion(); + + // Verify: graceful GOAWAY counter should NOT increment (shutdownNotice uses adapter + // implementation) + EXPECT_EQ(0, server_stats_store_.counter("http2.goaway_graceful_sent").value()); +} + TEST_P(Http2CodecImplTest, ProtocolErrorForTest) { initialize(); EXPECT_EQ(absl::nullopt, request_encoder_->http1StreamEncoderOptions()); @@ -1472,7 +1641,7 @@ TEST_P(Http2CodecImplTest, DumpsStreamlessConnectionWithoutAllocatingMemory) { "max_headers_kb_: 60, max_headers_count_: 100, " "per_stream_buffer_limit_: 16777216, allow_metadata_: 0, " "stream_error_on_invalid_http_messaging_: 0, is_outbound_flood_monitored_control_frame_: " - "0, dispatching_: 0, raised_goaway_: 0, " + "0, dispatching_: 0, raised_goaway_: 0, graceful_goaway_in_progress_: 0, " "pending_deferred_reset_streams_.size(): 0\n" " &protocol_constraints_: \n" " ProtocolConstraints")); @@ -4362,7 +4531,7 @@ TEST_P(Http2CodecImplTest, ServerDispatchLoadShedPointCanCauseServerToSendGoAway } driveToCompletion(); - EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_sent").value()); + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_graceful_sent").value()); } TEST_P(Http2CodecImplTest, ServerDispatchLoadShedPointSendGoAwayAndClose) { @@ -4382,7 +4551,7 @@ TEST_P(Http2CodecImplTest, ServerDispatchLoadShedPointSendGoAwayAndClose) { driveToCompletion(); - EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_sent").value()); + EXPECT_EQ(1, server_stats_store_.counter("http2.goaway_graceful_sent").value()); } TEST_P(Http2CodecImplTest, ServerDispatchLoadShedPointsAreOnlyConsultedOncePerDispatch) { diff --git a/test/integration/multiplexed_integration_test.cc b/test/integration/multiplexed_integration_test.cc index e1161c9eae798..9d707f8989e65 100644 --- a/test/integration/multiplexed_integration_test.cc +++ b/test/integration/multiplexed_integration_test.cc @@ -1254,6 +1254,142 @@ TEST_P(MultiplexedIntegrationTest, SendGoAwayTriggerredInLocalReplyEncoderFilter ASSERT_TRUE(codec_client_->waitForDisconnect()); } +// Test graceful GOAWAY behavior when triggered by L7 filter for HTTP/2 only. +// This tests that the graceful_goaway_timeout configuration works for filter-triggered scenarios. +// Note: Drain scenarios use shutdownNotice() which has its own adapter specific implementation +// and do not use the gracefulGoAway method. +TEST_P(MultiplexedIntegrationTest, Http2GracefulGoAwayTriggerredByFilter) { + EXCLUDE_DOWNSTREAM_HTTP3 + if (downstreamProtocol() != Http::CodecType::HTTP2) { + return; + } + + // Configure graceful GOAWAY timeout - this should affect filter-triggered GOAWAY + config_helper_.addConfigModifier( + [&](envoy::extensions::filters::network::http_connection_manager::v3::HttpConnectionManager& + hcm) -> void { + auto* http2_options = hcm.mutable_http2_protocol_options(); + http2_options->mutable_graceful_goaway_timeout()->set_nanos(100000000); // 100ms + }); + + config_helper_.addFilter("name: send-goaway-during-decode-filter"); + initialize(); + + codec_client_ = makeHttpConnection(lookupPort("http")); + auto response = codec_client_->makeHeaderOnlyRequest( + Http::TestRequestHeaderMapImpl{{":method", "GET"}, + {":path", "/test/long/url"}, + {":scheme", "http"}, + {":authority", "host"}, + {"skip-goaway", "false"}}); + + // Should receive graceful GOAWAY sequence and then reset + ASSERT_TRUE(response->waitForReset()); + ASSERT_TRUE(codec_client_->waitForDisconnect()); + + // Verify graceful GOAWAY stats were incremented + test_server_->waitForCounterGe("http2.goaway_graceful_sent", 1); +} + +// Test interaction between graceful GOAWAY and connection manager drain. +// Verifies that drain-triggered GOAWAY uses shutdownNotice() and doesn't use +// graceful_goaway_timeout. +TEST_P(MultiplexedIntegrationTest, Http2GracefulGoAwayDrainInteraction) { + EXCLUDE_DOWNSTREAM_HTTP3 + if (downstreamProtocol() != Http::CodecType::HTTP2) { + return; + } + + // Configure graceful GOAWAY timeout - this should NOT affect drain-triggered GOAWAY + config_helper_.addConfigModifier( + [&](envoy::extensions::filters::network::http_connection_manager::v3::HttpConnectionManager& + hcm) -> void { + auto* http2_options = hcm.mutable_http2_protocol_options(); + http2_options->mutable_graceful_goaway_timeout()->set_nanos(100000000); // 100ms + }); + + autonomous_upstream_ = true; + initialize(); + + // Establish connection and start a request + codec_client_ = makeHttpConnection(lookupPort("http")); + auto response = codec_client_->makeHeaderOnlyRequest(default_request_headers_); + ASSERT_TRUE(response->waitForEndStream()); + EXPECT_TRUE(response->complete()); + EXPECT_EQ("200", response->headers().getStatusValue()); + + // Start drain sequence - this should trigger shutdownNotice() not graceful GOAWAY + absl::Notification drain_sequence_started; + test_server_->server().dispatcher().post([this, &drain_sequence_started]() { + test_server_->drainManager().startDrainSequence(Network::DrainDirection::All, [] {}); + drain_sequence_started.Notify(); + }); + drain_sequence_started.WaitForNotification(); + + // Make requests until drain close happens - this mimics the drain test pattern + IntegrationStreamDecoderPtr response2; + while (!test_server_->counter("http.config_test.downstream_cx_drain_close")->value()) { + response2 = codec_client_->makeHeaderOnlyRequest(default_request_headers_); + ASSERT_TRUE(response2->waitForEndStream()); + } + + // Connection should eventually close due to drain + ASSERT_TRUE(codec_client_->waitForDisconnect()); + + // Now check that GOAWAY was sent via shutdownNotice (not graceful GOAWAY) + EXPECT_TRUE(codec_client_->sawGoAway()); + + // Verify drain stats but graceful GOAWAY stats should NOT be incremented + // because drain uses shutdownNotice() not graceful GOAWAY mechanism + EXPECT_EQ(test_server_->counter("http2.goaway_graceful_sent")->value(), 0); +} + +// Test that filter-triggered graceful GOAWAY works during drain phase. +// This verifies the behavior when both mechanisms could be active. +TEST_P(MultiplexedIntegrationTest, Http2FilterGracefulGoAwayDuringDrain) { + EXCLUDE_DOWNSTREAM_HTTP3 + if (downstreamProtocol() != Http::CodecType::HTTP2) { + return; + } + + // Configure graceful GOAWAY timeout + config_helper_.addConfigModifier( + [&](envoy::extensions::filters::network::http_connection_manager::v3::HttpConnectionManager& + hcm) -> void { + auto* http2_options = hcm.mutable_http2_protocol_options(); + http2_options->mutable_graceful_goaway_timeout()->set_nanos(50000000); // 50ms + }); + + config_helper_.addFilter("name: send-goaway-during-decode-filter"); + autonomous_upstream_ = true; + initialize(); + + // Start drain sequence first + absl::Notification drain_sequence_started; + test_server_->server().dispatcher().post([this, &drain_sequence_started]() { + test_server_->drainManager().startDrainSequence(Network::DrainDirection::All, [] {}); + drain_sequence_started.Notify(); + }); + drain_sequence_started.WaitForNotification(); + + // Now establish connection and trigger filter-based graceful GOAWAY + codec_client_ = makeHttpConnection(lookupPort("http")); + auto response = codec_client_->makeHeaderOnlyRequest( + Http::TestRequestHeaderMapImpl{{":method", "GET"}, + {":path", "/test/filter/goaway"}, + {":scheme", "http"}, + {":authority", "host"}, + {"skip-goaway", "false"}}); + + // Should receive graceful GOAWAY sequence from filter, even during drain + ASSERT_TRUE(response->waitForReset()); + ASSERT_TRUE(codec_client_->waitForDisconnect()); + + // Verify filter-triggered graceful GOAWAY stats were incremented + // This shows filter-triggered graceful GOAWAY still works during drain + test_server_->waitForCounterGe("http2.goaway_graceful_sent", 1); +} + // Send client headers, a GoAway and then a body and ensure the full request and // response are received. TEST_P(MultiplexedIntegrationTest, GoAway) { diff --git a/test/integration/overload_integration_test.cc b/test/integration/overload_integration_test.cc index 8887f2155e0a2..02f1bc3447db0 100644 --- a/test/integration/overload_integration_test.cc +++ b/test/integration/overload_integration_test.cc @@ -1129,11 +1129,21 @@ TEST_P(LoadShedPointIntegrationTest, Http1ServerDispatchAbortClosesConnectionWhe } TEST_P(LoadShedPointIntegrationTest, Http2ServerDispatchSendsGoAwayCompletingPendingRequests) { - // Test only applies to HTTP2. + // Test that when HTTP2 server enters overload state during request dispatch, + // it sends GOAWAY frames while allowing pending requests to complete gracefully. if (downstreamProtocol() != Http::CodecClient::Type::HTTP2) { return; } autonomous_upstream_ = true; + + // Configure a very short graceful GOAWAY timeout so the final GOAWAY is sent quickly + config_helper_.addConfigModifier( + [=](envoy::extensions::filters::network::http_connection_manager::v3::HttpConnectionManager& + hcm) -> void { + auto* http2_options = hcm.mutable_http2_protocol_options(); + http2_options->mutable_graceful_goaway_timeout()->set_nanos(100000000); // 100ms + }); + initializeOverloadManager( TestUtility::parseYaml(R"EOF( name: "envoy.load_shed_points.http2_server_go_away_on_dispatch" @@ -1161,13 +1171,27 @@ TEST_P(LoadShedPointIntegrationTest, Http2ServerDispatchSendsGoAwayCompletingPen first_request_encoder.encodeData(first_request_body, true); ASSERT_TRUE(first_request_decoder->waitForEndStream()); - EXPECT_TRUE(codec_client_->sawGoAway()); + // Wait for the server to send the graceful GOAWAY + test_server_->waitForCounterEq("http2.goaway_graceful_sent", 1); + + // NOTE: After sending GOAWAY, the server should ignore frames on streams with identifiers + // higher than the last stream ID in the GOAWAY frame. However, Envoy's current + // implementation in ServerConnectionImpl::onBeginHeaders() may still create new + // streams without checking GOAWAY state. oghttp2 may continue the request and ignore + // goaway while nghttp2 is stricter and does a reset leading to different results. + // Thus we check if the stream has ended first. + bool second_request_completed = second_request_decoder->waitForEndStream(); + if (second_request_completed) { + EXPECT_TRUE(second_request_decoder->complete()); + } else { + EXPECT_TRUE(second_request_decoder->reset()); + } + + // Wait for the graceful GOAWAY timeout to expire and final GOAWAY to be sent test_server_->waitForCounterEq("http2.goaway_sent", 1); - // The GOAWAY gets submitted with the first created stream as the last stream - // that will be processed on this connection, so the second stream's frames - // are ignored. - EXPECT_FALSE(second_request_decoder->complete()); + // Verify the client eventually receives the GOAWAY + EXPECT_TRUE(codec_client_->sawGoAway()); updateResource(0.80); test_server_->waitForGaugeEq( @@ -1206,7 +1230,7 @@ TEST_P(LoadShedPointIntegrationTest, Http2ServerDispatchSendsGoAwayAndClosesConn // The downstream should receive the GOAWAY and the connection should be closed. ASSERT_TRUE(codec_client_->waitForDisconnect()); EXPECT_TRUE(codec_client_->sawGoAway()); - test_server_->waitForCounterEq("http2.goaway_sent", 1); + test_server_->waitForCounterEq("http2.goaway_graceful_sent", 1); test_server_->waitForCounterEq("http.config_test.downstream_rq_overload_close", 1); // The second request will not complete.