-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathstager.py
More file actions
948 lines (844 loc) · 38.6 KB
/
stager.py
File metadata and controls
948 lines (844 loc) · 38.6 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
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
# 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.
#
"""Support for installing custom code and required dependencies.
Workflows, with the exception of very simple ones, are organized in multiple
modules and packages. Typically, these modules and packages have
dependencies on other standard libraries. Beam relies on the Python
setuptools package to handle these scenarios. For further details please read:
https://pythonhosted.org/an_example_pypi_project/setuptools.html
When a runner tries to run a pipeline it will check for a --requirements_file
and a --setup_file option.
If --setup_file is present then it is assumed that the folder containing the
file specified by the option has the typical layout required by setuptools and
it will run 'python setup.py sdist' to produce a source distribution. The
resulting tarball (a .tar or .tar.gz file) will be staged at the staging
location specified as job option. When a worker starts it will check for the
presence of this file and will run 'easy_install tarball' to install the
package in the worker.
If --requirements_file is present then the file specified by the option will be
staged in the staging location. When a worker starts it will check for the
presence of this file and will run 'pip install -r requirements.txt'. A
requirements file can be easily generated by running 'pip freeze -r
requirements.txt'. The reason a runner does not run this automatically is
because quite often only a small fraction of the dependencies present in a
requirements.txt file are actually needed for remote execution and therefore a
one-time manual trimming is desirable.
TODO(silviuc): Should we allow several setup packages?
TODO(silviuc): We should allow customizing the exact command for setup build.
"""
# pytype: skip-file
import glob
import hashlib
import logging
import os
import shutil
import subprocess
import sys
import tempfile
from importlib.metadata import distribution
from typing import Callable
from typing import List
from typing import Optional
from typing import Tuple
from urllib.parse import urlparse
from packaging import version
from apache_beam.internal import pickler
from apache_beam.internal.http_client import get_new_http
from apache_beam.io.filesystem import CompressionTypes
from apache_beam.io.filesystems import FileSystems
from apache_beam.options.pipeline_options import DebugOptions
from apache_beam.options.pipeline_options import PipelineOptions # pylint: disable=unused-import
from apache_beam.options.pipeline_options import SetupOptions
from apache_beam.options.pipeline_options import WorkerOptions
from apache_beam.portability import common_urns
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.runners.internal import names
from apache_beam.utils import processes
from apache_beam.utils import retry
# All constants are for internal use only; no backwards-compatibility
# guarantees.
# Standard file names used for staging files.
WORKFLOW_TARBALL_FILE = 'workflow.tar.gz'
REQUIREMENTS_FILE = 'requirements.txt'
EXTRA_PACKAGES_FILE = 'extra_packages.txt'
# Filename that stores the submission environment dependencies.
SUBMISSION_ENV_DEPENDENCIES_FILE = 'submission_environment_dependencies.txt'
# One of the choices for user to use for requirements cache during staging
SKIP_REQUIREMENTS_CACHE = 'skip'
_LOGGER = logging.getLogger(__name__)
def retry_on_non_zero_exit(exception):
if (isinstance(exception, processes.CalledProcessError) and
exception.returncode != 0):
return True
class Stager(object):
"""Abstract Stager identifies and copies the appropriate artifacts to the
staging location.
Implementation of this stager has to implement :func:`stage_artifact` and
:func:`commit_manifest`.
"""
_DEFAULT_CHUNK_SIZE = 2 << 20
def stage_artifact(
self, local_path_to_artifact: str, artifact_name: str,
sha256: str) -> None:
""" Stages the artifact to Stager._staging_location and adds artifact_name
to the manifest of artifacts that have been staged."""
raise NotImplementedError
def commit_manifest(self):
"""Commits manifest."""
raise NotImplementedError
@staticmethod
def _create_file_stage_to_artifact(local_path, staged_name):
return beam_runner_api_pb2.ArtifactInformation(
type_urn=common_urns.artifact_types.FILE.urn,
type_payload=beam_runner_api_pb2.ArtifactFilePayload(
path=local_path).SerializeToString(),
role_urn=common_urns.artifact_roles.STAGING_TO.urn,
role_payload=beam_runner_api_pb2.ArtifactStagingToRolePayload(
staged_name=staged_name).SerializeToString())
@staticmethod
def _create_file_pip_requirements_artifact(local_path):
return beam_runner_api_pb2.ArtifactInformation(
type_urn=common_urns.artifact_types.FILE.urn,
type_payload=beam_runner_api_pb2.ArtifactFilePayload(
path=local_path).SerializeToString(),
role_urn=common_urns.artifact_roles.PIP_REQUIREMENTS_FILE.urn)
@staticmethod
def extract_staging_tuple_iter(
artifacts: List[beam_runner_api_pb2.ArtifactInformation]):
for artifact in artifacts:
if artifact.type_urn == common_urns.artifact_types.FILE.urn:
file_payload = beam_runner_api_pb2.ArtifactFilePayload()
file_payload.ParseFromString(artifact.type_payload)
src = file_payload.path
sha256 = file_payload.sha256
if artifact.role_urn == common_urns.artifact_roles.STAGING_TO.urn:
role_payload = beam_runner_api_pb2.ArtifactStagingToRolePayload()
role_payload.ParseFromString(artifact.role_payload)
dst = role_payload.staged_name
elif (artifact.role_urn ==
common_urns.artifact_roles.PIP_REQUIREMENTS_FILE.urn):
dst = hashlib.sha256(artifact.SerializeToString()).hexdigest()
else:
raise RuntimeError("unknown role type: %s" % artifact.role_urn)
yield (src, dst, sha256)
else:
raise RuntimeError("unknown artifact type: %s" % artifact.type_urn)
@staticmethod
def create_job_resources(
options: PipelineOptions,
temp_dir: str,
build_setup_args: Optional[List[str]] = None,
pypi_requirements: Optional[List[str]] = None,
populate_requirements_cache: Optional[Callable[[str, str, bool],
None]] = None,
skip_prestaged_dependencies: Optional[bool] = False,
log_submission_env_dependencies: Optional[bool] = True,
):
"""For internal use only; no backwards-compatibility guarantees.
Creates (if needed) a list of job resources.
Args:
options: Command line options. More specifically the function will
expect requirements_file, setup_file, and save_main_session options
to be present.
temp_dir: Temporary folder where the resource building can happen. If
None then a unique temp directory will be created. Used only for
testing.
build_setup_args: A list of command line arguments used to build a
setup package. Used only if options.setup_file is not None. Used
only for testing.
pypi_requirements: A list of PyPI requirements used to cache source
packages.
populate_requirements_cache: Callable for populating the requirements
cache. Used only for testing.
skip_prestaged_dependencies: Skip staging dependencies that can be
added into SDK containers during prebuilding.
log_submission_env_dependencies: (Optional) param to stage and log
submission environment dependencies. Defaults to True.
Returns:
A list of ArtifactInformation to be used for staging resources.
Raises:
RuntimeError: If files specified are not found or error encountered
while trying to create the resources (e.g., build a setup package).
"""
resources: List[beam_runner_api_pb2.ArtifactInformation] = []
setup_options = options.view_as(SetupOptions)
use_beam_default_container = options.view_as(
WorkerOptions).sdk_container_image is None
pickler.set_library(setup_options.pickle_library)
# We can skip boot dependencies: apache beam sdk, python packages from
# requirements.txt, python packages from extra_packages and workflow tarball
# if we know we are using a dependency pre-installed sdk container image.
if not skip_prestaged_dependencies:
requirements_cache_path = (
os.path.join(tempfile.gettempdir(), 'beam-requirements-cache') if
(setup_options.requirements_cache
is None) else setup_options.requirements_cache)
if (setup_options.requirements_cache != SKIP_REQUIREMENTS_CACHE and
not os.path.exists(requirements_cache_path)):
os.makedirs(requirements_cache_path, exist_ok=True)
# Track packages to stage for this specific run.
packages_to_stage = set()
# Stage a requirements file if present.
if setup_options.requirements_file is not None:
if not os.path.isfile(setup_options.requirements_file):
raise RuntimeError(
'The file %s cannot be found. It was specified in the '
'--requirements_file command line option.' %
setup_options.requirements_file)
extra_packages, thinned_requirements_file = (
Stager._extract_local_packages(setup_options.requirements_file))
if extra_packages:
setup_options.extra_packages = (
setup_options.extra_packages or []) + extra_packages
resources.append(
Stager._create_file_stage_to_artifact(
thinned_requirements_file, REQUIREMENTS_FILE))
# Populate cache with packages from the requirement file option and
# stage the files in the cache.
if not use_beam_default_container:
_LOGGER.warning(
'When using a custom container image, prefer installing'
' additional PyPI dependencies directly into the image,'
' instead of specifying them via runtime options, '
'such as --requirements_file. ')
if setup_options.requirements_cache != SKIP_REQUIREMENTS_CACHE:
populate_cache_callable = (
populate_requirements_cache if populate_requirements_cache else
Stager._populate_requirements_cache)
downloaded_packages = populate_cache_callable(
setup_options.requirements_file,
requirements_cache_path,
setup_options.requirements_cache_only_sources)
if downloaded_packages:
packages_to_stage.update(downloaded_packages)
if pypi_requirements:
tf = tempfile.NamedTemporaryFile(mode='w', delete=False)
tf.writelines(pypi_requirements)
tf.close()
resources.append(Stager._create_file_pip_requirements_artifact(tf.name))
# Populate cache with packages from PyPI requirements and stage
# the files in the cache.
if setup_options.requirements_cache != SKIP_REQUIREMENTS_CACHE:
populate_cache_callable = (
populate_requirements_cache if populate_requirements_cache else
Stager._populate_requirements_cache)
downloaded_packages = populate_cache_callable(
tf.name,
requirements_cache_path,
setup_options.requirements_cache_only_sources)
if downloaded_packages:
packages_to_stage.update(downloaded_packages)
if (setup_options.requirements_cache != SKIP_REQUIREMENTS_CACHE) and (
setup_options.requirements_file is not None or pypi_requirements):
for pkg in packages_to_stage:
pkg_path = os.path.join(requirements_cache_path, pkg)
if os.path.exists(pkg_path):
resources.append(
Stager._create_file_stage_to_artifact(pkg_path, pkg))
# Handle a setup file if present.
# We will build the setup package locally and then copy it to the staging
# location because the staging location is a remote path and the file
# cannot be created directly there.
if setup_options.setup_file is not None:
if not os.path.isfile(setup_options.setup_file):
raise RuntimeError(
'The file %s cannot be found. It was specified in the '
'--setup_file command line option.' % setup_options.setup_file)
if os.path.basename(setup_options.setup_file) not in ('setup.py',
'pyproject.toml'):
raise RuntimeError(
'The --setup_file option expects the full path to a file named '
'setup.py or pyproject.toml instead of %s' %
setup_options.setup_file)
tarball_file = Stager._build_setup_package(
setup_options.setup_file, temp_dir, build_setup_args)
resources.append(
Stager._create_file_stage_to_artifact(
tarball_file, WORKFLOW_TARBALL_FILE))
if setup_options.files_to_stage is not None:
for file in setup_options.files_to_stage:
resources.append(
Stager._create_file_stage_to_artifact(
file, os.path.basename(file)))
# Handle extra local packages that should be staged.
if setup_options.extra_packages is not None:
resources.extend(
Stager._create_extra_packages(
setup_options.extra_packages, temp_dir=temp_dir))
if hasattr(setup_options, 'sdk_location'):
sdk_location = setup_options.sdk_location
if Stager._is_remote_path(sdk_location):
try:
resources.extend(
Stager._create_beam_sdk(
sdk_remote_location=setup_options.sdk_location,
temp_dir=temp_dir,
))
except:
raise RuntimeError(
'The --sdk_location option was used with an unsupported '
'type of location: %s' % sdk_location)
elif sdk_location == 'default':
# Use default location for a runner.
pass
elif sdk_location == 'container':
# Used in the past to indicate that SDK should be used from container
# image instead of being staged.
# Equivalent to 'default' now, leaving for backwards compatibility.
pass
else:
if os.path.isdir(setup_options.sdk_location):
sdk_path = os.path.join(
setup_options.sdk_location, names.STAGED_SDK_SOURCES_FILENAME)
else:
sdk_path = setup_options.sdk_location
if os.path.isfile(sdk_path):
_LOGGER.info('Copying Beam SDK "%s" to staging location.', sdk_path)
resources.append(
Stager._create_file_stage_to_artifact(
sdk_path,
Stager._desired_sdk_filename_in_staging_location(
setup_options.sdk_location)))
else:
if setup_options.sdk_location == 'default':
raise RuntimeError(
'Cannot find default Beam SDK tar file "%s"' % sdk_path)
elif not setup_options.sdk_location:
_LOGGER.info(
'Beam SDK will not be staged since --sdk_location '
'is empty.')
else:
raise RuntimeError(
'The file "%s" cannot be found. Its location was specified '
'by the --sdk_location command-line option.' % sdk_path)
# The following artifacts are not processed by python sdk container boot
# sequence in a setup mode and hence should not be skipped even if a
# prebuilt sdk container image is used.
# TODO(heejong): remove jar_packages experimental flag when cross-language
# dependency management is implemented for all runners.
# Handle jar packages that should be staged for Java SDK Harness.
jar_packages = options.view_as(DebugOptions).lookup_experiment(
'jar_packages')
if jar_packages is not None:
resources.extend(
Stager._create_jar_packages(
jar_packages.split(','), temp_dir=temp_dir))
# Pickle the main session if requested.
# We will create the pickled main session locally and then copy it to the
# staging location because the staging location is a remote path and the
# file cannot be created directly there.
if setup_options.save_main_session:
pickled_session_file = os.path.join(
temp_dir, names.PICKLED_MAIN_SESSION_FILE)
pickler.dump_session(pickled_session_file)
if os.path.exists(pickled_session_file):
resources.append(
Stager._create_file_stage_to_artifact(
pickled_session_file, names.PICKLED_MAIN_SESSION_FILE))
# stage the submission environment dependencies, if enabled.
if (log_submission_env_dependencies and
not options.view_as(DebugOptions).lookup_experiment(
'disable_logging_submission_environment')):
resources.extend(
Stager._create_stage_submission_env_dependencies(temp_dir))
return resources
def stage_job_resources(
self,
resources: List[Tuple[str, str, str]],
staging_location: Optional[str] = None):
"""For internal use only; no backwards-compatibility guarantees.
Stages job resources to staging_location.
Args:
resources: A list of tuples of local file paths and file names (no
paths) to be used for staging resources.
staging_location: Location to stage the file.
Returns:
A list of file names (no paths) for the resources staged. All the
files are assumed to be staged at staging_location.
Raises:
RuntimeError: If files specified are not found or error encountered
while trying to create the resources (e.g., build a setup package).
"""
# Make sure that all required options are specified.
if staging_location is None:
raise RuntimeError('The staging_location must be specified.')
staged_resources = []
for file_path, staged_path, sha256 in resources:
self.stage_artifact(
file_path, FileSystems.join(staging_location, staged_path), sha256)
staged_resources.append(staged_path)
return staged_resources
def create_and_stage_job_resources(
self,
options: PipelineOptions,
build_setup_args: Optional[List[str]] = None,
temp_dir: Optional[str] = None,
pypi_requirements: Optional[List[str]] = None,
populate_requirements_cache: Optional[Callable[[str, str, bool],
None]] = None,
staging_location: Optional[str] = None):
"""For internal use only; no backwards-compatibility guarantees.
Creates (if needed) and stages job resources to staging_location.
Args:
options: Command line options. More specifically the function will
expect requirements_file, setup_file, and save_main_session options
to be present.
build_setup_args: A list of command line arguments used to build a
setup package. Used only if options.setup_file is not None. Used
only for testing.
temp_dir: Temporary folder where the resource building can happen. If
None then a unique temp directory will be created. Used only for
testing.
pypi_requirements: A list of PyPI requirements used to cache source
packages.
populate_requirements_cache: Callable for populating the requirements
cache. Used only for testing.
staging_location: Location to stage the file.
Returns:
A tuple of:
1) retrieval token
2) A list of file names (no paths) for the resources staged. All the
files are assumed to be staged at staging_location
Raises:
RuntimeError: If files specified are not found or error encountered
while trying to create the resources (e.g., build a setup package).
"""
temp_dir = temp_dir or tempfile.mkdtemp()
resources = self.create_job_resources(
options,
temp_dir,
build_setup_args,
pypi_requirements=pypi_requirements,
populate_requirements_cache=populate_requirements_cache)
staged_resources = self.stage_job_resources(
list(Stager.extract_staging_tuple_iter(resources)), staging_location)
# Delete all temp files created while staging job resources.
shutil.rmtree(temp_dir)
retrieval_token = self.commit_manifest()
return retrieval_token, staged_resources
@staticmethod
@retry.with_exponential_backoff(num_retries=4)
def _download_file(from_url, to_path):
"""Downloads a file over http/https from a url or copy it from a remote
path to local path."""
if from_url.startswith('http://') or from_url.startswith('https://'):
# TODO(silviuc): We should cache downloads so we do not do it for every
# job.
try:
# We check if the file is actually there because wget returns a file
# even for a 404 response (file will contain the contents of the 404
# response).
response, content = get_new_http().request(from_url)
if int(response['status']) >= 400:
raise RuntimeError(
'Artifact not found at %s (response: %s)' % (from_url, response))
with open(to_path, 'wb') as f:
f.write(content)
except Exception:
_LOGGER.info('Failed to download Artifact from %s', from_url)
raise
else:
try:
read_handle = FileSystems.open(
from_url, compression_type=CompressionTypes.UNCOMPRESSED)
with read_handle as fin:
with open(to_path, 'wb') as f:
while True:
chunk = fin.read(Stager._DEFAULT_CHUNK_SIZE)
if not chunk:
break
f.write(chunk)
_LOGGER.info('Copied remote file from %s to %s.', from_url, to_path)
return
except Exception as e:
_LOGGER.info(
'Failed to download file from %s via apache_beam.io.filesystems.'
'Trying to copy directly. %s',
from_url,
repr(e))
if not os.path.isdir(os.path.dirname(to_path)):
_LOGGER.info(
'Created folder (since we have not done yet, and any errors '
'will follow): %s ',
os.path.dirname(to_path))
os.mkdir(os.path.dirname(to_path))
shutil.copyfile(from_url, to_path)
@staticmethod
def _is_remote_path(path):
return path.find('://') != -1
@staticmethod
def _create_jar_packages(
jar_packages, temp_dir) -> List[beam_runner_api_pb2.ArtifactInformation]:
"""Creates a list of local jar packages for Java SDK Harness.
:param jar_packages: Ordered list of local paths to jar packages to be
staged. Only packages on localfile system and GCS are supported.
:param temp_dir: Temporary folder where the resource building can happen.
:return: A list of tuples of local file paths and file names (no paths) for
the resource staged. All the files are assumed to be staged in
staging_location.
:raises:
RuntimeError: If files specified are not found or do not have expected
name patterns.
"""
resources: List[beam_runner_api_pb2.ArtifactInformation] = []
staging_temp_dir = tempfile.mkdtemp(dir=temp_dir)
local_packages: List[str] = []
for package in jar_packages:
if not os.path.basename(package).endswith('.jar'):
raise RuntimeError(
'The --experiment=\'jar_packages=\' option expects a full path '
'ending with ".jar" instead of %s' % package)
if not os.path.isfile(package):
if Stager._is_remote_path(package):
# Download remote package.
_LOGGER.info(
'Downloading jar package: %s locally before staging', package)
_, last_component = FileSystems.split(package)
local_file_path = FileSystems.join(staging_temp_dir, last_component)
Stager._download_file(package, local_file_path)
else:
raise RuntimeError(
'The file %s cannot be found. It was specified in the '
'--experiment=\'jar_packages=\' command line option.' % package)
else:
local_packages.append(package)
local_packages.extend([
FileSystems.join(staging_temp_dir, f)
for f in os.listdir(staging_temp_dir)
])
for package in local_packages:
basename = os.path.basename(package)
resources.append(Stager._create_file_stage_to_artifact(package, basename))
return resources
@staticmethod
def _create_extra_packages(
extra_packages,
temp_dir) -> List[beam_runner_api_pb2.ArtifactInformation]:
"""Creates a list of local extra packages.
Args:
extra_packages: Ordered list of local paths to extra packages to be
staged. Only packages on localfile system and GCS are supported.
temp_dir: Temporary folder where the resource building can happen.
Caller is responsible for cleaning up this folder after this function
returns.
Returns:
A list of ArtifactInformation of local file paths and file names
(no paths) for the resources staged. All the files are assumed to be
staged in staging_location.
Raises:
RuntimeError: If files specified are not found or do not have expected
name patterns.
"""
resources: List[beam_runner_api_pb2.ArtifactInformation] = []
staging_temp_dir = tempfile.mkdtemp(dir=temp_dir)
local_packages: List[str] = []
for package in extra_packages:
if not (os.path.basename(package).endswith('.tar') or
os.path.basename(package).endswith('.tar.gz') or
os.path.basename(package).endswith('.whl') or
os.path.basename(package).endswith('.zip')):
raise RuntimeError(
'The --extra_package option expects a full path ending with '
'".tar", ".tar.gz", ".whl" or ".zip" instead of %s' % package)
if os.path.basename(package).endswith('.whl'):
_LOGGER.warning(
'The .whl package "%s" provided in --extra_package '
'must be binary-compatible with the worker runtime environment.' %
package)
if not os.path.isfile(package):
if Stager._is_remote_path(package):
# Download remote package.
_LOGGER.info(
'Downloading extra package: %s locally before staging', package)
_, last_component = FileSystems.split(package)
local_file_path = FileSystems.join(staging_temp_dir, last_component)
Stager._download_file(package, local_file_path)
else:
raise RuntimeError(
'The file %s cannot be found. It was specified in the '
'--extra_packages command line option.' % package)
else:
local_packages.append(package)
local_packages.extend([
FileSystems.join(staging_temp_dir, f)
for f in os.listdir(staging_temp_dir)
])
for package in local_packages:
basename = os.path.basename(package)
resources.append(Stager._create_file_stage_to_artifact(package, basename))
# Create a file containing the list of extra packages and stage it.
# The file is important so that in the worker the packages are installed
# exactly in the order specified. This approach will avoid extra PyPI
# requests. For example if package A depends on package B and package A
# is installed first then the installer will try to satisfy the
# dependency on B by downloading the package from PyPI. If package B is
# installed first this is avoided.
with open(os.path.join(temp_dir, EXTRA_PACKAGES_FILE), 'wt') as f:
for package in local_packages:
f.write('%s\n' % os.path.basename(package))
# Note that the caller of this function is responsible for deleting the
# temporary folder where all temp files are created, including this one.
resources.append(
Stager._create_file_stage_to_artifact(
os.path.join(temp_dir, EXTRA_PACKAGES_FILE), EXTRA_PACKAGES_FILE))
return resources
@staticmethod
def _get_python_executable():
# Allow overriding the python executable to use for downloading and
# installing dependencies, otherwise use the python executable for
# the current process.
python_bin = os.environ.get('BEAM_PYTHON') or sys.executable
if not python_bin:
raise ValueError('Could not find Python executable.')
return python_bin
@staticmethod
def _remove_dependency_from_requirements(
requirements_file: str, dependency_to_remove: str, temp_directory_path):
"""Function to remove dependencies from a given requirements file."""
# read all the dependency names
with open(requirements_file, 'r') as f:
lines = f.readlines()
tmp_requirements_filename = os.path.join(
temp_directory_path, 'tmp_requirements.txt')
with open(tmp_requirements_filename, 'w') as tf:
for i in range(len(lines)):
if not lines[i].startswith(dependency_to_remove):
tf.write(lines[i])
return tmp_requirements_filename
@staticmethod
def _extract_local_packages(requirements_file):
local_deps = []
pypi_deps = []
with open(requirements_file, 'r') as fin:
for line in fin:
dep = line.strip()
if os.path.exists(dep):
local_deps.append(dep)
else:
pypi_deps.append(dep)
if local_deps:
with tempfile.NamedTemporaryFile(suffix='-requirements.txt',
delete=False) as fout:
fout.write('\n'.join(pypi_deps).encode('utf-8'))
return local_deps, fout.name
else:
return [], requirements_file
@staticmethod
def _get_platform_for_default_sdk_container():
"""
Get the platform for apache beam SDK container based on Pip version.
Note: pip is still expected to download compatible wheel of a package
with platform tag manylinux1 if the package on PyPI doesn't
have (manylinux2014) or (manylinux2010) wheels.
Reference: https://www.python.org/dev/peps/pep-0599/#id21
"""
# TODO(anandinguva): When https://github.com/pypa/pip/issues/10760 is
# addressed, download wheel based on glibc version in Beam's Python
# Base image
pip_version = distribution('pip').version
if version.parse(pip_version) >= version.parse('19.3'):
# pip can only recognize manylinux2014_x86_64 wheels
# from version 19.3.
return 'manylinux2014_x86_64'
else:
return 'manylinux2010_x86_64'
@staticmethod
@retry.with_exponential_backoff(
num_retries=4, retry_filter=retry_on_non_zero_exit)
def _populate_requirements_cache(
requirements_file, cache_dir, populate_cache_with_sdists=False):
# The 'pip download' command will not download again if it finds the
# tarball with the proper version already present.
# It will get the packages downloaded in the order they are presented in
# the requirements file and will download package dependencies.
# The apache-beam dependency is excluded from requirements cache population
# because we stage the SDK separately.
with tempfile.TemporaryDirectory() as temp_directory:
tmp_requirements_filepath = Stager._remove_dependency_from_requirements(
requirements_file=requirements_file,
dependency_to_remove='apache-beam',
temp_directory_path=temp_directory)
# Download to a temporary directory first, then copy to cache.
# This allows us to track exactly which packages are needed for this
# requirements file.
download_dir = tempfile.mkdtemp(dir=temp_directory)
cmd_args = [
Stager._get_python_executable(),
'-m',
'pip',
'download',
'--dest',
download_dir,
'--find-links',
cache_dir,
'-r',
tmp_requirements_filepath,
'--exists-action',
'i',
'--no-deps'
]
if populate_cache_with_sdists:
cmd_args.extend(['--no-binary', ':all:'])
else:
language_implementation_tag = 'cp'
abi_suffix = 'm' if sys.version_info < (3, 8) else ''
abi_tag = 'cp%d%d%s' % (
sys.version_info[0], sys.version_info[1], abi_suffix)
platform_tag = Stager._get_platform_for_default_sdk_container()
cmd_args.extend([
'--implementation',
language_implementation_tag,
'--abi',
abi_tag,
'--platform',
platform_tag
])
_LOGGER.info('Executing command: %s', cmd_args)
processes.check_output(cmd_args, stderr=processes.STDOUT)
# Get list of downloaded packages and copy them to the cache
downloaded_packages = set()
for pkg_file in os.listdir(download_dir):
downloaded_packages.add(pkg_file)
src_path = os.path.join(download_dir, pkg_file)
dest_path = os.path.join(cache_dir, pkg_file)
# Only copy if not already in cache
if not os.path.exists(dest_path):
shutil.copy2(src_path, dest_path)
return downloaded_packages
@staticmethod
def _build_setup_package(
setup_file: str,
temp_dir: str,
build_setup_args: Optional[List[str]] = None) -> str:
saved_current_directory = os.getcwd()
try:
os.chdir(os.path.dirname(setup_file))
if build_setup_args is None:
# if build is installed in the user env, use it to
# build the sdist else fallback to legacy
# setup.py sdist call for setup.py file.
try:
build_setup_args = [
Stager._get_python_executable(),
'-m',
'build',
'--no-isolation', # Otherwise, we need internet access to PyPI.
'--sdist',
'--outdir',
temp_dir,
os.path.dirname(setup_file),
]
_LOGGER.info('Executing command: %s', build_setup_args)
processes.check_output(build_setup_args)
except RuntimeError:
if setup_file.endswith('setup.py'):
build_setup_args = [
Stager._get_python_executable(),
os.path.basename(setup_file),
'sdist',
'--dist-dir',
temp_dir
]
_LOGGER.info('Executing command: %s', build_setup_args)
processes.check_output(build_setup_args)
else:
# If it's pyproject.toml and `python -m build` failed,
# there's no direct legacy fallback.
raise RuntimeError(
f"Failed to build package from '{setup_file}' using . "
f"'python -m build'. Please ensure that the 'build' module "
f"is installed and your project's build configuration is valid."
)
output_files = glob.glob(os.path.join(temp_dir, '*.tar.gz'))
if not output_files:
raise RuntimeError(
'File %s not found.' % os.path.join(temp_dir, '*.tar.gz'))
return output_files[0]
finally:
os.chdir(saved_current_directory)
@staticmethod
def _desired_sdk_filename_in_staging_location(sdk_location) -> str:
"""Returns the name that SDK file should have in the staging location.
Args:
sdk_location: Full path to SDK file.
"""
if sdk_location.endswith('.whl'):
_, wheel_filename = FileSystems.split(sdk_location)
if wheel_filename.startswith('apache_beam'):
return wheel_filename
else:
raise RuntimeError('Unrecognized SDK wheel file: %s' % sdk_location)
else:
return names.STAGED_SDK_SOURCES_FILENAME
@staticmethod
def _create_beam_sdk(
sdk_remote_location,
temp_dir) -> List[beam_runner_api_pb2.ArtifactInformation]:
"""Creates a Beam SDK file with the appropriate version.
Args:
sdk_remote_location: A URL from which the file can be downloaded or a
remote file location. The SDK file can be a tarball or a wheel.
temp_dir: path to temporary location where the file should be
downloaded.
Returns:
A list of ArtifactInformation of local files path and SDK files that
will be staged to the staging location.
Raises:
RuntimeError: if staging was not successful.
"""
sdk_remote_parsed = urlparse(sdk_remote_location)
sdk_remote_filename = os.path.basename(sdk_remote_parsed.path)
local_download_file = os.path.join(temp_dir, sdk_remote_filename)
Stager._download_file(sdk_remote_location, local_download_file)
staged_name = Stager._desired_sdk_filename_in_staging_location(
local_download_file)
_LOGGER.info('Staging Beam SDK from %s', sdk_remote_location)
return [
Stager._create_file_stage_to_artifact(local_download_file, staged_name)
]
@staticmethod
def _create_stage_submission_env_dependencies(temp_dir):
"""Create and stage a file with list of dependencies installed in the
submission environment.
This list can be used at runtime to compare against the dependencies in the
runtime environment. This allows runners to warn users about any potential
dependency mismatches and help debug issues related to
environment mismatches.
Args:
temp_dir: path to temporary location where the file should be
downloaded.
Returns:
A list of ArtifactInformation of local file path that will be staged to
the staging location.
"""
try:
local_dependency_file_path = os.path.join(
temp_dir, SUBMISSION_ENV_DEPENDENCIES_FILE)
dependencies = subprocess.check_output(
[sys.executable, '-m', 'pip', 'freeze'])
local_python_path = f"Python Path: {sys.executable}\n"
with open(local_dependency_file_path, 'w') as f:
f.write(local_python_path + str(dependencies))
return [
Stager._create_file_stage_to_artifact(
local_dependency_file_path, SUBMISSION_ENV_DEPENDENCIES_FILE),
]
except Exception as e:
_LOGGER.warning(
"Couldn't stage a list of installed dependencies in "
"submission environment. Got exception: %s",
e)
return []