github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/portability/job_server.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 # pytype: skip-file 19 20 import atexit 21 import shutil 22 import signal 23 import tempfile 24 import threading 25 26 import grpc 27 28 from apache_beam.options import pipeline_options 29 from apache_beam.portability.api import beam_job_api_pb2_grpc 30 from apache_beam.runners.portability import local_job_service 31 from apache_beam.utils import subprocess_server 32 from apache_beam.version import __version__ as beam_version 33 34 35 class JobServer(object): 36 def start(self): 37 """Starts this JobServer, returning a grpc service to which to submit jobs. 38 """ 39 raise NotImplementedError(type(self)) 40 41 def stop(self): 42 """Stops this job server.""" 43 raise NotImplementedError(type(self)) 44 45 46 class ExternalJobServer(JobServer): 47 def __init__(self, endpoint, timeout=None): 48 self._endpoint = endpoint 49 self._timeout = timeout 50 51 def start(self): 52 # type: () -> beam_job_api_pb2_grpc.JobServiceStub 53 channel = grpc.insecure_channel(self._endpoint) 54 grpc.channel_ready_future(channel).result(timeout=self._timeout) 55 return beam_job_api_pb2_grpc.JobServiceStub(channel) 56 57 def stop(self): 58 pass 59 60 61 class EmbeddedJobServer(JobServer): 62 def start(self): 63 # type: () -> local_job_service.LocalJobServicer 64 return local_job_service.LocalJobServicer() 65 66 def stop(self): 67 pass 68 69 70 class StopOnExitJobServer(JobServer): 71 """Wraps a JobServer such that its stop will automatically be called on exit. 72 """ 73 def __init__(self, job_server): 74 self._lock = threading.Lock() 75 self._job_server = job_server 76 self._started = False 77 78 def start(self): 79 with self._lock: 80 if not self._started: 81 self._endpoint = self._job_server.start() 82 self._started = True 83 atexit.register(self.stop) 84 signal.signal(signal.SIGINT, self.stop) 85 return self._endpoint 86 87 def stop(self): 88 with self._lock: 89 if self._started: 90 self._job_server.stop() 91 self._started = False 92 93 94 class SubprocessJobServer(JobServer): 95 """An abstract base class for JobServers run as an external process.""" 96 def __init__(self): 97 self._local_temp_root = None 98 self._server = None 99 100 def subprocess_cmd_and_endpoint(self): 101 raise NotImplementedError(type(self)) 102 103 def start(self): 104 if self._server is None: 105 self._local_temp_root = tempfile.mkdtemp(prefix='beam-temp') 106 cmd, endpoint = self.subprocess_cmd_and_endpoint() 107 port = int(endpoint.split(':')[-1]) 108 self._server = subprocess_server.SubprocessServer( 109 beam_job_api_pb2_grpc.JobServiceStub, cmd, port=port) 110 return self._server.start() 111 112 def stop(self): 113 if self._local_temp_root: 114 shutil.rmtree(self._local_temp_root) 115 self._local_temp_root = None 116 return self._server.stop() 117 118 def local_temp_dir(self, **kwargs): 119 return tempfile.mkdtemp(dir=self._local_temp_root, **kwargs) 120 121 122 class JavaJarJobServer(SubprocessJobServer): 123 def __init__(self, options): 124 super().__init__() 125 options = options.view_as(pipeline_options.JobServerOptions) 126 self._job_port = options.job_port 127 self._artifact_port = options.artifact_port 128 self._expansion_port = options.expansion_port 129 self._artifacts_dir = options.artifacts_dir 130 self._java_launcher = options.job_server_java_launcher 131 self._jvm_properties = options.job_server_jvm_properties 132 133 def java_arguments( 134 self, job_port, artifact_port, expansion_port, artifacts_dir): 135 raise NotImplementedError(type(self)) 136 137 def path_to_jar(self): 138 raise NotImplementedError(type(self)) 139 140 @staticmethod 141 def path_to_beam_jar(gradle_target, artifact_id=None): 142 return subprocess_server.JavaJarServer.path_to_beam_jar( 143 gradle_target, artifact_id=artifact_id) 144 145 @staticmethod 146 def local_jar(url): 147 return subprocess_server.JavaJarServer.local_jar(url) 148 149 def subprocess_cmd_and_endpoint(self): 150 jar_path = self.local_jar(self.path_to_jar()) 151 artifacts_dir = ( 152 self._artifacts_dir if self._artifacts_dir else self.local_temp_dir( 153 prefix='artifacts')) 154 job_port, = subprocess_server.pick_port(self._job_port) 155 subprocess_cmd = [self._java_launcher, '-jar'] + self._jvm_properties + [ 156 jar_path 157 ] + list( 158 self.java_arguments( 159 job_port, self._artifact_port, self._expansion_port, artifacts_dir)) 160 return (subprocess_cmd, 'localhost:%s' % job_port)