github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/options/pipeline_options.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 """Pipeline options obtained from command line parsing.""" 19 20 # pytype: skip-file 21 22 import argparse 23 import json 24 import logging 25 import os 26 from typing import Any 27 from typing import Callable 28 from typing import Dict 29 from typing import List 30 from typing import Optional 31 from typing import Type 32 from typing import TypeVar 33 34 import apache_beam as beam 35 from apache_beam.options.value_provider import RuntimeValueProvider 36 from apache_beam.options.value_provider import StaticValueProvider 37 from apache_beam.options.value_provider import ValueProvider 38 from apache_beam.transforms.display import HasDisplayData 39 40 __all__ = [ 41 'PipelineOptions', 42 'StandardOptions', 43 'TypeOptions', 44 'DirectOptions', 45 'GoogleCloudOptions', 46 'AzureOptions', 47 'HadoopFileSystemOptions', 48 'WorkerOptions', 49 'DebugOptions', 50 'ProfilingOptions', 51 'SetupOptions', 52 'TestOptions', 53 'S3Options' 54 ] 55 56 PipelineOptionsT = TypeVar('PipelineOptionsT', bound='PipelineOptions') 57 58 _LOGGER = logging.getLogger(__name__) 59 60 # Map defined with option names to flag names for boolean options 61 # that have a destination(dest) in parser.add_argument() different 62 # from the flag name and whose default value is `None`. 63 _FLAG_THAT_SETS_FALSE_VALUE = {'use_public_ips': 'no_use_public_ips'} 64 65 66 def _static_value_provider_of(value_type): 67 """"Helper function to plug a ValueProvider into argparse. 68 69 Args: 70 value_type: the type of the value. Since the type param of argparse's 71 add_argument will always be ValueProvider, we need to 72 preserve the type of the actual value. 73 Returns: 74 A partially constructed StaticValueProvider in the form of a function. 75 76 """ 77 def _f(value): 78 _f.__name__ = value_type.__name__ 79 return StaticValueProvider(value_type, value) 80 81 return _f 82 83 84 class _BeamArgumentParser(argparse.ArgumentParser): 85 """An ArgumentParser that supports ValueProvider options. 86 87 Example Usage:: 88 89 class TemplateUserOptions(PipelineOptions): 90 @classmethod 91 92 def _add_argparse_args(cls, parser): 93 parser.add_value_provider_argument('--vp_arg1', default='start') 94 parser.add_value_provider_argument('--vp_arg2') 95 parser.add_argument('--non_vp_arg') 96 97 """ 98 def add_value_provider_argument(self, *args, **kwargs): 99 """ValueProvider arguments can be either of type keyword or positional. 100 At runtime, even positional arguments will need to be supplied in the 101 key/value form. 102 """ 103 # Extract the option name from positional argument ['pos_arg'] 104 assert args != () and len(args[0]) >= 1 105 if args[0][0] != '-': 106 option_name = args[0] 107 if kwargs.get('nargs') is None: # make them optionally templated 108 kwargs['nargs'] = '?' 109 else: 110 # or keyword arguments like [--kw_arg, -k, -w] or [--kw-arg] 111 option_name = [i.replace('--', '') for i in args if i[:2] == '--'][0] 112 113 # reassign the type to make room for using 114 # StaticValueProvider as the type for add_argument 115 value_type = kwargs.get('type') or str 116 kwargs['type'] = _static_value_provider_of(value_type) 117 118 # reassign default to default_value to make room for using 119 # RuntimeValueProvider as the default for add_argument 120 default_value = kwargs.get('default') 121 kwargs['default'] = RuntimeValueProvider( 122 option_name=option_name, 123 value_type=value_type, 124 default_value=default_value) 125 126 # have add_argument do most of the work 127 self.add_argument(*args, **kwargs) 128 129 # The argparse package by default tries to autocomplete option names. This 130 # results in an "ambiguous option" error from argparse when an unknown option 131 # matching multiple known ones are used. This suppresses that behavior. 132 def error(self, message): 133 if message.startswith('ambiguous option: '): 134 return 135 super().error(message) 136 137 138 class _DictUnionAction(argparse.Action): 139 """ 140 argparse Action take union of json loads values. If a key is specified in more 141 than one of the values, the last value takes precedence. 142 """ 143 def __call__(self, parser, namespace, values, option_string=None): 144 if not hasattr(namespace, 145 self.dest) or getattr(namespace, self.dest) is None: 146 setattr(namespace, self.dest, {}) 147 getattr(namespace, self.dest).update(values) 148 149 150 class PipelineOptions(HasDisplayData): 151 """This class and subclasses are used as containers for command line options. 152 153 These classes are wrappers over the standard argparse Python module 154 (see https://docs.python.org/3/library/argparse.html). To define one option 155 or a group of options, create a subclass from PipelineOptions. 156 157 Example Usage:: 158 159 class XyzOptions(PipelineOptions): 160 161 @classmethod 162 def _add_argparse_args(cls, parser): 163 parser.add_argument('--abc', default='start') 164 parser.add_argument('--xyz', default='end') 165 166 The arguments for the add_argument() method are exactly the ones 167 described in the argparse public documentation. 168 169 Pipeline objects require an options object during initialization. 170 This is obtained simply by initializing an options class as defined above. 171 172 Example Usage:: 173 174 p = Pipeline(options=XyzOptions()) 175 if p.options.xyz == 'end': 176 raise ValueError('Option xyz has an invalid value.') 177 178 Instances of PipelineOptions or any of its subclass have access to values 179 defined by other PipelineOption subclasses (see get_all_options()), and 180 can be converted to an instance of another PipelineOptions subclass 181 (see view_as()). All views share the underlying data structure that stores 182 option key-value pairs. 183 184 By default the options classes will use command line arguments to initialize 185 the options. 186 """ 187 def __init__(self, flags=None, **kwargs): 188 # type: (Optional[List[str]], **Any) -> None 189 190 """Initialize an options class. 191 192 The initializer will traverse all subclasses, add all their argparse 193 arguments and then parse the command line specified by flags or by default 194 the one obtained from sys.argv. 195 196 The subclasses of PipelineOptions do not need to redefine __init__. 197 198 Args: 199 flags: An iterable of command line arguments to be used. If not specified 200 then sys.argv will be used as input for parsing arguments. 201 202 **kwargs: Add overrides for arguments passed in flags. For overrides 203 of arguments, please pass the `option names` instead of 204 flag names. 205 Option names: These are defined as dest in the 206 parser.add_argument() for each flag. Passing flags 207 like {no_use_public_ips: True}, for which the dest is 208 defined to a different flag name in the parser, 209 would be discarded. Instead, pass the dest of 210 the flag (dest of no_use_public_ips is use_public_ips). 211 """ 212 # Initializing logging configuration in case the user did not set it up. 213 logging.basicConfig() 214 215 # self._flags stores a list of not yet parsed arguments, typically, 216 # command-line flags. This list is shared across different views. 217 # See: view_as(). 218 self._flags = flags 219 220 # Build parser that will parse options recognized by the [sub]class of 221 # PipelineOptions whose object is being instantiated. 222 parser = _BeamArgumentParser() 223 for cls in type(self).mro(): 224 if cls == PipelineOptions: 225 break 226 elif '_add_argparse_args' in cls.__dict__: 227 cls._add_argparse_args(parser) # type: ignore 228 229 # The _visible_options attribute will contain options that were recognized 230 # by the parser. 231 self._visible_options, _ = parser.parse_known_args(flags) 232 233 # self._all_options is initialized with overrides to flag values, 234 # provided in kwargs, and will store key-value pairs for options recognized 235 # by current PipelineOptions [sub]class and its views that may be created. 236 # See: view_as(). 237 # This dictionary is shared across different views, and is lazily updated 238 # as each new views are created. 239 # Users access this dictionary store via __getattr__ / __setattr__ methods. 240 self._all_options = kwargs 241 242 # Initialize values of keys defined by this class. 243 for option_name in self._visible_option_list(): 244 # Note that options specified in kwargs will not be overwritten. 245 if option_name not in self._all_options: 246 self._all_options[option_name] = getattr( 247 self._visible_options, option_name) 248 249 @classmethod 250 def _add_argparse_args(cls, parser): 251 # type: (_BeamArgumentParser) -> None 252 # Override this in subclasses to provide options. 253 pass 254 255 @classmethod 256 def from_dictionary(cls, options): 257 """Returns a PipelineOptions from a dictionary of arguments. 258 259 Args: 260 options: Dictionary of argument value pairs. 261 262 Returns: 263 A PipelineOptions object representing the given arguments. 264 """ 265 flags = [] 266 for k, v in options.items(): 267 # Note: If a boolean flag is True in the dictionary, 268 # implicitly the method assumes the boolean flag is 269 # specified as a command line argument. If the 270 # boolean flag is False, this method simply discards them. 271 # Eg: {no_auth: True} is similar to python your_file.py --no_auth 272 # {no_auth: False} is similar to python your_file.py. 273 if isinstance(v, bool): 274 if v: 275 flags.append('--%s' % k) 276 elif k in _FLAG_THAT_SETS_FALSE_VALUE: 277 # Capture overriding flags, which have a different dest 278 # from the flag name defined in the parser.add_argument 279 # Eg: no_use_public_ips, which has the dest=use_public_ips 280 # different from flag name 281 flag_that_disables_the_option = (_FLAG_THAT_SETS_FALSE_VALUE[k]) 282 flags.append('--%s' % flag_that_disables_the_option) 283 elif isinstance(v, list): 284 for i in v: 285 flags.append('--%s=%s' % (k, i)) 286 elif isinstance(v, dict): 287 flags.append('--%s=%s' % (k, json.dumps(v))) 288 elif v is None: 289 # Don't process None type args here, they will be treated 290 # as strings when parsed by BeamArgumentParser.. 291 logging.warning('Not setting flag with value None: %s', k) 292 else: 293 flags.append('--%s=%s' % (k, v)) 294 295 return cls(flags) 296 297 def get_all_options( 298 self, 299 drop_default=False, 300 add_extra_args_fn=None, # type: Optional[Callable[[_BeamArgumentParser], None]] 301 retain_unknown_options=False 302 ): 303 # type: (...) -> Dict[str, Any] 304 305 """Returns a dictionary of all defined arguments. 306 307 Returns a dictionary of all defined arguments (arguments that are defined in 308 any subclass of PipelineOptions) into a dictionary. 309 310 Args: 311 drop_default: If set to true, options that are equal to their default 312 values, are not returned as part of the result dictionary. 313 add_extra_args_fn: Callback to populate additional arguments, can be used 314 by runner to supply otherwise unknown args. 315 retain_unknown_options: If set to true, options not recognized by any 316 known pipeline options class will still be included in the result. If 317 set to false, they will be discarded. 318 319 Returns: 320 Dictionary of all args and values. 321 """ 322 323 # TODO(https://github.com/apache/beam/issues/18197): PipelineOption 324 # sub-classes in the main session might be repeated. Pick last unique 325 # instance of each subclass to avoid conflicts. 326 subset = {} 327 parser = _BeamArgumentParser() 328 for cls in PipelineOptions.__subclasses__(): 329 subset[str(cls)] = cls 330 for cls in subset.values(): 331 cls._add_argparse_args(parser) # pylint: disable=protected-access 332 if add_extra_args_fn: 333 add_extra_args_fn(parser) 334 335 known_args, unknown_args = parser.parse_known_args(self._flags) 336 if retain_unknown_options: 337 i = 0 338 while i < len(unknown_args): 339 # Treat all unary flags as booleans, and all binary argument values as 340 # strings. 341 if not unknown_args[i].startswith('-'): 342 i += 1 343 continue 344 if i + 1 >= len(unknown_args) or unknown_args[i + 1].startswith('-'): 345 split = unknown_args[i].split('=', 1) 346 if len(split) == 1: 347 parser.add_argument(unknown_args[i], action='store_true') 348 else: 349 parser.add_argument(split[0], type=str) 350 i += 1 351 elif unknown_args[i].startswith('--'): 352 parser.add_argument(unknown_args[i], type=str) 353 i += 2 354 else: 355 # skip all binary flags used with '-' and not '--'. 356 # ex: using -f instead of --f (or --flexrs_goal) will prevent 357 # argument validation before job submission and can be incorrectly 358 # submitted to job. 359 _LOGGER.warning( 360 "Discarding flag %s, single dash flags are not allowed.", 361 unknown_args[i]) 362 i += 2 363 continue 364 parsed_args, _ = parser.parse_known_args(self._flags) 365 else: 366 if unknown_args: 367 _LOGGER.warning("Discarding unparseable args: %s", unknown_args) 368 parsed_args = known_args 369 result = vars(parsed_args) 370 371 overrides = self._all_options.copy() 372 # Apply the overrides if any 373 for k in list(result): 374 overrides.pop(k, None) 375 if k in self._all_options: 376 result[k] = self._all_options[k] 377 if (drop_default and parser.get_default(k) == result[k] and 378 not isinstance(parser.get_default(k), ValueProvider)): 379 del result[k] 380 381 if overrides: 382 if retain_unknown_options: 383 result.update(overrides) 384 else: 385 _LOGGER.warning("Discarding invalid overrides: %s", overrides) 386 387 return result 388 389 def display_data(self): 390 return self.get_all_options(drop_default=True, retain_unknown_options=True) 391 392 def view_as(self, cls): 393 # type: (Type[PipelineOptionsT]) -> PipelineOptionsT 394 395 """Returns a view of current object as provided PipelineOption subclass. 396 397 Example Usage:: 398 399 options = PipelineOptions(['--runner', 'Direct', '--streaming']) 400 standard_options = options.view_as(StandardOptions) 401 if standard_options.streaming: 402 # ... start a streaming job ... 403 404 Note that options objects may have multiple views, and modifications 405 of values in any view-object will apply to current object and other 406 view-objects. 407 408 Args: 409 cls: PipelineOptions class or any of its subclasses. 410 411 Returns: 412 An instance of cls that is initialized using options contained in current 413 object. 414 415 """ 416 view = cls(self._flags) 417 418 for option_name in view._visible_option_list(): 419 # Initialize values of keys defined by a cls. 420 # 421 # Note that we do initialization only once per key to make sure that 422 # values in _all_options dict are not-recreated with each new view. 423 # This is important to make sure that values of multi-options keys are 424 # backed by the same list across multiple views, and that any overrides of 425 # pipeline options already stored in _all_options are preserved. 426 if option_name not in self._all_options: 427 self._all_options[option_name] = getattr( 428 view._visible_options, option_name) 429 # Note that views will still store _all_options of the source object. 430 view._all_options = self._all_options 431 return view 432 433 def _visible_option_list(self): 434 # type: () -> List[str] 435 return sorted( 436 option for option in dir(self._visible_options) if option[0] != '_') 437 438 def __dir__(self): 439 # type: () -> List[str] 440 return sorted( 441 dir(type(self)) + list(self.__dict__) + self._visible_option_list()) 442 443 def __getattr__(self, name): 444 # Special methods which may be accessed before the object is 445 # fully constructed (e.g. in unpickling). 446 if name[:2] == name[-2:] == '__': 447 return object.__getattribute__(self, name) 448 elif name in self._visible_option_list(): 449 return self._all_options[name] 450 else: 451 raise AttributeError( 452 "'%s' object has no attribute '%s'" % (type(self).__name__, name)) 453 454 def __setattr__(self, name, value): 455 if name in ('_flags', '_all_options', '_visible_options'): 456 super().__setattr__(name, value) 457 elif name in self._visible_option_list(): 458 self._all_options[name] = value 459 else: 460 raise AttributeError( 461 "'%s' object has no attribute '%s'" % (type(self).__name__, name)) 462 463 def __str__(self): 464 return '%s(%s)' % ( 465 type(self).__name__, 466 ', '.join( 467 '%s=%s' % (option, getattr(self, option)) 468 for option in self._visible_option_list())) 469 470 471 class StandardOptions(PipelineOptions): 472 473 DEFAULT_RUNNER = 'DirectRunner' 474 475 ALL_KNOWN_RUNNERS = ( 476 'apache_beam.runners.dataflow.dataflow_runner.DataflowRunner', 477 'apache_beam.runners.direct.direct_runner.BundleBasedDirectRunner', 478 'apache_beam.runners.direct.direct_runner.DirectRunner', 479 'apache_beam.runners.direct.direct_runner.SwitchingDirectRunner', 480 'apache_beam.runners.interactive.interactive_runner.InteractiveRunner', 481 'apache_beam.runners.portability.flink_runner.FlinkRunner', 482 'apache_beam.runners.portability.portable_runner.PortableRunner', 483 'apache_beam.runners.portability.spark_runner.SparkRunner', 484 'apache_beam.runners.test.TestDirectRunner', 485 'apache_beam.runners.test.TestDataflowRunner', 486 ) 487 488 KNOWN_RUNNER_NAMES = [path.split('.')[-1] for path in ALL_KNOWN_RUNNERS] 489 490 @classmethod 491 def _add_argparse_args(cls, parser): 492 parser.add_argument( 493 '--runner', 494 help=( 495 'Pipeline runner used to execute the workflow. Valid values are ' 496 'one of %s, or the fully qualified name of a PipelineRunner ' 497 'subclass. If unspecified, defaults to %s.' % 498 (', '.join(cls.KNOWN_RUNNER_NAMES), cls.DEFAULT_RUNNER))) 499 # Whether to enable streaming mode. 500 parser.add_argument( 501 '--streaming', 502 default=False, 503 action='store_true', 504 help='Whether to enable streaming mode.') 505 506 parser.add_argument( 507 '--resource_hint', 508 '--resource_hints', 509 dest='resource_hints', 510 action='append', 511 default=[], 512 help=( 513 'Resource hint to set in the pipeline execution environment.' 514 'Hints specified via this option override hints specified ' 515 'at transform level. Interpretation of hints is defined by ' 516 'Beam runners.')) 517 518 519 class CrossLanguageOptions(PipelineOptions): 520 @classmethod 521 def _add_argparse_args(cls, parser): 522 parser.add_argument( 523 '--beam_services', 524 type=json.loads, 525 default={}, 526 help=( 527 'For convenience, Beam provides the ability to automatically ' 528 'download and start various services (such as expansion services) ' 529 'used at pipeline construction and execution. These services are ' 530 'identified by gradle target. This option provides the ability to ' 531 'use pre-started services or non-default pre-existing artifacts to ' 532 'start the given service. ' 533 'Should be a json mapping of gradle build targets to pre-built ' 534 'artifacts (e.g. jar files) expansion endpoints (e.g. host:port).')) 535 536 537 def additional_option_ptransform_fn(): 538 beam.transforms.ptransform.ptransform_fn_typehints_enabled = True 539 540 541 # Optional type checks that aren't enabled by default. 542 additional_type_checks = { 543 'ptransform_fn': additional_option_ptransform_fn, 544 } # type: Dict[str, Callable[[], None]] 545 546 547 def enable_all_additional_type_checks(): 548 """Same as passing --type_check_additional=all.""" 549 for f in additional_type_checks.values(): 550 f() 551 552 553 class TypeOptions(PipelineOptions): 554 @classmethod 555 def _add_argparse_args(cls, parser): 556 # TODO(laolu): Add a type inferencing option here once implemented. 557 parser.add_argument( 558 '--type_check_strictness', 559 default='DEFAULT_TO_ANY', 560 choices=['ALL_REQUIRED', 'DEFAULT_TO_ANY'], 561 help='The level of exhaustive manual type-hint ' 562 'annotation required') 563 parser.add_argument( 564 '--type_check_additional', 565 default='', 566 help='Comma separated list of additional type checking features to ' 567 'enable. Options: all, ptransform_fn. For details see:' 568 'https://beam.apache.org/documentation/sdks/python-type-safety/') 569 parser.add_argument( 570 '--no_pipeline_type_check', 571 dest='pipeline_type_check', 572 action='store_false', 573 help='Disable type checking at pipeline construction ' 574 'time') 575 parser.add_argument( 576 '--runtime_type_check', 577 default=False, 578 action='store_true', 579 help='Enable type checking at pipeline execution ' 580 'time. NOTE: only supported with the ' 581 'DirectRunner') 582 parser.add_argument( 583 '--performance_runtime_type_check', 584 default=False, 585 action='store_true', 586 help='Enable faster type checking via sampling at pipeline execution ' 587 'time. NOTE: only supported with portable runners ' 588 '(including the DirectRunner)') 589 parser.add_argument( 590 '--allow_non_deterministic_key_coders', 591 default=False, 592 action='store_true', 593 help='Use non-deterministic coders (such as pickling) for key-grouping ' 594 'operations such as GroupByKey. This is unsafe, as runners may group ' 595 'keys based on their encoded bytes, but is available for backwards ' 596 'compatibility. See BEAM-11719.') 597 parser.add_argument( 598 '--allow_unsafe_triggers', 599 default=False, 600 action='store_true', 601 help='Allow the use of unsafe triggers. Unsafe triggers have the ' 602 'potential to cause data loss due to finishing and/or never having ' 603 'their condition met. Some operations, such as GroupByKey, disallow ' 604 'this. This exists for cases where such loss is acceptable and for ' 605 'backwards compatibility. See BEAM-9487.') 606 607 def validate(self, unused_validator): 608 errors = [] 609 if beam.version.__version__ >= '3': 610 errors.append( 611 'Update --type_check_additional default to include all ' 612 'available additional checks at Beam 3.0 release time.') 613 keys = self.type_check_additional.split(',') 614 615 for key in keys: 616 if not key: 617 continue 618 elif key == 'all': 619 enable_all_additional_type_checks() 620 elif key in additional_type_checks: 621 additional_type_checks[key]() 622 else: 623 errors.append('Unrecognized --type_check_additional feature: %s' % key) 624 625 return errors 626 627 628 class DirectOptions(PipelineOptions): 629 """DirectRunner-specific execution options.""" 630 @classmethod 631 def _add_argparse_args(cls, parser): 632 parser.add_argument( 633 '--no_direct_runner_use_stacked_bundle', 634 action='store_false', 635 dest='direct_runner_use_stacked_bundle', 636 help='DirectRunner uses stacked WindowedValues within a Bundle for ' 637 'memory optimization. Set --no_direct_runner_use_stacked_bundle to ' 638 'avoid it.') 639 parser.add_argument( 640 '--direct_runner_bundle_repeat', 641 type=int, 642 default=0, 643 help='replay every bundle this many extra times, for profiling' 644 'and debugging') 645 parser.add_argument( 646 '--direct_num_workers', 647 type=int, 648 default=1, 649 help='number of parallel running workers.') 650 parser.add_argument( 651 '--direct_running_mode', 652 default='in_memory', 653 choices=['in_memory', 'multi_threading', 'multi_processing'], 654 help='Workers running environment.') 655 parser.add_argument( 656 '--direct_embed_docker_python', 657 default=False, 658 action='store_true', 659 dest='direct_embed_docker_python', 660 help='DirectRunner uses the embedded Python environment when ' 661 'the default Python docker environment is specified.') 662 parser.add_argument( 663 '--direct_test_splits', 664 default={}, 665 type=json.loads, 666 help='Split test configuration of the json form ' 667 '{"step_name": {"timings": [...], "fractions": [...]}, ...} ' 668 'where step_name is the name of a step controlling the stage to which ' 669 'splits will be sent, timings is a list of floating-point times ' 670 '(in seconds) at which the split requests will be sent, and ' 671 'fractions is a corresponding list of floating points to use in the ' 672 'split requests themselves.') 673 674 675 class GoogleCloudOptions(PipelineOptions): 676 """Google Cloud Dataflow service execution options.""" 677 678 BIGQUERY_API_SERVICE = 'bigquery.googleapis.com' 679 COMPUTE_API_SERVICE = 'compute.googleapis.com' 680 STORAGE_API_SERVICE = 'storage.googleapis.com' 681 DATAFLOW_ENDPOINT = 'https://dataflow.googleapis.com' 682 OAUTH_SCOPES = [ 683 'https://www.googleapis.com/auth/bigquery', 684 'https://www.googleapis.com/auth/cloud-platform', 685 'https://www.googleapis.com/auth/devstorage.full_control', 686 'https://www.googleapis.com/auth/userinfo.email', 687 'https://www.googleapis.com/auth/datastore', 688 'https://www.googleapis.com/auth/spanner.admin', 689 'https://www.googleapis.com/auth/spanner.data' 690 ] 691 692 @classmethod 693 def _add_argparse_args(cls, parser): 694 parser.add_argument( 695 '--dataflow_endpoint', 696 default=cls.DATAFLOW_ENDPOINT, 697 help=( 698 'The URL for the Dataflow API. If not set, the default public URL ' 699 'will be used.')) 700 # Remote execution must check that this option is not None. 701 parser.add_argument( 702 '--project', 703 default=None, 704 help='Name of the Cloud project owning the Dataflow ' 705 'job.') 706 # Remote execution must check that this option is not None. 707 parser.add_argument( 708 '--job_name', default=None, help='Name of the Cloud Dataflow job.') 709 # Remote execution must check that this option is not None. 710 parser.add_argument( 711 '--staging_location', 712 default=None, 713 help='GCS path for staging code packages needed by ' 714 'workers.') 715 # Remote execution must check that this option is not None. 716 # If staging_location is not set, it defaults to temp_location. 717 parser.add_argument( 718 '--temp_location', 719 default=None, 720 help='GCS path for saving temporary workflow jobs.') 721 # The Google Compute Engine region for creating Dataflow jobs. See 722 # https://cloud.google.com/compute/docs/regions-zones/regions-zones for a 723 # list of valid options. 724 parser.add_argument( 725 '--region', 726 default=None, 727 help='The Google Compute Engine region for creating ' 728 'Dataflow job.') 729 parser.add_argument( 730 '--service_account_email', 731 default=None, 732 help='Identity to run virtual machines as.') 733 parser.add_argument( 734 '--no_auth', 735 dest='no_auth', 736 action='store_true', 737 default=False, 738 help='Skips authorizing credentials with Google Cloud.') 739 # Option to run templated pipelines 740 parser.add_argument( 741 '--template_location', 742 default=None, 743 help='Save job to specified local or GCS location.') 744 parser.add_argument( 745 '--label', 746 '--labels', 747 dest='labels', 748 action='append', 749 default=None, 750 help='Labels to be applied to this Dataflow job. ' 751 'Labels are key value pairs separated by = ' 752 '(e.g. --label key=value) or ' 753 '(--labels=\'{ "key": "value", "mass": "1_3kg", "count": "3" }\').') 754 parser.add_argument( 755 '--update', 756 default=False, 757 action='store_true', 758 help='Update an existing streaming Cloud Dataflow job. ' 759 'See https://cloud.google.com/dataflow/docs/guides/' 760 'updating-a-pipeline') 761 parser.add_argument( 762 '--transform_name_mapping', 763 default=None, 764 type=json.loads, 765 help='The transform mapping that maps the named ' 766 'transforms in your prior pipeline code to names ' 767 'in your replacement pipeline code.' 768 'See https://cloud.google.com/dataflow/docs/guides/' 769 'updating-a-pipeline') 770 parser.add_argument( 771 '--enable_streaming_engine', 772 default=False, 773 action='store_true', 774 help='Enable Windmill Service for this Dataflow job. ') 775 parser.add_argument( 776 '--dataflow_kms_key', 777 default=None, 778 help='Set a Google Cloud KMS key name to be used in ' 779 'Dataflow state operations (GBK, Streaming).') 780 parser.add_argument( 781 '--create_from_snapshot', 782 default=None, 783 help='The snapshot from which the job should be created.') 784 parser.add_argument( 785 '--flexrs_goal', 786 default=None, 787 choices=['COST_OPTIMIZED', 'SPEED_OPTIMIZED'], 788 help='Set the Flexible Resource Scheduling mode') 789 parser.add_argument( 790 '--dataflow_service_option', 791 '--dataflow_service_options', 792 dest='dataflow_service_options', 793 action='append', 794 default=None, 795 help=( 796 'Options to configure the Dataflow service. These ' 797 'options decouple service side feature availability ' 798 'from the Apache Beam release cycle.' 799 'Note: If set programmatically, must be set as a ' 800 'list of strings')) 801 parser.add_argument( 802 '--enable_hot_key_logging', 803 default=False, 804 action='store_true', 805 help='When true, will enable the direct logging of any detected hot ' 806 'keys into Cloud Logging. Warning: this will log the literal key as an ' 807 'unobfuscated string.') 808 parser.add_argument( 809 '--enable_artifact_caching', 810 default=False, 811 action='store_true', 812 help='When true, artifacts will be cached across job submissions in ' 813 'the GCS staging bucket') 814 parser.add_argument( 815 '--impersonate_service_account', 816 default=None, 817 help='All API requests will be made as the given service account or ' 818 'target service account in an impersonation delegation chain ' 819 'instead of the currently selected account. You can specify ' 820 'either a single service account as the impersonator, or a ' 821 'comma-separated list of service accounts to create an ' 822 'impersonation delegation chain.') 823 parser.add_argument( 824 '--gcp_oauth_scope', 825 '--gcp_oauth_scopes', 826 dest='gcp_oauth_scopes', 827 action='append', 828 default=cls.OAUTH_SCOPES, 829 help=( 830 'Controls the OAuth scopes that will be requested when creating ' 831 'GCP credentials. Note: If set programmatically, must be set as a ' 832 'list of strings')) 833 834 def _create_default_gcs_bucket(self): 835 try: 836 from apache_beam.io.gcp import gcsio 837 except ImportError: 838 _LOGGER.warning('Unable to create default GCS bucket.') 839 return None 840 bucket = gcsio.get_or_create_default_gcs_bucket(self) 841 if bucket: 842 return 'gs://%s' % bucket.id 843 else: 844 return None 845 846 def validate(self, validator): 847 errors = [] 848 if validator.is_service_runner(): 849 errors.extend(validator.validate_cloud_options(self)) 850 851 # Validating temp_location, or adding a default if there are issues 852 temp_location_errors = validator.validate_gcs_path(self, 'temp_location') 853 if temp_location_errors: 854 default_bucket = self._create_default_gcs_bucket() 855 if default_bucket is None: 856 errors.extend(temp_location_errors) 857 else: 858 setattr(self, 'temp_location', default_bucket) 859 860 if getattr(self, 'staging_location', 861 None) or getattr(self, 'temp_location', None) is None: 862 errors.extend(validator.validate_gcs_path(self, 'staging_location')) 863 864 if self.view_as(DebugOptions).dataflow_job_file: 865 if self.view_as(GoogleCloudOptions).template_location: 866 errors.append( 867 '--dataflow_job_file and --template_location ' 868 'are mutually exclusive.') 869 870 # Validate that dataflow_service_options is a list 871 if self.dataflow_service_options: 872 errors.extend( 873 validator.validate_repeatable_argument_passed_as_list( 874 self, 'dataflow_service_options')) 875 876 return errors 877 878 def get_cloud_profiler_service_name(self): 879 _ENABLE_GOOGLE_CLOUD_PROFILER = 'enable_google_cloud_profiler' 880 if self.dataflow_service_options: 881 if _ENABLE_GOOGLE_CLOUD_PROFILER in self.dataflow_service_options: 882 return os.environ["JOB_NAME"] 883 for option_name in self.dataflow_service_options: 884 if option_name.startswith(_ENABLE_GOOGLE_CLOUD_PROFILER + '='): 885 return option_name.split('=', 1)[1] 886 887 experiments = self.view_as(DebugOptions).experiments or [] 888 if _ENABLE_GOOGLE_CLOUD_PROFILER in experiments: 889 return os.environ["JOB_NAME"] 890 891 return None 892 893 894 class AzureOptions(PipelineOptions): 895 """Azure Blob Storage options.""" 896 @classmethod 897 def _add_argparse_args(cls, parser): 898 parser.add_argument( 899 '--azure_connection_string', 900 default=None, 901 help='Connection string of the Azure Blob Storage Account.') 902 parser.add_argument( 903 '--blob_service_endpoint', 904 default=None, 905 help='URL of the Azure Blob Storage Account.') 906 parser.add_argument( 907 '--azure_managed_identity_client_id', 908 default=None, 909 help='Client ID of a user-assigned managed identity.') 910 911 def validate(self, validator): 912 errors = [] 913 if self.azure_connection_string: 914 if self.blob_service_endpoint: 915 errors.append( 916 '--azure_connection_string and ' 917 '--blob_service_endpoint are mutually exclusive.') 918 919 return errors 920 921 922 class HadoopFileSystemOptions(PipelineOptions): 923 """``HadoopFileSystem`` connection options.""" 924 @classmethod 925 def _add_argparse_args(cls, parser): 926 parser.add_argument( 927 '--hdfs_host', 928 default=None, 929 help=('Hostname or address of the HDFS namenode.')) 930 parser.add_argument( 931 '--hdfs_port', default=None, help=('Port of the HDFS namenode.')) 932 parser.add_argument( 933 '--hdfs_user', default=None, help=('HDFS username to use.')) 934 parser.add_argument( 935 '--hdfs_full_urls', 936 default=False, 937 action='store_true', 938 help=( 939 'If set, URLs will be parsed as "hdfs://server/path/...", instead ' 940 'of "hdfs://path/...". The "server" part will be unused (use ' 941 '--hdfs_host and --hdfs_port).')) 942 943 def validate(self, validator): 944 errors = [] 945 errors.extend(validator.validate_optional_argument_positive(self, 'port')) 946 return errors 947 948 949 # Command line options controlling the worker pool configuration. 950 # TODO(silviuc): Update description when autoscaling options are in. 951 class WorkerOptions(PipelineOptions): 952 """Worker pool configuration options.""" 953 @classmethod 954 def _add_argparse_args(cls, parser): 955 parser.add_argument( 956 '--num_workers', 957 type=int, 958 default=None, 959 help=( 960 'Number of workers to use when executing the Dataflow job. If not ' 961 'set, the Dataflow service will use a reasonable default.')) 962 parser.add_argument( 963 '--max_num_workers', 964 type=int, 965 default=None, 966 help=( 967 'Maximum number of workers to use when executing the Dataflow job.' 968 )) 969 parser.add_argument( 970 '--autoscaling_algorithm', 971 type=str, 972 choices=['NONE', 'THROUGHPUT_BASED'], 973 default=None, # Meaning unset, distinct from 'NONE' meaning don't scale 974 help= 975 ('If and how to autoscale the workerpool.')) 976 parser.add_argument( 977 '--worker_machine_type', 978 '--machine_type', 979 dest='machine_type', 980 default=None, 981 help=( 982 'Machine type to create Dataflow worker VMs as. See ' 983 'https://cloud.google.com/compute/docs/machine-types ' 984 'for a list of valid options. If not set, ' 985 'the Dataflow service will choose a reasonable ' 986 'default.')) 987 parser.add_argument( 988 '--disk_size_gb', 989 type=int, 990 default=None, 991 help=( 992 'Remote worker disk size, in gigabytes, or 0 to use the default ' 993 'size. If not set, the Dataflow service will use a reasonable ' 994 'default.')) 995 parser.add_argument( 996 '--worker_disk_type', 997 '--disk_type', 998 dest='disk_type', 999 default=None, 1000 help=('Specifies what type of persistent disk should be used.')) 1001 parser.add_argument( 1002 '--worker_region', 1003 default=None, 1004 help=( 1005 'The Compute Engine region (https://cloud.google.com/compute/docs/' 1006 'regions-zones/regions-zones) in which worker processing should ' 1007 'occur, e.g. "us-west1". Mutually exclusive with worker_zone. If ' 1008 'neither worker_region nor worker_zone is specified, default to ' 1009 'same value as --region.')) 1010 parser.add_argument( 1011 '--worker_zone', 1012 default=None, 1013 help=( 1014 'The Compute Engine zone (https://cloud.google.com/compute/docs/' 1015 'regions-zones/regions-zones) in which worker processing should ' 1016 'occur, e.g. "us-west1-a". Mutually exclusive with worker_region. ' 1017 'If neither worker_region nor worker_zone is specified, the ' 1018 'Dataflow service will choose a zone in --region based on ' 1019 'available capacity.')) 1020 parser.add_argument( 1021 '--zone', 1022 default=None, 1023 help=( 1024 'GCE availability zone for launching workers. Default is up to the ' 1025 'Dataflow service. This flag is deprecated, and will be replaced ' 1026 'by worker_zone.')) 1027 parser.add_argument( 1028 '--network', 1029 default=None, 1030 help=( 1031 'GCE network for launching workers. Default is up to the Dataflow ' 1032 'service.')) 1033 parser.add_argument( 1034 '--subnetwork', 1035 default=None, 1036 help=( 1037 'GCE subnetwork for launching workers. Default is up to the ' 1038 'Dataflow service. Expected format is ' 1039 'regions/REGION/subnetworks/SUBNETWORK or the fully qualified ' 1040 'subnetwork name. For more information, see ' 1041 'https://cloud.google.com/compute/docs/vpc/')) 1042 parser.add_argument( 1043 '--worker_harness_container_image', 1044 default=None, 1045 help=( 1046 'Docker registry location of container image to use for the ' 1047 'worker harness. If not set, an appropriate approved Google Cloud ' 1048 'Dataflow image will be used based on the version of the ' 1049 'SDK. Note: This flag is deprecated and only supports ' 1050 'approved Google Cloud Dataflow container images. To provide a ' 1051 'custom container image, use sdk_container_image instead.')) 1052 parser.add_argument( 1053 '--sdk_container_image', 1054 default=None, 1055 help=( 1056 'Docker registry location of container image to use for the ' 1057 'worker harness. If not set, an appropriate approved Google Cloud ' 1058 'Dataflow image will be used based on the version of the ' 1059 'SDK. If set for a non-portable pipeline, only official ' 1060 'Google Cloud Dataflow container images may be used here.')) 1061 parser.add_argument( 1062 '--sdk_harness_container_image_overrides', 1063 action='append', 1064 default=None, 1065 help=( 1066 'Overrides for SDK harness container images. Could be for the ' 1067 'local SDK or for a remote SDK that pipeline has to support due ' 1068 'to a cross-language transform. Each entry consist of two values ' 1069 'separated by a comma where first value gives a regex to ' 1070 'identify the container image to override and the second value ' 1071 'gives the replacement container image.')) 1072 parser.add_argument( 1073 '--default_sdk_harness_log_level', 1074 default=None, 1075 help=( 1076 'Controls the default log level of all loggers without a log level ' 1077 'override. Values can be either a labeled level or a number ' 1078 '(See https://docs.python.org/3/library/logging.html#levels). ' 1079 'Default log level is INFO.')) 1080 parser.add_argument( 1081 '--sdk_harness_log_level_overrides', 1082 type=json.loads, 1083 action=_DictUnionAction, 1084 default=None, 1085 help=( 1086 'Controls the log levels for specifically named loggers. The ' 1087 'expected format is a json string: \'{"module":"log_level",...}\'. ' 1088 'For example, by specifying the value \'{"a.b.c":"DEBUG"}\', ' 1089 'the logger underneath the module "a.b.c" will be configured to ' 1090 'output logs at the DEBUG level. Similarly, by specifying the ' 1091 'value \'{"a.b.c":"WARNING"}\' all loggers underneath the "a.b.c" ' 1092 'module will be configured to output logs at the WARNING level. ' 1093 'Also, note that when multiple overrides are specified, the exact ' 1094 'name followed by the closest parent takes precedence.')) 1095 parser.add_argument( 1096 '--use_public_ips', 1097 default=None, 1098 action='store_true', 1099 help='Whether to assign public IP addresses to the worker VMs.') 1100 parser.add_argument( 1101 '--no_use_public_ips', 1102 dest='use_public_ips', 1103 default=None, 1104 action='store_false', 1105 help='Whether to assign only private IP addresses to the worker VMs.') 1106 parser.add_argument( 1107 '--min_cpu_platform', 1108 dest='min_cpu_platform', 1109 type=str, 1110 help='GCE minimum CPU platform. Default is determined by GCP.') 1111 1112 def validate(self, validator): 1113 errors = [] 1114 errors.extend(validator.validate_sdk_container_image_options(self)) 1115 1116 if validator.is_service_runner(): 1117 errors.extend(validator.validate_num_workers(self)) 1118 errors.extend(validator.validate_worker_region_zone(self)) 1119 return errors 1120 1121 1122 class DebugOptions(PipelineOptions): 1123 @classmethod 1124 def _add_argparse_args(cls, parser): 1125 parser.add_argument( 1126 '--dataflow_job_file', 1127 default=None, 1128 help='Debug file to write the workflow specification.') 1129 parser.add_argument( 1130 '--experiment', 1131 '--experiments', 1132 dest='experiments', 1133 action='append', 1134 default=None, 1135 help=( 1136 'Runners may provide a number of experimental features that can be ' 1137 'enabled with this flag. Please sync with the owners of the runner ' 1138 'before enabling any experiments.')) 1139 1140 parser.add_argument( 1141 '--number_of_worker_harness_threads', 1142 type=int, 1143 default=None, 1144 help=( 1145 'Number of threads per worker to use on the runner. If left ' 1146 'unspecified, the runner will compute an appropriate number of ' 1147 'threads to use. Currently only enabled for DataflowRunner when ' 1148 'experiment \'use_runner_v2\' is enabled.')) 1149 1150 def add_experiment(self, experiment): 1151 # pylint: disable=access-member-before-definition 1152 if self.experiments is None: 1153 self.experiments = [] 1154 if experiment not in self.experiments: 1155 self.experiments.append(experiment) 1156 1157 def lookup_experiment(self, key, default=None): 1158 if not self.experiments: 1159 return default 1160 elif key in self.experiments: 1161 return True 1162 for experiment in self.experiments: 1163 if experiment.startswith(key + '='): 1164 return experiment.split('=', 1)[1] 1165 return default 1166 1167 def validate(self, validator): 1168 errors = [] 1169 if self.experiments: 1170 errors.extend( 1171 validator.validate_repeatable_argument_passed_as_list( 1172 self, 'experiments')) 1173 return errors 1174 1175 1176 class ProfilingOptions(PipelineOptions): 1177 @classmethod 1178 def _add_argparse_args(cls, parser): 1179 parser.add_argument( 1180 '--profile_cpu', 1181 action='store_true', 1182 help='Enable work item CPU profiling.') 1183 parser.add_argument( 1184 '--profile_memory', 1185 action='store_true', 1186 help='Enable work item heap profiling.') 1187 parser.add_argument( 1188 '--profile_location', 1189 default=None, 1190 help='path for saving profiler data.') 1191 parser.add_argument( 1192 '--profile_sample_rate', 1193 type=float, 1194 default=1.0, 1195 help='A number between 0 and 1 indicating the ratio ' 1196 'of bundles that should be profiled.') 1197 1198 1199 class SetupOptions(PipelineOptions): 1200 @classmethod 1201 def _add_argparse_args(cls, parser): 1202 # Options for installing dependencies in the worker. 1203 parser.add_argument( 1204 '--requirements_file', 1205 default=None, 1206 help=( 1207 'Path to a requirements file containing package dependencies. ' 1208 'Typically it is produced by a pip freeze command. More details: ' 1209 'https://pip.pypa.io/en/latest/reference/pip_freeze.html. ' 1210 'If used, all the packages specified will be downloaded, ' 1211 'cached (use --requirements_cache to change default location), ' 1212 'and then staged so that they can be automatically installed in ' 1213 'workers during startup. The cache is refreshed as needed ' 1214 'avoiding extra downloads for existing packages. Typically the ' 1215 'file is named requirements.txt.')) 1216 parser.add_argument( 1217 '--requirements_cache', 1218 default=None, 1219 help=( 1220 'Path to a folder to cache the packages specified in ' 1221 'the requirements file using the --requirements_file option.' 1222 'If you want to skip populating requirements cache, please ' 1223 'specify --requirements_cache="skip".')) 1224 parser.add_argument( 1225 '--requirements_cache_only_sources', 1226 action='store_true', 1227 help=( 1228 'Enable this flag to populate requirements cache only ' 1229 'with Source distributions(sdists) of the dependencies ' 1230 'mentioned in the --requirements_file' 1231 'Note: (BEAM-4032): This flag may significantly slow down ' 1232 'the pipeline submission. It is added to preserve the requirements' 1233 ' cache behavior prior to 2.37.0 and will likely be removed in ' 1234 'future releases.')) 1235 parser.add_argument( 1236 '--setup_file', 1237 default=None, 1238 help=( 1239 'Path to a setup Python file containing package dependencies. If ' 1240 'specified, the file\'s containing folder is assumed to have the ' 1241 'structure required for a setuptools setup package. The file must ' 1242 'be named setup.py. More details: ' 1243 'https://pythonhosted.org/an_example_pypi_project/setuptools.html ' 1244 'During job submission a source distribution will be built and ' 1245 'the worker will install the resulting package before running any ' 1246 'custom code.')) 1247 parser.add_argument( 1248 '--beam_plugin', 1249 '--beam_plugins', 1250 dest='beam_plugins', 1251 action='append', 1252 default=None, 1253 help=( 1254 'Bootstrap the python process before executing any code by ' 1255 'importing all the plugins used in the pipeline. Please pass a ' 1256 'comma separated list of import paths to be included. This is ' 1257 'currently an experimental flag and provides no stability. ' 1258 'Multiple --beam_plugin options can be specified if more than ' 1259 'one plugin is needed.')) 1260 parser.add_argument( 1261 '--pickle_library', 1262 default='default', 1263 help=( 1264 'Chooses which pickle library to use. Options are dill, ' 1265 'cloudpickle or default.'), 1266 choices=['cloudpickle', 'default', 'dill']) 1267 parser.add_argument( 1268 '--save_main_session', 1269 default=False, 1270 action='store_true', 1271 help=( 1272 'Save the main session state so that pickled functions and classes ' 1273 'defined in __main__ (e.g. interactive session) can be unpickled. ' 1274 'Some workflows do not need the session state if for instance all ' 1275 'their functions/classes are defined in proper modules ' 1276 '(not __main__) and the modules are importable in the worker. ')) 1277 parser.add_argument( 1278 '--sdk_location', 1279 default='default', 1280 help=( 1281 'Override the default location from where the Beam SDK is ' 1282 'downloaded. It can be a URL, a GCS path, or a local path to an ' 1283 'SDK tarball. Workflow submissions will download or copy an SDK ' 1284 'tarball from here. If set to the string "default", a standard ' 1285 'SDK location is used. If empty, no SDK is copied.')) 1286 parser.add_argument( 1287 '--extra_package', 1288 '--extra_packages', 1289 dest='extra_packages', 1290 action='append', 1291 default=None, 1292 help=( 1293 'Local path to a Python package file. The file is expected to be ' 1294 '(1) a package tarball (".tar"), (2) a compressed package tarball ' 1295 '(".tar.gz"), (3) a Wheel file (".whl") or (4) a compressed ' 1296 'package zip file (".zip") which can be installed using the ' 1297 '"pip install" command of the standard pip package. Multiple ' 1298 '--extra_package options can be specified if more than one ' 1299 'package is needed. During job submission, the files will be ' 1300 'staged in the staging area (--staging_location option) and the ' 1301 'workers will install them in same order they were specified on ' 1302 'the command line.')) 1303 parser.add_argument( 1304 '--prebuild_sdk_container_engine', 1305 help=( 1306 'Prebuild sdk worker container image before job submission. If ' 1307 'enabled, SDK invokes the boot sequence in SDK worker ' 1308 'containers to install all pipeline dependencies in the ' 1309 'container, and uses the prebuilt image in the pipeline ' 1310 'environment. This may speed up pipeline execution. To enable, ' 1311 'select the Docker build engine: local_docker using ' 1312 'locally-installed Docker or cloud_build for using Google Cloud ' 1313 'Build (requires a GCP project with Cloud Build API enabled). You ' 1314 'can also subclass SdkContainerImageBuilder and use that to build ' 1315 'in other environments.')) 1316 parser.add_argument( 1317 '--prebuild_sdk_container_base_image', 1318 default=None, 1319 help=('Deprecated. Use --sdk_container_image instead.')) 1320 parser.add_argument( 1321 '--cloud_build_machine_type', 1322 default=None, 1323 help=( 1324 'If specified, use the machine type explicitly when prebuilding' 1325 'SDK container image on Google Cloud Build.')) 1326 parser.add_argument( 1327 '--docker_registry_push_url', 1328 default=None, 1329 help=( 1330 'Docker registry url to use for tagging and pushing the prebuilt ' 1331 'sdk worker container image.')) 1332 1333 def validate(self, validator): 1334 errors = [] 1335 errors.extend(validator.validate_container_prebuilding_options(self)) 1336 return errors 1337 1338 1339 class PortableOptions(PipelineOptions): 1340 """Portable options are common options expected to be understood by most of 1341 the portable runners. Should generally be kept in sync with 1342 PortablePipelineOptions.java. 1343 """ 1344 @classmethod 1345 def _add_argparse_args(cls, parser): 1346 parser.add_argument( 1347 '--job_endpoint', 1348 default=None, 1349 help=( 1350 'Job service endpoint to use. Should be in the form of host ' 1351 'and port, e.g. localhost:8099.')) 1352 parser.add_argument( 1353 '--artifact_endpoint', 1354 default=None, 1355 help=( 1356 'Artifact staging endpoint to use. Should be in the form of host ' 1357 'and port, e.g. localhost:8098. If none is specified, the ' 1358 'artifact endpoint sent from the job server is used.')) 1359 parser.add_argument( 1360 '--job_server_timeout', 1361 '--job-server-timeout', # For backwards compatibility. 1362 default=60, 1363 type=int, 1364 help=( 1365 'Job service request timeout in seconds. The timeout ' 1366 'determines the max time the driver program will wait to ' 1367 'get a response from the job server. NOTE: the timeout does not ' 1368 'apply to the actual pipeline run time. The driver program can ' 1369 'still wait for job completion indefinitely.')) 1370 parser.add_argument( 1371 '--environment_type', 1372 default=None, 1373 help=( 1374 'Set the default environment type for running ' 1375 'user code. DOCKER (default) runs user code in a container. ' 1376 'PROCESS runs user code in processes that are automatically ' 1377 'started on each worker node. LOOPBACK runs user code on the ' 1378 'same process that originally submitted the job.')) 1379 parser.add_argument( 1380 '--environment_config', 1381 default=None, 1382 help=( 1383 'Set environment configuration for running the user code.\n For ' 1384 'DOCKER: Url for the docker image.\n For PROCESS: json of the ' 1385 'form {"os": "<OS>", "arch": "<ARCHITECTURE>", "command": ' 1386 '"<process to execute>", "env":{"<Environment variables 1>": ' 1387 '"<ENV_VAL>"} }. All fields in the json are optional except ' 1388 'command.\n\nPrefer using --environment_options instead.')) 1389 parser.add_argument( 1390 '--environment_option', 1391 '--environment_options', 1392 dest='environment_options', 1393 action='append', 1394 default=None, 1395 help=( 1396 'Environment configuration for running the user code. ' 1397 'Recognized options depend on --environment_type.\n ' 1398 'For DOCKER: docker_container_image (optional)\n ' 1399 'For PROCESS: process_command (required), process_variables ' 1400 '(optional, comma-separated)\n ' 1401 'For EXTERNAL: external_service_address (required)')) 1402 parser.add_argument( 1403 '--sdk_worker_parallelism', 1404 default=1, 1405 help=( 1406 'Sets the number of sdk worker processes that will run on each ' 1407 'worker node. Default is 1. If 0, a value will be chosen by the ' 1408 'runner.')) 1409 parser.add_argument( 1410 '--environment_cache_millis', 1411 default=0, 1412 help=( 1413 'Duration in milliseconds for environment cache within a job. ' 1414 '0 means no caching.')) 1415 parser.add_argument( 1416 '--output_executable_path', 1417 default=None, 1418 help=( 1419 'Create an executable jar at this path rather than running ' 1420 'the pipeline.')) 1421 1422 def validate(self, validator): 1423 return validator.validate_environment_options(self) 1424 1425 def add_environment_option(self, option): 1426 # pylint: disable=access-member-before-definition 1427 if self.environment_options is None: 1428 self.environment_options = [] 1429 if option not in self.environment_options: 1430 self.environment_options.append(option) 1431 1432 def lookup_environment_option(self, key, default=None): 1433 if not self.environment_options: 1434 return default 1435 elif key in self.environment_options: 1436 return True 1437 for option in self.environment_options: 1438 if option.startswith(key + '='): 1439 return option.split('=', 1)[1] 1440 return default 1441 1442 1443 class JobServerOptions(PipelineOptions): 1444 """Options for starting a Beam job server. Roughly corresponds to 1445 JobServerDriver.ServerConfiguration in Java. 1446 """ 1447 @classmethod 1448 def _add_argparse_args(cls, parser): 1449 parser.add_argument( 1450 '--artifacts_dir', 1451 default=None, 1452 help='The location to store staged artifact files. ' 1453 'Any Beam-supported file system is allowed. ' 1454 'If unset, the local temp dir will be used.') 1455 parser.add_argument( 1456 '--job_port', 1457 default=0, 1458 type=int, 1459 help='Port to use for the job service. 0 to use a ' 1460 'dynamic port.') 1461 parser.add_argument( 1462 '--artifact_port', 1463 default=0, 1464 type=int, 1465 help='Port to use for artifact staging. 0 to use a ' 1466 'dynamic port.') 1467 parser.add_argument( 1468 '--expansion_port', 1469 default=0, 1470 type=int, 1471 help='Port to use for artifact staging. 0 to use a ' 1472 'dynamic port.') 1473 parser.add_argument( 1474 '--job_server_java_launcher', 1475 default='java', 1476 help='The Java Application Launcher executable file to use for ' 1477 'starting a Java job server. If unset, `java` from the ' 1478 'environment\'s $PATH is used.') 1479 parser.add_argument( 1480 '--job_server_jvm_properties', 1481 '--job_server_jvm_property', 1482 dest='job_server_jvm_properties', 1483 action='append', 1484 default=[], 1485 help='JVM properties to pass to a Java job server.') 1486 1487 1488 class FlinkRunnerOptions(PipelineOptions): 1489 1490 # These should stay in sync with gradle.properties. 1491 PUBLISHED_FLINK_VERSIONS = ['1.12', '1.13', '1.14', '1.15', '1.16'] 1492 1493 @classmethod 1494 def _add_argparse_args(cls, parser): 1495 parser.add_argument( 1496 '--flink_master', 1497 default='[auto]', 1498 help='Flink master address (http://host:port)' 1499 ' Use "[local]" to start a local cluster' 1500 ' for the execution. Use "[auto]" if you' 1501 ' plan to either execute locally or let the' 1502 ' Flink job server infer the cluster address.') 1503 parser.add_argument( 1504 '--flink_version', 1505 default=cls.PUBLISHED_FLINK_VERSIONS[-1], 1506 choices=cls.PUBLISHED_FLINK_VERSIONS, 1507 help='Flink version to use.') 1508 parser.add_argument( 1509 '--flink_job_server_jar', help='Path or URL to a flink jobserver jar.') 1510 parser.add_argument( 1511 '--flink_submit_uber_jar', 1512 default=False, 1513 action='store_true', 1514 help='Create and upload an uberjar to the flink master' 1515 ' directly, rather than starting up a job server.' 1516 ' Only applies when flink_master is set to a' 1517 ' cluster address. Requires Python 3.6+.') 1518 parser.add_argument( 1519 '--parallelism', 1520 type=int, 1521 default=-1, 1522 help='The degree of parallelism to be used when distributing' 1523 ' operations onto workers. If the parallelism is not set, the' 1524 ' configured Flink default is used, or 1 if none can be found.') 1525 parser.add_argument( 1526 '--max_parallelism', 1527 type=int, 1528 default=-1, 1529 help='The pipeline wide maximum degree of parallelism to be used. The' 1530 ' maximum parallelism specifies the upper limit for dynamic scaling' 1531 ' and the number of key groups used for partitioned state.') 1532 1533 1534 class SparkRunnerOptions(PipelineOptions): 1535 @classmethod 1536 def _add_argparse_args(cls, parser): 1537 parser.add_argument( 1538 '--spark_master_url', 1539 default='local[4]', 1540 help='Spark master URL (spark://HOST:PORT). ' 1541 'Use "local" (single-threaded) or "local[*]" ' 1542 '(multi-threaded) to start a local cluster for ' 1543 'the execution.') 1544 parser.add_argument( 1545 '--spark_job_server_jar', 1546 help='Path or URL to a Beam Spark job server jar. ' 1547 'Overrides --spark_version.') 1548 parser.add_argument( 1549 '--spark_submit_uber_jar', 1550 default=False, 1551 action='store_true', 1552 help='Create and upload an uber jar to the Spark REST' 1553 ' endpoint, rather than starting up a job server.' 1554 ' Requires Python 3.6+.') 1555 parser.add_argument( 1556 '--spark_rest_url', 1557 help='URL for the Spark REST endpoint. ' 1558 'Only required when using spark_submit_uber_jar. ' 1559 'For example, http://hostname:6066') 1560 parser.add_argument( 1561 '--spark_version', 1562 default='3', 1563 choices=['3'], 1564 help='Spark major version to use.') 1565 1566 1567 class TestOptions(PipelineOptions): 1568 @classmethod 1569 def _add_argparse_args(cls, parser): 1570 # Options for e2e test pipeline. 1571 parser.add_argument( 1572 '--on_success_matcher', 1573 default=None, 1574 help=( 1575 'Verify state/output of e2e test pipeline. This is pickled ' 1576 'version of the matcher which should extends ' 1577 'hamcrest.core.base_matcher.BaseMatcher.')) 1578 parser.add_argument( 1579 '--dry_run', 1580 default=False, 1581 help=( 1582 'Used in unit testing runners without submitting the ' 1583 'actual job.')) 1584 parser.add_argument( 1585 '--wait_until_finish_duration', 1586 default=None, 1587 type=int, 1588 help='The time to wait (in milliseconds) for test pipeline to finish. ' 1589 'If it is set to None, it will wait indefinitely until the job ' 1590 'is finished.') 1591 1592 def validate(self, validator): 1593 errors = [] 1594 if self.view_as(TestOptions).on_success_matcher: 1595 errors.extend(validator.validate_test_matcher(self, 'on_success_matcher')) 1596 return errors 1597 1598 1599 class TestDataflowOptions(PipelineOptions): 1600 @classmethod 1601 def _add_argparse_args(cls, parser): 1602 # This option is passed to Dataflow Runner's Pub/Sub client. The camelCase 1603 # style in 'dest' matches the runner's. 1604 parser.add_argument( 1605 '--pubsub_root_url', 1606 dest='pubsubRootUrl', 1607 default=None, 1608 help='Root URL for use with the Google Cloud Pub/Sub API.', 1609 ) 1610 1611 1612 # TODO(silviuc): Add --files_to_stage option. 1613 # This could potentially replace the --requirements_file and --setup_file. 1614 1615 # TODO(silviuc): Non-standard options. Keep them? If yes, add help too! 1616 # Remote execution must check that this option is not None. 1617 1618 1619 class OptionsContext(object): 1620 """Set default pipeline options for pipelines created in this block. 1621 1622 This is particularly useful for pipelines implicitly created with the 1623 1624 [python list] | PTransform 1625 1626 construct. 1627 1628 Can also be used as a decorator. 1629 """ 1630 overrides = [] # type: List[Dict[str, Any]] 1631 1632 def __init__(self, **options): 1633 self.options = options 1634 1635 def __enter__(self): 1636 self.overrides.append(self.options) 1637 1638 def __exit__(self, *exn_info): 1639 self.overrides.pop() 1640 1641 def __call__(self, f, *args, **kwargs): 1642 def wrapper(*args, **kwargs): 1643 with self: 1644 f(*args, **kwargs) 1645 1646 return wrapper 1647 1648 @classmethod 1649 def augment_options(cls, options): 1650 for override in cls.overrides: 1651 for name, value in override.items(): 1652 setattr(options, name, value) 1653 return options 1654 1655 1656 class S3Options(PipelineOptions): 1657 @classmethod 1658 def _add_argparse_args(cls, parser): 1659 # These options are passed to the S3 IO Client 1660 parser.add_argument( 1661 '--s3_access_key_id', 1662 default=None, 1663 help='The secret key to use when creating the s3 client.') 1664 parser.add_argument( 1665 '--s3_secret_access_key', 1666 default=None, 1667 help='The secret key to use when creating the s3 client.') 1668 parser.add_argument( 1669 '--s3_session_token', 1670 default=None, 1671 help='The session token to use when creating the s3 client.') 1672 parser.add_argument( 1673 '--s3_endpoint_url', 1674 default=None, 1675 help='The complete URL to use for the constructed s3 client.') 1676 parser.add_argument( 1677 '--s3_region_name', 1678 default=None, 1679 help='The name of the region associated with the s3 client.') 1680 parser.add_argument( 1681 '--s3_api_version', 1682 default=None, 1683 help='The API version to use with the s3 client.') 1684 parser.add_argument( 1685 '--s3_verify', 1686 default=None, 1687 help='Whether or not to verify SSL certificates with the s3 client.') 1688 parser.add_argument( 1689 '--s3_disable_ssl', 1690 default=False, 1691 action='store_true', 1692 help=( 1693 'Whether or not to use SSL with the s3 client. ' 1694 'By default, SSL is used.'))