github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/environments.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 """Environments concepts. 19 20 For internal use only. No backwards compatibility guarantees.""" 21 22 # pytype: skip-file 23 24 import json 25 import logging 26 import sys 27 import tempfile 28 from types import MappingProxyType 29 from typing import TYPE_CHECKING 30 from typing import Any 31 from typing import Callable 32 from typing import Dict 33 from typing import Iterable 34 from typing import Iterator 35 from typing import List 36 from typing import Mapping 37 from typing import Optional 38 from typing import Set 39 from typing import Tuple 40 from typing import Type 41 from typing import TypeVar 42 from typing import Union 43 from typing import overload 44 45 from google.protobuf import message 46 47 from apache_beam import coders 48 from apache_beam.options.pipeline_options import SetupOptions 49 from apache_beam.portability import common_urns 50 from apache_beam.portability import python_urns 51 from apache_beam.portability.api import beam_runner_api_pb2 52 from apache_beam.portability.api import endpoints_pb2 53 from apache_beam.runners.portability import stager 54 from apache_beam.runners.portability.sdk_container_builder import SdkContainerImageBuilder 55 from apache_beam.transforms.resources import resource_hints_from_options 56 from apache_beam.utils import proto_utils 57 58 if TYPE_CHECKING: 59 from apache_beam.options.pipeline_options import PipelineOptions 60 from apache_beam.options.pipeline_options import PortableOptions 61 from apache_beam.runners.pipeline_context import PipelineContext 62 63 __all__ = [ 64 'Environment', 65 'DefaultEnvironment', 66 'DockerEnvironment', 67 'ProcessEnvironment', 68 'ExternalEnvironment', 69 'EmbeddedPythonEnvironment', 70 'EmbeddedPythonGrpcEnvironment', 71 'SubprocessSDKEnvironment', 72 'PyPIArtifactRegistry' 73 ] 74 75 T = TypeVar('T') 76 EnvironmentT = TypeVar('EnvironmentT', bound='Environment') 77 ConstructorFn = Callable[[ 78 Optional[Any], 79 Iterable[str], 80 Iterable[beam_runner_api_pb2.ArtifactInformation], 81 Mapping[str, bytes], 82 'PipelineContext' 83 ], 84 Any] 85 86 87 def looks_like_json(s): 88 import re 89 return re.match(r'\s*\{.*\}\s*$', s) 90 91 92 APACHE_BEAM_DOCKER_IMAGE_PREFIX = 'apache/beam' 93 APACHE_BEAM_JAVA_CONTAINER_NAME_PREFIX = 'beam_java' 94 SDK_VERSION_CAPABILITY_PREFIX = 'beam:version:sdk_base:' 95 96 97 def is_apache_beam_container(container_image): 98 return container_image and container_image.startswith( 99 APACHE_BEAM_DOCKER_IMAGE_PREFIX) 100 101 102 class Environment(object): 103 """Abstract base class for environments. 104 105 Represents a type and configuration of environment. 106 Each type of Environment should have a unique urn. 107 108 For internal use only. No backwards compatibility guarantees. 109 """ 110 111 _known_urns = {} # type: Dict[str, Tuple[Optional[type], ConstructorFn]] 112 _urn_to_env_cls = {} # type: Dict[str, type] 113 114 def __init__(self, 115 capabilities=(), # type: Iterable[str] 116 artifacts=(), # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 117 resource_hints=None, # type: Optional[Mapping[str, bytes]] 118 ): 119 # type: (...) -> None 120 self._capabilities = capabilities 121 self._artifacts = sorted(artifacts, key=lambda x: x.SerializeToString()) 122 # Hints on created environments should be immutable since pipeline context 123 # stores environments in hash maps and we use hints to compute the hash. 124 self._resource_hints = MappingProxyType( 125 dict(resource_hints) if resource_hints else {}) 126 127 def __eq__(self, other): 128 return ( 129 self.__class__ == other.__class__ and 130 self._artifacts == other._artifacts 131 # Assuming that we don't have instances of the same Environment subclass 132 # with different set of capabilities. 133 and self._resource_hints == other._resource_hints) 134 135 def __hash__(self): 136 # type: () -> int 137 return hash((self.__class__, frozenset(self._resource_hints.items()))) 138 139 def artifacts(self): 140 # type: () -> Iterable[beam_runner_api_pb2.ArtifactInformation] 141 return self._artifacts 142 143 def to_runner_api_parameter(self, context): 144 # type: (PipelineContext) -> Tuple[str, Optional[Union[message.Message, bytes, str]]] 145 raise NotImplementedError 146 147 def capabilities(self): 148 # type: () -> Iterable[str] 149 return self._capabilities 150 151 def resource_hints(self): 152 # type: () -> Mapping[str, bytes] 153 return self._resource_hints 154 155 @classmethod 156 @overload 157 def register_urn( 158 cls, 159 urn, # type: str 160 parameter_type, # type: Type[T] 161 ): 162 # type: (...) -> Callable[[Union[type, Callable[[T, Iterable[str], PipelineContext], Any]]], Callable[[T, Iterable[str], PipelineContext], Any]] 163 pass 164 165 @classmethod 166 @overload 167 def register_urn( 168 cls, 169 urn, # type: str 170 parameter_type, # type: None 171 ): 172 # type: (...) -> Callable[[Union[type, Callable[[bytes, Iterable[str], Iterable[beam_runner_api_pb2.ArtifactInformation], PipelineContext], Any]]], Callable[[bytes, Iterable[str], PipelineContext], Any]] 173 pass 174 175 @classmethod 176 @overload 177 def register_urn(cls, 178 urn, # type: str 179 parameter_type, # type: Type[T] 180 constructor # type: Callable[[T, Iterable[str], Iterable[beam_runner_api_pb2.ArtifactInformation], PipelineContext], Any] 181 ): 182 # type: (...) -> None 183 pass 184 185 @classmethod 186 @overload 187 def register_urn(cls, 188 urn, # type: str 189 parameter_type, # type: None 190 constructor # type: Callable[[bytes, Iterable[str], Iterable[beam_runner_api_pb2.ArtifactInformation], PipelineContext], Any] 191 ): 192 # type: (...) -> None 193 pass 194 195 @classmethod 196 def register_urn(cls, urn, parameter_type, constructor=None): 197 def register(constructor): 198 if isinstance(constructor, type): 199 constructor.from_runner_api_parameter = register( 200 constructor.from_runner_api_parameter) 201 # register environment urn to environment class 202 cls._urn_to_env_cls[urn] = constructor 203 return constructor 204 205 else: 206 cls._known_urns[urn] = parameter_type, constructor 207 return staticmethod(constructor) 208 209 if constructor: 210 # Used as a statement. 211 register(constructor) 212 else: 213 # Used as a decorator. 214 return register 215 216 @classmethod 217 def get_env_cls_from_urn(cls, urn): 218 # type: (str) -> Type[Environment] 219 return cls._urn_to_env_cls[urn] 220 221 def to_runner_api(self, context): 222 # type: (PipelineContext) -> beam_runner_api_pb2.Environment 223 urn, typed_param = self.to_runner_api_parameter(context) 224 return beam_runner_api_pb2.Environment( 225 urn=urn, 226 payload=typed_param.SerializeToString() if isinstance( 227 typed_param, message.Message) else typed_param if 228 (isinstance(typed_param, bytes) or 229 typed_param is None) else typed_param.encode('utf-8'), 230 capabilities=self.capabilities(), 231 dependencies=self.artifacts(), 232 resource_hints=self.resource_hints()) 233 234 @classmethod 235 def from_runner_api(cls, 236 proto, # type: Optional[beam_runner_api_pb2.Environment] 237 context # type: PipelineContext 238 ): 239 # type: (...) -> Optional[Environment] 240 if proto is None or not proto.urn: 241 return None 242 parameter_type, constructor = cls._known_urns[proto.urn] 243 244 return constructor( 245 proto_utils.parse_Bytes(proto.payload, parameter_type), 246 proto.capabilities, 247 proto.dependencies, 248 proto.resource_hints, 249 context) 250 251 @classmethod 252 def from_options(cls, options): 253 # type: (Type[EnvironmentT], PortableOptions) -> EnvironmentT 254 255 """Creates an Environment object from PortableOptions. 256 257 Args: 258 options: The PortableOptions object. 259 """ 260 raise NotImplementedError 261 262 263 @Environment.register_urn(common_urns.environments.DEFAULT.urn, None) 264 class DefaultEnvironment(Environment): 265 """Used as a stub when context is missing a default environment.""" 266 def to_runner_api_parameter(self, context): 267 return common_urns.environments.DEFAULT.urn, None 268 269 @staticmethod 270 def from_runner_api_parameter(payload, # type: beam_runner_api_pb2.DockerPayload 271 capabilities, # type: Iterable[str] 272 artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 273 resource_hints, # type: Mapping[str, bytes] 274 context # type: PipelineContext 275 ): 276 # type: (...) -> DefaultEnvironment 277 return DefaultEnvironment( 278 capabilities=capabilities, 279 artifacts=artifacts, 280 resource_hints=resource_hints) 281 282 283 @Environment.register_urn( 284 common_urns.environments.DOCKER.urn, beam_runner_api_pb2.DockerPayload) 285 class DockerEnvironment(Environment): 286 def __init__( 287 self, 288 container_image=None, # type: Optional[str] 289 capabilities=(), # type: Iterable[str] 290 artifacts=(), # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 291 resource_hints=None, # type: Optional[Mapping[str, bytes]] 292 ): 293 super().__init__(capabilities, artifacts, resource_hints) 294 if container_image: 295 logging.info( 296 'Using provided Python SDK container image: %s' % (container_image)) 297 self.container_image = container_image 298 else: 299 logging.info('No image given, using default Python SDK image') 300 self.container_image = self.default_docker_image() 301 302 logging.info( 303 'Python SDK container image set to "%s" for Docker environment' % 304 (self.container_image)) 305 306 def __eq__(self, other): 307 return ( 308 super().__eq__(other) and self.container_image == other.container_image) 309 310 def __hash__(self): 311 return hash((super().__hash__(), self.container_image)) 312 313 def __repr__(self): 314 return 'DockerEnvironment(container_image=%s)' % self.container_image 315 316 def to_runner_api_parameter(self, context): 317 # type: (PipelineContext) -> Tuple[str, beam_runner_api_pb2.DockerPayload] 318 return ( 319 common_urns.environments.DOCKER.urn, 320 beam_runner_api_pb2.DockerPayload(container_image=self.container_image)) 321 322 @staticmethod 323 def from_runner_api_parameter(payload, # type: beam_runner_api_pb2.DockerPayload 324 capabilities, # type: Iterable[str] 325 artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 326 resource_hints, # type: Mapping[str, bytes] 327 context # type: PipelineContext 328 ): 329 # type: (...) -> DockerEnvironment 330 return DockerEnvironment( 331 container_image=payload.container_image, 332 capabilities=capabilities, 333 artifacts=artifacts, 334 resource_hints=resource_hints) 335 336 @classmethod 337 def from_options(cls, options): 338 # type: (PortableOptions) -> DockerEnvironment 339 if options.view_as(SetupOptions).prebuild_sdk_container_engine: 340 prebuilt_container_image = SdkContainerImageBuilder.build_container_image( 341 options) 342 return cls.from_container_image( 343 container_image=prebuilt_container_image, 344 artifacts=python_sdk_dependencies(options), 345 resource_hints=resource_hints_from_options(options), 346 ) 347 return cls.from_container_image( 348 container_image=options.lookup_environment_option( 349 'docker_container_image') or options.environment_config, 350 artifacts=python_sdk_dependencies(options), 351 resource_hints=resource_hints_from_options(options), 352 ) 353 354 @classmethod 355 def from_container_image( 356 cls, container_image, artifacts=(), resource_hints=None): 357 # type: (str, Iterable[beam_runner_api_pb2.ArtifactInformation], Optional[Mapping[str, bytes]]) -> DockerEnvironment 358 return cls( 359 container_image=container_image, 360 capabilities=python_sdk_docker_capabilities(), 361 artifacts=artifacts, 362 resource_hints=resource_hints) 363 364 @staticmethod 365 def default_docker_image(): 366 # type: () -> str 367 from apache_beam import version as beam_version 368 369 sdk_version = beam_version.__version__ 370 version_suffix = '.'.join([str(i) for i in sys.version_info[0:2]]) 371 372 image = ( 373 APACHE_BEAM_DOCKER_IMAGE_PREFIX + 374 '_python{version_suffix}_sdk:{tag}'.format( 375 version_suffix=version_suffix, tag=sdk_version)) 376 logging.info('Default Python SDK image for environment is %s' % (image)) 377 return image 378 379 380 @Environment.register_urn( 381 common_urns.environments.PROCESS.urn, beam_runner_api_pb2.ProcessPayload) 382 class ProcessEnvironment(Environment): 383 def __init__( 384 self, 385 command, # type: str 386 os='', # type: str 387 arch='', # type: str 388 env=None, # type: Optional[Mapping[str, str]] 389 capabilities=(), # type: Iterable[str] 390 artifacts=(), # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 391 resource_hints=None, # type: Optional[Mapping[str, bytes]] 392 ): 393 # type: (...) -> None 394 super().__init__(capabilities, artifacts, resource_hints) 395 self.command = command 396 self.os = os 397 self.arch = arch 398 self.env = env or {} 399 400 def __eq__(self, other): 401 return ( 402 super().__eq__(other) and self.command == other.command and 403 self.os == other.os and self.arch == other.arch and 404 self.env == other.env) 405 406 def __hash__(self): 407 # type: () -> int 408 return hash(( 409 super().__hash__(), 410 self.command, 411 self.os, 412 self.arch, 413 frozenset(self.env.items()))) 414 415 def __repr__(self): 416 # type: () -> str 417 repr_parts = ['command=%s' % self.command] 418 if self.os: 419 repr_parts.append('os=%s' % self.os) 420 if self.arch: 421 repr_parts.append('arch=%s' % self.arch) 422 repr_parts.append('env=%s' % self.env) 423 return 'ProcessEnvironment(%s)' % ','.join(repr_parts) 424 425 def to_runner_api_parameter(self, context): 426 # type: (PipelineContext) -> Tuple[str, beam_runner_api_pb2.ProcessPayload] 427 return ( 428 common_urns.environments.PROCESS.urn, 429 beam_runner_api_pb2.ProcessPayload( 430 os=self.os, arch=self.arch, command=self.command, env=self.env)) 431 432 @staticmethod 433 def from_runner_api_parameter(payload, 434 capabilities, # type: Iterable[str] 435 artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 436 resource_hints, # type: Mapping[str, bytes] 437 context # type: PipelineContext 438 ): 439 # type: (...) -> ProcessEnvironment 440 return ProcessEnvironment( 441 command=payload.command, 442 os=payload.os, 443 arch=payload.arch, 444 env=payload.env, 445 capabilities=capabilities, 446 artifacts=artifacts, 447 resource_hints=resource_hints, 448 ) 449 450 @staticmethod 451 def parse_environment_variables(variables): 452 env = {} 453 for var in variables: 454 try: 455 name, value = var.split('=', 1) 456 env[name] = value 457 except ValueError: 458 raise ValueError( 459 'Invalid process_variables "%s" (expected assignment in the ' 460 'form "FOO=bar").' % var) 461 return env 462 463 @classmethod 464 def from_options(cls, options): 465 # type: (PortableOptions) -> ProcessEnvironment 466 if options.environment_config: 467 config = json.loads(options.environment_config) 468 return cls( 469 config.get('command'), 470 os=config.get('os', ''), 471 arch=config.get('arch', ''), 472 env=config.get('env', ''), 473 capabilities=python_sdk_capabilities(), 474 artifacts=python_sdk_dependencies(options), 475 resource_hints=resource_hints_from_options(options), 476 ) 477 env = cls.parse_environment_variables( 478 options.lookup_environment_option('process_variables').split(',') 479 if options.lookup_environment_option('process_variables') else []) 480 return cls( 481 options.lookup_environment_option('process_command'), 482 env=env, 483 capabilities=python_sdk_capabilities(), 484 artifacts=python_sdk_dependencies(options), 485 resource_hints=resource_hints_from_options(options), 486 ) 487 488 489 @Environment.register_urn( 490 common_urns.environments.EXTERNAL.urn, beam_runner_api_pb2.ExternalPayload) 491 class ExternalEnvironment(Environment): 492 def __init__( 493 self, 494 url, # type: str 495 params=None, # type: Optional[Mapping[str, str]] 496 capabilities=(), # type: Iterable[str] 497 artifacts=(), # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 498 resource_hints=None, # type: Optional[Mapping[str, bytes]] 499 ): 500 super().__init__(capabilities, artifacts, resource_hints) 501 self.url = url 502 self.params = params 503 504 def __eq__(self, other): 505 return ( 506 super().__eq__(other) and self.url == other.url and 507 self.params == other.params) 508 509 def __hash__(self): 510 # type: () -> int 511 return hash(( 512 super().__hash__(), 513 self.url, 514 frozenset(self.params.items()) if self.params is not None else None)) 515 516 def __repr__(self): 517 # type: () -> str 518 return 'ExternalEnvironment(url=%s,params=%s)' % (self.url, self.params) 519 520 def to_runner_api_parameter(self, context): 521 # type: (PipelineContext) -> Tuple[str, beam_runner_api_pb2.ExternalPayload] 522 return ( 523 common_urns.environments.EXTERNAL.urn, 524 beam_runner_api_pb2.ExternalPayload( 525 endpoint=endpoints_pb2.ApiServiceDescriptor(url=self.url), 526 params=self.params)) 527 528 @staticmethod 529 def from_runner_api_parameter(payload, # type: beam_runner_api_pb2.ExternalPayload 530 capabilities, # type: Iterable[str] 531 artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 532 resource_hints, # type: Mapping[str, bytes] 533 context # type: PipelineContext 534 ): 535 # type: (...) -> ExternalEnvironment 536 return ExternalEnvironment( 537 payload.endpoint.url, 538 params=payload.params or None, 539 capabilities=capabilities, 540 artifacts=artifacts, 541 resource_hints=resource_hints) 542 543 @classmethod 544 def from_options(cls, options): 545 # type: (PortableOptions) -> ExternalEnvironment 546 if looks_like_json(options.environment_config): 547 config = json.loads(options.environment_config) 548 url = config.get('url') 549 if not url: 550 raise ValueError('External environment endpoint must be set.') 551 params = config.get('params') 552 elif options.environment_config: 553 url = options.environment_config 554 params = None 555 else: 556 url = options.lookup_environment_option('external_service_address') 557 params = None 558 559 return cls( 560 url, 561 params=params, 562 capabilities=python_sdk_capabilities(), 563 artifacts=python_sdk_dependencies(options), 564 resource_hints=resource_hints_from_options(options)) 565 566 567 @Environment.register_urn(python_urns.EMBEDDED_PYTHON, None) 568 class EmbeddedPythonEnvironment(Environment): 569 def to_runner_api_parameter(self, context): 570 # type: (PipelineContext) -> Tuple[str, None] 571 return python_urns.EMBEDDED_PYTHON, None 572 573 @staticmethod 574 def from_runner_api_parameter(unused_payload, # type: None 575 capabilities, # type: Iterable[str] 576 artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 577 resource_hints, # type: Mapping[str, bytes] 578 context # type: PipelineContext 579 ): 580 # type: (...) -> EmbeddedPythonEnvironment 581 return EmbeddedPythonEnvironment(capabilities, artifacts, resource_hints) 582 583 @classmethod 584 def from_options(cls, options): 585 # type: (PortableOptions) -> EmbeddedPythonEnvironment 586 return cls( 587 capabilities=python_sdk_capabilities(), 588 artifacts=python_sdk_dependencies(options), 589 resource_hints=resource_hints_from_options(options), 590 ) 591 592 @classmethod 593 def default(cls): 594 # type: () -> EmbeddedPythonEnvironment 595 return cls(capabilities=python_sdk_capabilities(), artifacts=()) 596 597 598 @Environment.register_urn(python_urns.EMBEDDED_PYTHON_GRPC, bytes) 599 class EmbeddedPythonGrpcEnvironment(Environment): 600 def __init__( 601 self, 602 state_cache_size=None, 603 data_buffer_time_limit_ms=None, 604 capabilities=(), 605 artifacts=(), 606 resource_hints=None, 607 ): 608 super().__init__(capabilities, artifacts, resource_hints) 609 self.state_cache_size = state_cache_size 610 self.data_buffer_time_limit_ms = data_buffer_time_limit_ms 611 612 def __eq__(self, other): 613 return ( 614 super().__eq__(other) and 615 self.state_cache_size == other.state_cache_size and 616 self.data_buffer_time_limit_ms == other.data_buffer_time_limit_ms) 617 618 def __hash__(self): 619 # type: () -> int 620 return hash(( 621 super().__hash__(), 622 self.state_cache_size, 623 self.data_buffer_time_limit_ms)) 624 625 def __repr__(self): 626 # type: () -> str 627 repr_parts = [] 628 if not self.state_cache_size is None: 629 repr_parts.append('state_cache_size=%d' % self.state_cache_size) 630 if not self.data_buffer_time_limit_ms is None: 631 repr_parts.append( 632 'data_buffer_time_limit_ms=%d' % self.data_buffer_time_limit_ms) 633 return 'EmbeddedPythonGrpcEnvironment(%s)' % ','.join(repr_parts) 634 635 def to_runner_api_parameter(self, context): 636 # type: (PipelineContext) -> Tuple[str, bytes] 637 params = {} 638 if self.state_cache_size is not None: 639 params['state_cache_size'] = self.state_cache_size 640 if self.data_buffer_time_limit_ms is not None: 641 params['data_buffer_time_limit_ms'] = self.data_buffer_time_limit_ms 642 payload = json.dumps(params).encode('utf-8') 643 return python_urns.EMBEDDED_PYTHON_GRPC, payload 644 645 @staticmethod 646 def from_runner_api_parameter(payload, # type: bytes 647 capabilities, # type: Iterable[str] 648 artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 649 resource_hints, # type: Mapping[str, bytes] 650 context # type: PipelineContext 651 ): 652 # type: (...) -> EmbeddedPythonGrpcEnvironment 653 if payload: 654 config = EmbeddedPythonGrpcEnvironment.parse_config( 655 payload.decode('utf-8')) 656 return EmbeddedPythonGrpcEnvironment( 657 state_cache_size=config.get('state_cache_size'), 658 data_buffer_time_limit_ms=config.get('data_buffer_time_limit_ms'), 659 capabilities=capabilities, 660 artifacts=artifacts, 661 resource_hints=resource_hints) 662 else: 663 return EmbeddedPythonGrpcEnvironment() 664 665 @classmethod 666 def from_options(cls, options): 667 # type: (PortableOptions) -> EmbeddedPythonGrpcEnvironment 668 if options.environment_config: 669 config = EmbeddedPythonGrpcEnvironment.parse_config( 670 options.environment_config) 671 return cls( 672 state_cache_size=config.get('state_cache_size'), 673 data_buffer_time_limit_ms=config.get('data_buffer_time_limit_ms'), 674 capabilities=python_sdk_capabilities(), 675 artifacts=python_sdk_dependencies(options)) 676 else: 677 return cls( 678 capabilities=python_sdk_capabilities(), 679 artifacts=python_sdk_dependencies(options), 680 resource_hints=resource_hints_from_options(options)) 681 682 @staticmethod 683 def parse_config(s): 684 # type: (str) -> Dict[str, Any] 685 if looks_like_json(s): 686 config_dict = json.loads(s) 687 if 'state_cache_size' in config_dict: 688 config_dict['state_cache_size'] = int(config_dict['state_cache_size']) 689 690 if 'data_buffer_time_limit_ms' in config_dict: 691 config_dict['data_buffer_time_limit_ms'] = \ 692 int(config_dict['data_buffer_time_limit_ms']) 693 return config_dict 694 else: 695 return {'state_cache_size': int(s)} 696 697 @classmethod 698 def default(cls): 699 # type: () -> EmbeddedPythonGrpcEnvironment 700 return cls(capabilities=python_sdk_capabilities(), artifacts=()) 701 702 703 @Environment.register_urn(python_urns.SUBPROCESS_SDK, bytes) 704 class SubprocessSDKEnvironment(Environment): 705 def __init__( 706 self, 707 command_string, # type: str 708 capabilities=(), # type: Iterable[str] 709 artifacts=(), # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 710 resource_hints=None, # type: Optional[Mapping[str, bytes]] 711 ): 712 super().__init__(capabilities, artifacts, resource_hints) 713 self.command_string = command_string 714 715 def __eq__(self, other): 716 return ( 717 super().__eq__(other) and self.command_string == other.command_string) 718 719 def __hash__(self): 720 # type: () -> int 721 return hash((super().__hash__(), self.command_string)) 722 723 def __repr__(self): 724 # type: () -> str 725 return 'SubprocessSDKEnvironment(command_string=%s)' % self.command_string 726 727 def to_runner_api_parameter(self, context): 728 # type: (PipelineContext) -> Tuple[str, bytes] 729 return python_urns.SUBPROCESS_SDK, self.command_string.encode('utf-8') 730 731 @staticmethod 732 def from_runner_api_parameter(payload, # type: bytes 733 capabilities, # type: Iterable[str] 734 artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] 735 resource_hints, # type: Mapping[str, bytes] 736 context # type: PipelineContext 737 ): 738 # type: (...) -> SubprocessSDKEnvironment 739 return SubprocessSDKEnvironment( 740 payload.decode('utf-8'), capabilities, artifacts, resource_hints) 741 742 @classmethod 743 def from_options(cls, options): 744 # type: (PortableOptions) -> SubprocessSDKEnvironment 745 return cls( 746 options.environment_config, 747 capabilities=python_sdk_capabilities(), 748 artifacts=python_sdk_dependencies(options), 749 resource_hints=resource_hints_from_options(options)) 750 751 @classmethod 752 def from_command_string(cls, command_string): 753 # type: (str) -> SubprocessSDKEnvironment 754 return cls( 755 command_string, capabilities=python_sdk_capabilities(), artifacts=()) 756 757 758 class PyPIArtifactRegistry(object): 759 _registered_artifacts = set() # type: Set[Tuple[str, str]] 760 761 @classmethod 762 def register_artifact(cls, name, version): 763 cls._registered_artifacts.add((name, version)) 764 765 @classmethod 766 def get_artifacts(cls): 767 for artifact in cls._registered_artifacts: 768 yield artifact 769 770 771 def python_sdk_capabilities(): 772 # type: () -> List[str] 773 return list(_python_sdk_capabilities_iter()) 774 775 776 def python_sdk_docker_capabilities(): 777 return python_sdk_capabilities() + [common_urns.protocols.SIBLING_WORKERS.urn] 778 779 780 def sdk_base_version_capability(): 781 return ( 782 SDK_VERSION_CAPABILITY_PREFIX + DockerEnvironment.default_docker_image()) 783 784 785 def _python_sdk_capabilities_iter(): 786 # type: () -> Iterator[str] 787 for urn_spec in common_urns.coders.__dict__.values(): 788 if getattr(urn_spec, 'urn', None) in coders.Coder._known_urns: 789 yield urn_spec.urn 790 yield common_urns.protocols.LEGACY_PROGRESS_REPORTING.urn 791 yield common_urns.protocols.HARNESS_MONITORING_INFOS.urn 792 yield common_urns.protocols.WORKER_STATUS.urn 793 yield python_urns.PACKED_COMBINE_FN 794 yield sdk_base_version_capability() 795 yield common_urns.sdf_components.TRUNCATE_SIZED_RESTRICTION.urn 796 yield common_urns.primitives.TO_STRING.urn 797 yield common_urns.protocols.DATA_SAMPLING.urn 798 799 800 def python_sdk_dependencies(options, tmp_dir=None): 801 if tmp_dir is None: 802 tmp_dir = tempfile.mkdtemp() 803 skip_prestaged_dependencies = options.view_as( 804 SetupOptions).prebuild_sdk_container_engine is not None 805 return stager.Stager.create_job_resources( 806 options, 807 tmp_dir, 808 pypi_requirements=[ 809 artifact[0] + artifact[1] 810 for artifact in PyPIArtifactRegistry.get_artifacts() 811 ], 812 skip_prestaged_dependencies=skip_prestaged_dependencies)