This is an automated email from the ASF dual-hosted git repository.
shunping pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new e5a93388d54 Fix too-many-pings on FnAPI runner under grpc mode (#37013)
e5a93388d54 is described below
commit e5a93388d541a719d249bce78c5859e9fd8ce7f1
Author: Shunping Huang <[email protected]>
AuthorDate: Fri Dec 5 14:35:41 2025 -0500
Fix too-many-pings on FnAPI runner under grpc mode (#37013)
* Fix too-many-pings on FnAPI runner under grpc mode
* Fix lints
---
.../runners/portability/fn_api_runner/worker_handlers.py | 13 +++++++++----
1 file changed, 9 insertions(+), 4 deletions(-)
diff --git
a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py
b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py
index 338f6ece57c..d79b381f2d7 100644
---
a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py
+++
b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py
@@ -465,10 +465,15 @@ class GrpcServer(object):
# received or sent over the data plane. The actual buffer size
# is controlled in a layer above. Also, options to keep the server alive
# when too many pings are received.
- options = [("grpc.max_receive_message_length", -1),
- ("grpc.max_send_message_length", -1),
- ("grpc.http2.max_pings_without_data", 0),
- ("grpc.http2.max_ping_strikes", 0)]
+ options = [
+ ("grpc.max_receive_message_length", -1),
+ ("grpc.max_send_message_length", -1),
+ ("grpc.http2.max_pings_without_data", 0),
+ ("grpc.http2.max_ping_strikes", 0),
+ # match `grpc.keepalive_time_ms` defined in the client
+ # (channel_factory.py)
+ ("grpc.http2.min_ping_interval_without_data_ms", 20_000),
+ ]
self.state = state
self.provision_info = provision_info