github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/options/pipeline_options_validator.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 validator. 19 20 For internal use only; no backwards-compatibility guarantees. 21 """ 22 # pytype: skip-file 23 24 import logging 25 import re 26 27 from apache_beam.internal import pickler 28 from apache_beam.options.pipeline_options import DebugOptions 29 from apache_beam.options.pipeline_options import GoogleCloudOptions 30 from apache_beam.options.pipeline_options import PortableOptions 31 from apache_beam.options.pipeline_options import SetupOptions 32 from apache_beam.options.pipeline_options import StandardOptions 33 from apache_beam.options.pipeline_options import TestOptions 34 from apache_beam.options.pipeline_options import TypeOptions 35 from apache_beam.options.pipeline_options import WorkerOptions 36 37 _LOGGER = logging.getLogger(__name__) 38 39 40 class PipelineOptionsValidator(object): 41 """Validates PipelineOptions. 42 43 Goes through a list of known PipelineOption subclassess and calls:: 44 45 validate(validator) 46 47 if one is implemented. Aggregates a list of validation errors from all and 48 returns an aggregated list. 49 """ 50 51 # Validator will call validate on these subclasses of PipelineOptions 52 OPTIONS = [ 53 DebugOptions, 54 GoogleCloudOptions, 55 PortableOptions, 56 SetupOptions, 57 StandardOptions, 58 TestOptions, 59 TypeOptions, 60 WorkerOptions 61 ] 62 63 # Mutually exclusive options for different types of portable environments. 64 REQUIRED_ENVIRONMENT_OPTIONS = { 65 'DOCKER': [], 66 'PROCESS': ['process_command'], 67 'EXTERNAL': ['external_service_address'], 68 'LOOPBACK': [] 69 } 70 OPTIONAL_ENVIRONMENT_OPTIONS = { 71 'DOCKER': ['docker_container_image'], 72 'PROCESS': ['process_variables'], 73 'EXTERNAL': [], 74 'LOOPBACK': [] 75 } 76 77 # Possible validation errors. 78 ERR_MISSING_OPTION = 'Missing required option: %s.' 79 ERR_MISSING_GCS_PATH = 'Missing GCS path option: %s.' 80 ERR_INVALID_GCS_PATH = 'Invalid GCS path (%s), given for the option: %s.' 81 ERR_INVALID_GCS_BUCKET = ( 82 'Invalid GCS bucket (%s), given for the option: %s. See ' 83 'https://developers.google.com/storage/docs/bucketnaming ' 84 'for more details.') 85 ERR_INVALID_GCS_OBJECT = 'Invalid GCS object (%s), given for the option: %s.' 86 ERR_INVALID_JOB_NAME = ( 87 'Invalid job_name (%s); the name must consist of only the characters ' 88 '[-a-z0-9], starting with a letter and ending with a letter or number') 89 ERR_INVALID_PROJECT_NUMBER = ( 90 'Invalid Project ID (%s). Please make sure you specified the Project ID, ' 91 'not project number.') 92 ERR_INVALID_PROJECT_ID = ( 93 'Invalid Project ID (%s). Please make sure you specified the Project ID, ' 94 'not project description.') 95 ERR_INVALID_NOT_POSITIVE = ( 96 'Invalid value (%s) for option: %s. Value needs ' 97 'to be positive.') 98 ERR_INVALID_TEST_MATCHER_TYPE = ( 99 'Invalid value (%s) for option: %s. Please extend your matcher object ' 100 'from hamcrest.core.base_matcher.BaseMatcher.') 101 ERR_INVALID_TEST_MATCHER_UNPICKLABLE = ( 102 'Invalid value (%s) for option: %s. Please make sure the test matcher ' 103 'is unpicklable.') 104 ERR_INVALID_TRANSFORM_NAME_MAPPING = ( 105 'Invalid transform name mapping format. Please make sure the mapping is ' 106 'string key-value pairs. Invalid pair: (%s:%s)') 107 ERR_INVALID_ENVIRONMENT = ( 108 'Option %s is not compatible with environment type %s.') 109 ERR_ENVIRONMENT_CONFIG = ( 110 'Option environment_config is incompatible with option(s) %s.') 111 ERR_MISSING_REQUIRED_ENVIRONMENT_OPTION = ( 112 'Option %s is required for environment type %s.') 113 ERR_NUM_WORKERS_TOO_HIGH = ( 114 'num_workers (%s) cannot exceed max_num_workers (%s)') 115 ERR_REPEATABLE_OPTIONS_NOT_SET_AS_LIST = ( 116 '(%s) is a string. Programmatically set PipelineOptions like (%s) ' 117 'options need to be specified as a list.') 118 119 # GCS path specific patterns. 120 GCS_URI = '(?P<SCHEME>[^:]+)://(?P<BUCKET>[^/]+)(/(?P<OBJECT>.*))?' 121 GCS_BUCKET = '^[a-z0-9][-_a-z0-9.]+[a-z0-9]$' 122 GCS_SCHEME = 'gs' 123 124 # GoogleCloudOptions specific patterns. 125 JOB_PATTERN = '[a-z]([-a-z0-9]*[a-z0-9])?' 126 PROJECT_ID_PATTERN = '[a-z][-a-z0-9:.]+[a-z0-9]' 127 PROJECT_NUMBER_PATTERN = '[0-9]*' 128 ENDPOINT_PATTERN = r'https://[\S]*googleapis\.com[/]?' 129 130 def __init__(self, options, runner): 131 self.options = options 132 self.runner = runner 133 134 def validate(self): 135 """Calls validate on subclassess and returns a list of errors. 136 137 validate will call validate method on subclasses, accumulate the returned 138 list of errors, and returns the aggregate list. 139 140 Returns: 141 Aggregate list of errors after all calling all possible validate methods. 142 """ 143 errors = [] 144 for cls in self.OPTIONS: 145 if 'validate' in cls.__dict__ and callable(cls.__dict__['validate']): 146 errors.extend(self.options.view_as(cls).validate(self)) 147 return errors 148 149 def is_service_runner(self): 150 """True if pipeline will execute on the Google Cloud Dataflow service.""" 151 is_service_runner = ( 152 self.runner is not None and 153 type(self.runner).__name__ in ['DataflowRunner', 'TestDataflowRunner']) 154 155 dataflow_endpoint = ( 156 self.options.view_as(GoogleCloudOptions).dataflow_endpoint) 157 is_service_endpoint = ( 158 dataflow_endpoint is not None and 159 self.is_full_string_match(self.ENDPOINT_PATTERN, dataflow_endpoint)) 160 161 return is_service_runner and is_service_endpoint 162 163 def is_full_string_match(self, pattern, string): 164 """Returns True if the pattern matches the whole string.""" 165 pattern = '^%s$' % pattern 166 return re.search(pattern, string) is not None 167 168 def _validate_error(self, err, *args): 169 return [err % args] 170 171 def validate_gcs_path(self, view, arg_name): 172 """Validates a GCS path against gs://bucket/object URI format.""" 173 arg = getattr(view, arg_name, None) 174 if arg is None: 175 return self._validate_error(self.ERR_MISSING_GCS_PATH, arg_name) 176 match = re.match(self.GCS_URI, arg, re.DOTALL) 177 if match is None: 178 return self._validate_error(self.ERR_INVALID_GCS_PATH, arg, arg_name) 179 180 scheme = match.group('SCHEME') 181 bucket = match.group('BUCKET') 182 gcs_object = match.group('OBJECT') 183 184 if ((scheme is None) or (scheme.lower() != self.GCS_SCHEME) or 185 (bucket is None)): 186 return self._validate_error(self.ERR_INVALID_GCS_PATH, arg, arg_name) 187 188 if not self.is_full_string_match(self.GCS_BUCKET, bucket): 189 return self._validate_error(self.ERR_INVALID_GCS_BUCKET, arg, arg_name) 190 if gcs_object is None or '\n' in gcs_object or '\r' in gcs_object: 191 return self._validate_error(self.ERR_INVALID_GCS_OBJECT, arg, arg_name) 192 193 return [] 194 195 def validate_cloud_options(self, view): 196 """Validates job_name and project arguments.""" 197 errors = [] 198 if (view.job_name and 199 not self.is_full_string_match(self.JOB_PATTERN, view.job_name)): 200 errors.extend( 201 self._validate_error(self.ERR_INVALID_JOB_NAME, view.job_name)) 202 project = view.project 203 if project is None: 204 errors.extend(self._validate_error(self.ERR_MISSING_OPTION, 'project')) 205 else: 206 if self.is_full_string_match(self.PROJECT_NUMBER_PATTERN, project): 207 errors.extend( 208 self._validate_error(self.ERR_INVALID_PROJECT_NUMBER, project)) 209 elif not self.is_full_string_match(self.PROJECT_ID_PATTERN, project): 210 errors.extend( 211 self._validate_error(self.ERR_INVALID_PROJECT_ID, project)) 212 if view.update: 213 if not view.job_name: 214 errors.extend( 215 self._validate_error( 216 'Existing job name must be provided when updating a pipeline.')) 217 if view.transform_name_mapping: 218 if not view.update or not self.options.view_as(StandardOptions).streaming: 219 errors.append( 220 'Transform name mapping option is only useful when ' 221 '--update and --streaming is specified') 222 for _, (key, value) in enumerate(view.transform_name_mapping.items()): 223 if not isinstance(key, str) or not isinstance(value, str): 224 errors.extend( 225 self._validate_error( 226 self.ERR_INVALID_TRANSFORM_NAME_MAPPING, key, value)) 227 break 228 if view.region is None and self.is_service_runner(): 229 default_region = self.runner.get_default_gcp_region() 230 if default_region is None: 231 errors.extend(self._validate_error(self.ERR_MISSING_OPTION, 'region')) 232 else: 233 view.region = default_region 234 return errors 235 236 def validate_sdk_container_image_options(self, view): 237 errors = [] 238 if view.sdk_container_image and view.worker_harness_container_image: 239 # To be fully backwards-compatible, these options will be set to the same 240 # value. Check that the values are different. 241 if view.sdk_container_image != view.worker_harness_container_image: 242 errors.extend( 243 self._validate_error( 244 'Cannot use legacy flag --worker_harness_container_image along ' 245 'with view.sdk_container_image')) 246 elif view.worker_harness_container_image: 247 # Warn about legacy flag and set new flag to value of old flag. 248 _LOGGER.warning( 249 'Setting sdk_container_image to value of legacy flag ' 250 'worker_harness_container_image.') 251 view.sdk_container_image = view.worker_harness_container_image 252 elif view.sdk_container_image: 253 # Set legacy option to value of new option. 254 view.worker_harness_container_image = view.sdk_container_image 255 256 return errors 257 258 def validate_container_prebuilding_options(self, view): 259 errors = [] 260 custom_image = self.options.view_as(WorkerOptions).sdk_container_image 261 if (view.prebuild_sdk_container_base_image is not None and 262 custom_image != view.prebuild_sdk_container_base_image): 263 errors.extend( 264 self._validate_error( 265 'Don\'t use the deprecated option ' 266 '--prebuild_sdk_container_base_image. Use --sdk_container_image ' 267 'instead.')) 268 return errors 269 270 def validate_num_workers(self, view): 271 """Validates that Dataflow worker number is valid.""" 272 errors = self.validate_optional_argument_positive(view, 'num_workers') 273 errors.extend( 274 self.validate_optional_argument_positive(view, 'max_num_workers')) 275 276 num_workers = view.num_workers 277 max_num_workers = view.max_num_workers 278 if (num_workers is not None and max_num_workers is not None and 279 num_workers > max_num_workers): 280 errors.extend( 281 self._validate_error( 282 self.ERR_NUM_WORKERS_TOO_HIGH, num_workers, max_num_workers)) 283 284 return errors 285 286 def validate_worker_region_zone(self, view): 287 """Validates Dataflow worker region and zone arguments are consistent.""" 288 errors = [] 289 if view.zone and (view.worker_region or view.worker_zone): 290 errors.extend( 291 self._validate_error( 292 'Cannot use deprecated flag --zone along with worker_region or ' 293 'worker_zone.')) 294 if self.options.view_as(DebugOptions).lookup_experiment('worker_region')\ 295 and (view.worker_region or view.worker_zone): 296 errors.extend( 297 self._validate_error( 298 'Cannot use deprecated experiment worker_region along with ' 299 'worker_region or worker_zone.')) 300 if view.worker_region and view.worker_zone: 301 errors.extend( 302 self._validate_error( 303 'worker_region and worker_zone are mutually exclusive.')) 304 if view.zone: 305 _LOGGER.warning( 306 'Option --zone is deprecated. Please use --worker_zone instead.') 307 view.worker_zone = view.zone 308 view.zone = None 309 return errors 310 311 def validate_optional_argument_positive(self, view, arg_name): 312 """Validates that an optional argument (if set) has a positive value.""" 313 arg = getattr(view, arg_name, None) 314 if arg is not None and int(arg) <= 0: 315 return self._validate_error(self.ERR_INVALID_NOT_POSITIVE, arg, arg_name) 316 return [] 317 318 def validate_test_matcher(self, view, arg_name): 319 """Validates that on_success_matcher argument if set. 320 321 Validates that on_success_matcher is unpicklable and is instance 322 of `hamcrest.core.base_matcher.BaseMatcher`. 323 """ 324 # This is a test only method and requires hamcrest 325 from hamcrest.core.base_matcher import BaseMatcher 326 pickled_matcher = view.on_success_matcher 327 errors = [] 328 try: 329 matcher = pickler.loads(pickled_matcher) 330 if not isinstance(matcher, BaseMatcher): 331 errors.extend( 332 self._validate_error( 333 self.ERR_INVALID_TEST_MATCHER_TYPE, matcher, arg_name)) 334 except: # pylint: disable=bare-except 335 errors.extend( 336 self._validate_error( 337 self.ERR_INVALID_TEST_MATCHER_UNPICKLABLE, 338 pickled_matcher, 339 arg_name)) 340 return errors 341 342 def validate_environment_options(self, view): 343 """Validates portable environment options.""" 344 errors = [] 345 actual_environment_type = ( 346 view.environment_type.upper() if view.environment_type else None) 347 for environment_type, required in self.REQUIRED_ENVIRONMENT_OPTIONS.items(): 348 found_required_options = [ 349 opt for opt in required 350 if view.lookup_environment_option(opt) is not None 351 ] 352 found_optional_options = [ 353 opt for opt in self.OPTIONAL_ENVIRONMENT_OPTIONS[environment_type] 354 if view.lookup_environment_option(opt) is not None 355 ] 356 found_options = found_required_options + found_optional_options 357 if environment_type == actual_environment_type: 358 if view.environment_config: 359 if found_options: 360 errors.extend( 361 self._validate_error( 362 self.ERR_ENVIRONMENT_CONFIG, ', '.join(found_options))) 363 else: 364 missing_options = set(required).difference( 365 set(found_required_options)) 366 for opt in missing_options: 367 errors.extend( 368 self._validate_error( 369 self.ERR_MISSING_REQUIRED_ENVIRONMENT_OPTION, 370 opt, 371 environment_type)) 372 else: 373 # Environment options classes are mutually exclusive. 374 for opt in found_options: 375 errors.extend( 376 self._validate_error( 377 self.ERR_INVALID_ENVIRONMENT, opt, actual_environment_type)) 378 if actual_environment_type == 'LOOPBACK' and view.environment_config: 379 errors.extend( 380 self._validate_error( 381 self.ERR_INVALID_ENVIRONMENT, 'environment_config', 'LOOPBACK')) 382 return errors 383 384 def validate_repeatable_argument_passed_as_list(self, view, arg_name): 385 """Validates that repeatable PipelineOptions like dataflow_service_options 386 or experiments are specified as a list when set programmatically. This 387 way, users do not inadvertently specify it as a string, mirroring the way 388 they are set via the command lineRepeatable options, which are as passed a 389 list. 390 """ 391 arg = getattr(view, arg_name, None) 392 if not isinstance(arg, list): 393 return self._validate_error( 394 self.ERR_REPEATABLE_OPTIONS_NOT_SET_AS_LIST, arg, arg_name) 395 return []