-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathsubprocess_server.py
More file actions
625 lines (559 loc) · 21.2 KB
/
subprocess_server.py
File metadata and controls
625 lines (559 loc) · 21.2 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
#
# 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.
#
# pytype: skip-file
import contextlib
import dataclasses
import glob
import hashlib
import logging
import os
import re
import shutil
import signal
import socket
import subprocess
import threading
import time
import zipfile
from typing import Any
from typing import Set
from urllib.error import URLError
from urllib.request import Request
from urllib.request import urlopen
import grpc
from apache_beam.io.filesystems import FileSystems
from apache_beam.runners.internal.names import BEAM_SDK_NAME
from apache_beam.version import __version__ as beam_version
_LOGGER = logging.getLogger(__name__)
@dataclasses.dataclass
class _SharedCacheEntry:
obj: Any
owners: Set[str]
class _SharedCache:
"""A cache that keeps objects alive (and repeatedly returns the same instance)
until the last user indicates that they're done.
The typical usage is as follows::
try:
token = cache.register()
# All objects retrieved from the cache from this point on will be memoized
# and kept alive (including across other threads and callers) at least
# until the purge is called below (and possibly longer, if other calls
# to register were made).
obj = cache.get(...)
another_obj = cache.get(...)
...
finally:
cache.purge(token)
"""
def __init__(self, constructor, destructor):
self._constructor = constructor
self._destructor = destructor
self._live_owners = set()
self._cache = {}
self._lock = threading.Lock()
self._counter = 0
def _next_id(self):
with self._lock:
self._counter += 1
return self._counter
def register(self):
owner = self._next_id()
self._live_owners.add(owner)
return owner
def purge(self, owner):
if owner not in self._live_owners:
raise ValueError(f"{owner} not in {self._live_owners}")
self._live_owners.remove(owner)
to_delete = []
with self._lock:
for key, entry in list(self._cache.items()):
if owner in entry.owners:
entry.owners.remove(owner)
if not entry.owners:
to_delete.append(entry.obj)
del self._cache[key]
# Actually call the destructors outside of the lock.
for value in to_delete:
self._destructor(value)
def get(self, *key):
if not self._live_owners:
raise RuntimeError("At least one owner must be registered.")
with self._lock:
if key not in self._cache:
self._cache[key] = _SharedCacheEntry(self._constructor(*key), set())
for owner in self._live_owners:
self._cache[key].owners.add(owner)
return self._cache[key].obj
class JavaHelper:
@classmethod
def get_java(cls):
java_path = 'java'
java_home = os.environ.get('JAVA_HOME')
if java_home:
java_path = os.path.join(java_home, 'bin', 'java')
return java_path
class SubprocessServer(object):
"""An abstract base class for running GRPC Servers as an external process.
This class acts as a context which will start up a server, provides a stub
to connect to it, and then shuts the server down. For example::
with SubprocessServer(GrpcStubClass, [executable, arg, ...]) as stub:
stub.CallService(...)
"""
def __init__(self, stub_class, cmd, port=None, logger=None):
"""Creates the server object.
:param stub_class: the auto-generated GRPC client stub class used for
connecting to the GRPC service
:param cmd: command (including arguments) for starting up the server,
suitable for passing to `subprocess.POpen`.
:param port: (optional) the port at which the subprocess will serve its
service. If not given, one will be randomly chosen and the special
string "{{PORT}}" will be substituted in the command line arguments
with the chosen port.
:param logger: (optional) The logger or logger name to use for the
subprocess's stderr and stdout. If not given, the current module logger
would be used.
"""
self._owner_id = None
self._stub_class = stub_class
self._cmd = [str(arg) for arg in cmd]
self._port = port
self._grpc_channel = None
if isinstance(logger, str):
self._logger = logging.getLogger(logger)
elif isinstance(logger, logging.Logger):
self._logger = logger
else:
self._logger = _LOGGER
@classmethod
@contextlib.contextmanager
def cache_subprocesses(cls):
"""A context that ensures any subprocess created or used in its duration
stay alive for at least the duration of this context.
These subprocesses may be shared with other contexts as well.
"""
try:
unique_id = cls._cache.register()
yield
finally:
cls._cache.purge(unique_id)
def __enter__(self):
return self.start()
def __exit__(self, *unused_args):
self.stop()
def start(self):
try:
process, endpoint = self.start_process()
wait_secs = .1
channel_options = [
("grpc.max_receive_message_length", -1),
("grpc.max_send_message_length", -1),
# Default: 20000ms (20s), increased to 10 minutes for stability
("grpc.keepalive_timeout_ms", 600_000),
# Default: 2, set to 0 to allow unlimited pings without data
("grpc.http2.max_pings_without_data", 0),
# Default: False, set to True to allow keepalive pings when no calls
("grpc.keepalive_permit_without_calls", True),
# Default: 2, set to 0 to allow unlimited ping strikes
("grpc.http2.max_ping_strikes", 0),
# Default: 0 (disabled), enable socket reuse for better handling
("grpc.so_reuseport", 1),
]
self._grpc_channel = grpc.insecure_channel(
endpoint, options=channel_options)
channel_ready = grpc.channel_ready_future(self._grpc_channel)
while True:
if process is not None and process.poll() is not None:
_LOGGER.error("Started job service with %s", process.args)
raise RuntimeError(
'Service failed to start up with error %s' % process.poll())
try:
channel_ready.result(timeout=wait_secs)
break
except (grpc.FutureTimeoutError, grpc.RpcError):
wait_secs *= 1.2
logging.log(
logging.WARNING if wait_secs > 1 else logging.DEBUG,
'Waiting for grpc channel to be ready at %s.',
endpoint)
return self._stub_class(self._grpc_channel)
except: # pylint: disable=bare-except
_LOGGER.exception("Error bringing up service")
self.stop()
raise
def start_process(self):
if self._owner_id is not None:
self._cache.purge(self._owner_id)
self._owner_id = self._cache.register()
return self._cache.get(tuple(self._cmd), self._port, self._logger)
def _really_start_process(cmd, port, logger):
if not port:
port, = pick_port(None)
cmd = [arg.replace('{{PORT}}', str(port)) for arg in cmd] # pylint: disable=not-an-iterable
endpoint = 'localhost:%s' % port
_LOGGER.info("Starting service with %s", str(cmd).replace("',", "'"))
process = subprocess.Popen(
cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
# Emit the output of this command as info level logging.
def log_stdout():
line = process.stdout.readline()
while line:
# The log obtained from stdout is bytes, decode it into string.
# Remove newline via rstrip() to not print an empty line.
logger.info(line.decode(errors='backslashreplace').rstrip())
line = process.stdout.readline()
t = threading.Thread(target=log_stdout)
t.daemon = True
t.start()
return process, endpoint
def stop(self):
self.stop_process()
def stop_process(self):
if self._owner_id is not None:
self._cache.purge(self._owner_id)
self._owner_id = None
if self._grpc_channel:
try:
self._grpc_channel.close()
except: # pylint: disable=bare-except
_LOGGER.error(
"Could not close the gRPC channel started for the "
"expansion service")
finally:
self._grpc_channel = None
def _really_stop_process(process_and_endpoint):
process, _ = process_and_endpoint # pylint: disable=unpacking-non-sequence
if not process:
return
for _ in range(5):
if process.poll() is not None:
break
logging.debug("Sending SIGINT to process")
try:
process.send_signal(signal.SIGINT)
except ValueError:
# process.send_signal raises a ValueError on Windows.
process.terminate()
time.sleep(1)
if process.poll() is None:
process.kill()
_cache = _SharedCache(
constructor=_really_start_process, destructor=_really_stop_process)
class JavaJarServer(SubprocessServer):
MAVEN_CENTRAL_REPOSITORY = 'https://repo.maven.apache.org/maven2'
MAVEN_STAGING_REPOSITORY = (
'https://repository.apache.org/content/groups/staging')
GOOGLE_MAVEN_MIRROR = (
'https://maven-central.storage-download.googleapis.com/maven2')
BEAM_GROUP_ID = 'org.apache.beam'
JAR_CACHE = os.path.expanduser("~/.apache_beam/cache/jars")
_BEAM_SERVICES = type(
'local', (threading.local, ),
dict(__init__=lambda self: setattr(self, 'replacements', {})))()
_DEFAULT_USER_AGENT = f'{BEAM_SDK_NAME}/{beam_version}'
def __init__(
self,
stub_class,
path_to_jar,
java_arguments,
classpath=None,
cache_dir=None,
logger=None):
self._java_path = JavaHelper.get_java()
if classpath:
# java -jar ignores the classpath, so we make a new jar that embeds
# the requested classpath.
path_to_jar = self.make_classpath_jar(path_to_jar, classpath, cache_dir)
super().__init__(
stub_class,
[self._java_path, '-jar', path_to_jar] + list(java_arguments),
logger=logger)
self._existing_service = path_to_jar if is_service_endpoint(
path_to_jar) else None
def start_process(self):
if self._existing_service:
return None, self._existing_service
else:
if not shutil.which(self._java_path):
java_home = os.environ.get('JAVA_HOME')
if java_home:
raise RuntimeError(
'Java is not correctly installed in JAVA_HOME=%s to use this '
'transform/runner. Please check if JAVA_HOME is correctly set and'
' points to your Java installation directory.' % java_home)
else:
raise RuntimeError(
'Java must be installed on this system to use this '
'transform/runner.')
return super().start_process()
def stop_process(self):
if self._existing_service:
pass
else:
return super().stop_process()
@classmethod
def jar_name(cls, artifact_id, version, classifier=None, appendix=None):
return '-'.join(
filter(None, [artifact_id, appendix, version, classifier])) + '.jar'
@classmethod
def path_to_maven_jar(
cls,
artifact_id,
group_id,
version,
repository=MAVEN_CENTRAL_REPOSITORY,
classifier=None,
appendix=None):
return '/'.join([
repository,
group_id.replace('.', '/'),
artifact_id,
version,
cls.jar_name(artifact_id, version, classifier, appendix)
])
@classmethod
def parse_gradle_target(cls, gradle_target, artifact_id=None):
gradle_package = gradle_target.strip(':').rsplit(':', 1)[0]
if not artifact_id:
artifact_id = 'beam-' + gradle_package.replace(':', '-')
return gradle_package, artifact_id
@classmethod
def path_to_dev_beam_jar(
cls,
gradle_target,
appendix=None,
version=beam_version,
artifact_id=None):
gradle_package, artifact_id = cls.parse_gradle_target(
gradle_target, artifact_id)
project_root = os.path.sep.join(
os.path.abspath(__file__).split(os.path.sep)[:-5])
return os.path.join(
project_root,
gradle_package.replace(':', os.path.sep),
'build',
'libs',
cls.jar_name(
artifact_id,
version.replace('.dev', ''),
classifier='SNAPSHOT',
appendix=appendix))
@classmethod
def path_to_beam_jar(
cls,
gradle_target,
appendix=None,
version=beam_version,
artifact_id=None,
maven_repository_url=None):
if gradle_target in cls._BEAM_SERVICES.replacements:
return cls._BEAM_SERVICES.replacements[gradle_target]
_, artifact_id = cls.parse_gradle_target(gradle_target, artifact_id)
project_root = os.path.sep.join(
os.path.abspath(__file__).split(os.path.sep)[:-5])
local_path = cls.path_to_dev_beam_jar(
gradle_target, appendix, version, artifact_id)
if os.path.exists(local_path):
_LOGGER.info('Using pre-built snapshot at %s', local_path)
return local_path
maven_repo = maven_repository_url or cls.MAVEN_CENTRAL_REPOSITORY
if 'rc' in version:
# Release candidate
version = version.split('rc')[0]
maven_repo = cls.MAVEN_STAGING_REPOSITORY
elif '.dev' in version:
# TODO: Attempt to use nightly snapshots?
raise RuntimeError(
(
'%s not found. '
'Please build the server with \n cd %s; ./gradlew %s') %
(local_path, os.path.abspath(project_root), gradle_target))
return cls.path_to_maven_jar(
artifact_id, cls.BEAM_GROUP_ID, version, maven_repo, appendix=appendix)
@classmethod
def _download_jar_to_cache(
cls, download_url, cached_jar_path, user_agent=None):
"""Downloads a jar from the given URL to the specified cache path.
Args:
download_url (str): The URL to download from.
cached_jar_path (str): The local path where the jar should be cached.
user_agent (str): The user agent to use when downloading.
"""
# Issue warning when downloading from public repositories
public_repos = [
cls.MAVEN_CENTRAL_REPOSITORY,
cls.GOOGLE_MAVEN_MIRROR,
]
if any(download_url.startswith(repo) for repo in public_repos):
_LOGGER.warning(
" WARNING: Apache Beam is downloading dependencies from a "
"public repository at runtime.\n"
" This may pose security risks or cause instability due to "
"repository availability.\n"
" URL: %s\n"
" Destination: %s\n"
" Consider pre-staging dependencies or using a private repository "
"mirror.\n"
" For more information, see: "
"https://beam.apache.org/documentation/sdks/python-dependencies/",
download_url,
cached_jar_path)
try:
url_read = FileSystems.open(download_url)
except ValueError:
if user_agent is None:
user_agent = cls._DEFAULT_USER_AGENT
url_request = Request(download_url, headers={'User-Agent': user_agent})
url_read = urlopen(url_request)
with open(cached_jar_path + '.tmp', 'wb') as jar_write:
shutil.copyfileobj(url_read, jar_write, length=1 << 20)
try:
os.rename(cached_jar_path + '.tmp', cached_jar_path)
except FileNotFoundError:
# A race when multiple programs run in parallel and the cached_jar
# is already moved. Safe to ignore.
pass
@classmethod
def local_jar(cls, url, cache_dir=None, user_agent=None):
"""Returns a local path to the given jar, downloading it if necessary.
Args:
url (str): A URL or local path to a jar file.
cache_dir (str): The directory to use for caching downloaded jars. If not
specified, a default temporary directory will be used.
user_agent (str): The user agent to use when downloading the jar.
Returns:
str: The local path to the jar file.
"""
if cache_dir is None:
cache_dir = cls.JAR_CACHE
# TODO: Verify checksum?
if is_service_endpoint(url):
return url
elif os.path.exists(url):
return url
else:
cached_jar = os.path.join(cache_dir, os.path.basename(url))
if os.path.exists(cached_jar):
_LOGGER.info('Using cached job server jar from %s' % url)
else:
_LOGGER.info('Downloading job server jar from %s' % url)
if not os.path.exists(cache_dir):
os.makedirs(cache_dir)
# TODO: Clean up this cache according to some policy.
try:
cls._download_jar_to_cache(url, cached_jar, user_agent)
except URLError as e:
# Try Google Maven mirror as fallback if the original URL is from
# Maven Central
if url.startswith(cls.MAVEN_CENTRAL_REPOSITORY):
fallback_url = url.replace(
cls.MAVEN_CENTRAL_REPOSITORY, cls.GOOGLE_MAVEN_MIRROR)
_LOGGER.info(
'Trying Google Maven mirror fallback: %s' % fallback_url)
try:
cls._download_jar_to_cache(fallback_url, cached_jar, user_agent)
_LOGGER.info(
'Successfully downloaded from Google Maven mirror: %s' %
fallback_url)
except URLError as fallback_e:
raise RuntimeError(
f'Unable to fetch remote job server jar at {url}: {e}. '
f'Also failed to fetch from Google Maven mirror at '
f'{fallback_url}: {fallback_e}. '
f'If no Internet access at runtime, stage the jar at '
f'{cached_jar}')
else:
raise RuntimeError(
f'Unable to fetch remote job server jar at {url}: {e}. If no '
f'Internet access at runtime, stage the jar at {cached_jar}')
return cached_jar
@classmethod
@contextlib.contextmanager
def beam_services(cls, replacements):
try:
old = cls._BEAM_SERVICES.replacements
cls._BEAM_SERVICES.replacements = dict(old, **replacements)
yield
finally:
cls._BEAM_SERVICES.replacements = old
@classmethod
def make_classpath_jar(cls, main_jar, extra_jars, cache_dir=None):
if cache_dir is None:
cache_dir = cls.JAR_CACHE
composite_jar_dir = os.path.join(cache_dir, 'composite-jars')
os.makedirs(composite_jar_dir, exist_ok=True)
classpath = []
# Class-Path references from a jar must be relative, so we create
# a relatively-addressable subdirectory with symlinks to all the
# required jars.
for pattern in [main_jar] + list(extra_jars):
for path in glob.glob(pattern) or [pattern]:
path = os.path.abspath(path)
rel_path = hashlib.sha256(
path.encode('utf-8')).hexdigest() + os.path.splitext(path)[1]
classpath.append(rel_path)
if not os.path.lexists(os.path.join(composite_jar_dir, rel_path)):
os.symlink(path, os.path.join(composite_jar_dir, rel_path))
# Now create a single jar that simply references the rest and has the same
# main class as main_jar.
composite_jar = os.path.join(
composite_jar_dir,
hashlib.sha256(' '.join(sorted(classpath)).encode('ascii')).hexdigest()
+ '.jar')
if not os.path.exists(composite_jar):
with zipfile.ZipFile(main_jar) as main:
with main.open('META-INF/MANIFEST.MF') as manifest:
main_class = next(
filter(lambda line: line.startswith(b'Main-Class: '), manifest))
with zipfile.ZipFile(composite_jar + '.tmp', 'w') as composite:
with composite.open('META-INF/MANIFEST.MF', 'w') as manifest:
manifest.write(b'Manifest-Version: 1.0\n')
manifest.write(main_class)
manifest.write(
b'Class-Path: ' + '\n '.join(classpath).encode('ascii') + b'\n')
os.rename(composite_jar + '.tmp', composite_jar)
return composite_jar
def is_service_endpoint(path):
"""Checks whether the path conforms to the 'beam_services' PipelineOption."""
return re.match(r'^[a-zA-Z0-9.-]+:\d+$', path)
def pick_port(*ports):
"""
Returns a list of ports, same length as input ports list, but replaces
all None or 0 ports with a random free port.
"""
sockets = []
def find_free_port(port):
if port:
return port
else:
try:
s = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
except OSError as e:
# [Errno 97] Address family not supported by protocol
# Likely indicates we are in an IPv6-only environment (BEAM-10618). Try
# again with AF_INET6.
if e.errno == 97:
s = socket.socket(socket.AF_INET6, socket.SOCK_STREAM)
else:
raise e
sockets.append(s)
s.bind(('localhost', 0))
return s.getsockname()[1]
ports = list(map(find_free_port, ports))
# Close sockets only now to avoid the same port to be chosen twice
for s in sockets:
s.close()
return ports