[
https://issues.apache.org/jira/browse/BEAM-7980?focusedWorklogId=298861&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-298861
]
ASF GitHub Bot logged work on BEAM-7980:
----------------------------------------
Author: ASF GitHub Bot
Created on: 21/Aug/19 17:03
Start Date: 21/Aug/19 17:03
Worklog Time Spent: 10m
Work Description: tweise commented on pull request #9371: [BEAM-7980]
External environment with containerized worker pool
URL: https://github.com/apache/beam/pull/9371#discussion_r316296507
##########
File path: sdks/python/apache_beam/runners/worker/worker_pool_main.py
##########
@@ -0,0 +1,170 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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.
+#
+"""
+Worker pool entry point.
+
+The worker pool exposes an RPC service that is used with EXTERNAL
+environment to start and stop the SDK workers.
+
+This entry point is used by the Python SDK container in worker pool mode.
+"""
+
+from __future__ import absolute_import
+
+import argparse
+import atexit
+import logging
+import subprocess
+import sys
+import threading
+import time
+from concurrent import futures
+
+import grpc
+
+from apache_beam.portability.api import beam_fn_api_pb2
+from apache_beam.portability.api import beam_fn_api_pb2_grpc
+from apache_beam.runners.worker import sdk_worker
+
+
+class BeamFnExternalWorkerPoolServicer(
+ beam_fn_api_pb2_grpc.BeamFnExternalWorkerPoolServicer):
+
+ def __init__(self, worker_threads, use_process=False,
+ container_executable=None):
+ self._worker_threads = worker_threads
+ self._use_process = use_process
+ self._container_executable = container_executable
+ self._worker_processes = {}
+
+ @classmethod
+ def start(cls, worker_threads=1, use_process=False, port=0,
+ container_executable=None):
+ worker_server = grpc.server(futures.ThreadPoolExecutor(max_workers=10))
+ worker_address = 'localhost:%s' % worker_server.add_insecure_port(
+ '[::]:%s' % port)
+ beam_fn_api_pb2_grpc.add_BeamFnExternalWorkerPoolServicer_to_server(
+ cls(worker_threads,
+ use_process=use_process,
+ container_executable=container_executable),
+ worker_server)
+ worker_server.start()
+
+ # Register to kill the subprocesses on exit.
+ def kill_worker_processes():
+ for worker_process in cls._worker_processes.values():
+ worker_process.kill()
+ atexit.register(kill_worker_processes)
+
+ return worker_address, worker_server
+
+ def StartWorker(self, start_worker_request, unused_context):
+ try:
+ if self._use_process:
+ command = ['python', '-c',
+ 'from apache_beam.runners.worker.sdk_worker '
+ 'import SdkHarness; '
+ 'SdkHarness("%s",worker_count=%d,worker_id="%s").run()' % (
+ start_worker_request.control_endpoint.url,
+ self._worker_threads,
+ start_worker_request.worker_id)]
+ if self._container_executable:
+ # command as per container spec
+ command = [self._container_executable,
+ '--id=%s' % start_worker_request.worker_id,
+ '--logging_endpoint=%s'
+ % start_worker_request.logging_endpoint.url,
Review comment:
Example how the worker pool is started: `docker run --rm -p=50000:50000
tweise-docker-apache.bintray.io/beam/python:latest --worker_pool=true`
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 298861)
Time Spent: 3h 20m (was: 3h 10m)
> External environment with containerized worker pool
> ---------------------------------------------------
>
> Key: BEAM-7980
> URL: https://issues.apache.org/jira/browse/BEAM-7980
> Project: Beam
> Issue Type: Improvement
> Components: sdk-py-harness
> Reporter: Thomas Weise
> Assignee: Thomas Weise
> Priority: Major
> Time Spent: 3h 20m
> Remaining Estimate: 0h
>
> Augment Beam Python docker image and boot.go so that it can be used to launch
> BeamFnExternalWorkerPoolServicer.
> [https://docs.google.com/document/d/1z3LNrRtr8kkiFHonZ5JJM_L4NWNBBNcqRc_yAf6G0VI/edit#heading=h.lnhm75dhvhi0]
>
--
This message was sent by Atlassian Jira
(v8.3.2#803003)