Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -344,7 +344,10 @@ def _start_artifact_service(self, jar, requested_port):
for (env_id,
env) in self._pipeline_proto.components.environments.items()
})
self._artifact_staging_server = grpc.server(futures.ThreadPoolExecutor())
options = [("grpc.http2.max_pings_without_data", 0),
("grpc.http2.max_ping_strikes", 0)]
self._artifact_staging_server = grpc.server(
futures.ThreadPoolExecutor(), options=options)
port = self._artifact_staging_server.add_insecure_port(
'[::]:%s' % requested_port)
beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -457,26 +457,29 @@ def __init__(self,
worker_manager, # type: WorkerHandlerManager
):
# type: (...) -> None

# Options to have no limits (-1) on the size of the messages
# 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)]

self.state = state
self.provision_info = provision_info
self.control_server = grpc.server(
thread_pool_executor.shared_unbounded_instance())
thread_pool_executor.shared_unbounded_instance(), options=options)
self.control_port = self.control_server.add_insecure_port('[::]:0')
self.control_address = 'localhost:%s' % self.control_port

# Options to have no limits (-1) on the size of the messages
# received or sent over the data plane. The actual buffer size
# is controlled in a layer above.
no_max_message_sizes = [("grpc.max_receive_message_length", -1),
("grpc.max_send_message_length", -1)]
self.data_server = grpc.server(
thread_pool_executor.shared_unbounded_instance(),
options=no_max_message_sizes)
thread_pool_executor.shared_unbounded_instance(), options=options)
self.data_port = self.data_server.add_insecure_port('[::]:0')

self.state_server = grpc.server(
thread_pool_executor.shared_unbounded_instance(),
options=no_max_message_sizes)
thread_pool_executor.shared_unbounded_instance(), options=options)
self.state_port = self.state_server.add_insecure_port('[::]:0')

self.control_handler = BeamFnControlServicer(worker_manager)
Expand Down Expand Up @@ -510,8 +513,7 @@ def open_uncompressed(f):
GrpcStateServicer(state), self.state_server)

self.logging_server = grpc.server(
thread_pool_executor.shared_unbounded_instance(),
options=no_max_message_sizes)
thread_pool_executor.shared_unbounded_instance(), options=options)
self.logging_port = self.logging_server.add_insecure_port('[::]:0')
beam_fn_api_pb2_grpc.add_BeamFnLoggingServicer_to_server(
BasicLoggingService(), self.logging_server)
Expand Down
13 changes: 8 additions & 5 deletions sdks/python/apache_beam/runners/portability/local_job_service.py
Original file line number Diff line number Diff line change
Expand Up @@ -125,11 +125,12 @@ def get_service_address(self):
return 'localhost'

def start_grpc_server(self, port=0):
no_max_message_sizes = [("grpc.max_receive_message_length", -1),
("grpc.max_send_message_length", -1)]
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)]
self._server = grpc.server(
thread_pool_executor.shared_unbounded_instance(),
options=no_max_message_sizes)
thread_pool_executor.shared_unbounded_instance(), options=options)
port = self._server.add_insecure_port(
'%s:%d' % (self.get_bind_address(), port))
beam_job_api_pb2_grpc.add_JobServiceServicer_to_server(self, self._server)
Expand Down Expand Up @@ -183,8 +184,10 @@ def __init__(
self._worker_id = worker_id

def run(self):
options = [("grpc.http2.max_pings_without_data", 0),
("grpc.http2.max_ping_strikes", 0)]
logging_server = grpc.server(
thread_pool_executor.shared_unbounded_instance())
thread_pool_executor.shared_unbounded_instance(), options=options)
logging_port = logging_server.add_insecure_port('[::]:0')
logging_server.start()
logging_servicer = BeamFnLoggingServicer()
Expand Down
5 changes: 4 additions & 1 deletion sdks/python/apache_beam/runners/worker/channel_factory.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,10 @@


class GRPCChannelFactory(grpc.StreamStreamClientInterceptor):
DEFAULT_OPTIONS = [("grpc.keepalive_time_ms", 20000)]
DEFAULT_OPTIONS = [
("grpc.keepalive_time_ms", 20000),
("grpc.keepalive_timeout_ms", 300000),
]

def __init__(self):
pass
Expand Down
4 changes: 3 additions & 1 deletion sdks/python/apache_beam/runners/worker/worker_pool_main.py
Original file line number Diff line number Diff line change
Expand Up @@ -76,8 +76,10 @@ def start(
container_executable=None # type: Optional[str]
):
# type: (...) -> Tuple[str, grpc.Server]
options = [("grpc.http2.max_pings_without_data", 0),
("grpc.http2.max_ping_strikes", 0)]
worker_server = grpc.server(
thread_pool_executor.shared_unbounded_instance())
thread_pool_executor.shared_unbounded_instance(), options=options)
worker_address = 'localhost:%s' % worker_server.add_insecure_port(
'[::]:%s' % port)
worker_pool = cls(
Expand Down