Source code for airflow.providers.apache.beam.operators.beam
## Licensed to the Apache Software Foundation (ASF) under one# or more contributor license agreements. See the NOTICE file# distributed with this work for additional information# regarding copyright ownership. The ASF licenses this file# to you under the Apache License, Version 2.0 (the# "License"); you may not use this file except in compliance# with the License. You may obtain a copy of the License at## http://www.apache.org/licenses/LICENSE-2.0## Unless required by applicable law or agreed to in writing,# software distributed under the License is distributed on an# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY# KIND, either express or implied. See the License for the# specific language governing permissions and limitations# under the License."""This module contains Apache Beam operators."""from__future__importannotationsimportasyncioimportcontextlibimportcopyimportosimportstatimporttempfilefromabcimportABC,ABCMeta,abstractmethodfromconcurrent.futuresimportThreadPoolExecutor,as_completedfromcontextlibimportExitStackfromfunctoolsimportpartialfromtypingimportIO,TYPE_CHECKING,Any,Callable,Sequencefromairflow.configurationimportconffromairflow.exceptionsimportAirflowExceptionfromairflow.modelsimportBaseOperatorfromairflow.providers.apache.beam.hooks.beamimportBeamHook,BeamRunnerTypefromairflow.providers.apache.beam.triggers.beamimportBeamJavaPipelineTrigger,BeamPythonPipelineTriggerfromairflow.providers.google.cloud.hooks.dataflowimport(DataflowHook,process_line_and_extract_dataflow_job_id_callback,)fromairflow.providers.google.cloud.hooks.gcsimportGCSHook,_parse_gcs_urlfromairflow.providers.google.cloud.links.dataflowimportDataflowJobLinkfromairflow.providers.google.cloud.operators.dataflowimportCheckJobRunning,DataflowConfigurationfromairflow.utils.helpersimportconvert_camel_to_snake,exactly_onefromairflow.versionimportversionifTYPE_CHECKING:fromairflow.utils.contextimportContext
[docs]classBeamDataflowMixin(metaclass=ABCMeta):""" Helper class to store common, Dataflow specific logic for both. :class:`~airflow.providers.apache.beam.operators.beam.BeamRunPythonPipelineOperator`, :class:`~airflow.providers.apache.beam.operators.beam.BeamRunJavaPipelineOperator` and :class:`~airflow.providers.apache.beam.operators.beam.BeamRunGoPipelineOperator`. """
[docs]classBeamBasePipelineOperator(BaseOperator,BeamDataflowMixin,ABC):""" Abstract base class for Beam Pipeline Operators. :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used. Other possible options: DataflowRunner, SparkRunner, FlinkRunner, PortableRunner. See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType` See: https://beam.apache.org/documentation/runners/capability-matrix/ :param default_pipeline_options: Map of default pipeline options. :param pipeline_options: Map of pipeline options.The key must be a dictionary. The value can contain different types: * If the value is None, the single option - ``--key`` (without value) will be added. * If the value is False, this option will be skipped * If the value is True, the single option - ``--key`` (without value) will be added. * If the value is list, the many options will be added for each key. If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key=B`` options will be left * Other value types will be replaced with the Python textual representation. When defining labels (labels option), you can also provide a dictionary. :param gcp_conn_id: Optional. The connection ID to use connecting to Google Cloud Storage if python file is on GCS. :param dataflow_config: Dataflow's configuration, used when runner type is set to DataflowRunner, (optional) defaults to None. """def__init__(self,*,runner:str="DirectRunner",default_pipeline_options:dict|None=None,pipeline_options:dict|None=None,gcp_conn_id:str="google_cloud_default",dataflow_config:DataflowConfiguration|dict|None=None,**kwargs,)->None:super().__init__(**kwargs)self.runner=runnerself.default_pipeline_options=default_pipeline_optionsor{}self.pipeline_options=pipeline_optionsor{}# ``dataflow_config`` type will resolve into the execute methodself.dataflow_config=dataflow_configor{}# type: ignore[assignment]self.gcp_conn_id=gcp_conn_idself.beam_hook:BeamHookself.dataflow_hook:DataflowHook|None=Noneself._dataflow_job_id:str|None=Noneself._execute_context:Context|None=None@property
@dataflow_job_id.setterdefdataflow_job_id(self,new_value):ifall([new_value,notself._dataflow_job_id,self._execute_context]):# push job_id as soon as it's ready, to let Sensors work before the job finished# and job_id pushed as returned value item.self.xcom_push(context=self._execute_context,key="dataflow_job_id",value=new_value)self._dataflow_job_id=new_valuedef_cast_dataflow_config(self):ifisinstance(self.dataflow_config,dict):self.dataflow_config=DataflowConfiguration(**self.dataflow_config)else:self.dataflow_config=self.dataflow_configorDataflowConfiguration()ifnotself.dataflow_config.job_name:self.dataflow_config.job_name=self.task_idifself.dataflow_configandself.runner.lower()!=BeamRunnerType.DataflowRunner.lower():self.log.warning("dataflow_config is defined but runner is different than DataflowRunner (%s)",self.runner)def_init_pipeline_options(self,format_pipeline_options:bool=False,job_name_variable_key:str|None=None,)->tuple[bool,str|None,dict,Callable[[str],None]|None,Callable[[],bool|None]|None]:self.beam_hook=BeamHook(runner=self.runner)pipeline_options=self.default_pipeline_options.copy()process_line_callback:Callable[[str],None]|None=Nonecheck_job_status_callback:Callable[[],bool|None]|None=Noneis_dataflow=self.runner.lower()==BeamRunnerType.DataflowRunner.lower()dataflow_job_name:str|None=Noneifis_dataflow:(dataflow_job_name,pipeline_options,process_line_callback,check_job_status_callback,)=self._set_dataflow(pipeline_options=pipeline_options,job_name_variable_key=job_name_variable_key,)self.log.info(pipeline_options)pipeline_options.update(self.pipeline_options)ifformat_pipeline_options:snake_case_pipeline_options={convert_camel_to_snake(key):pipeline_options[key]forkeyinpipeline_options}return(is_dataflow,dataflow_job_name,snake_case_pipeline_options,process_line_callback,check_job_status_callback,)return(is_dataflow,dataflow_job_name,pipeline_options,process_line_callback,check_job_status_callback,)
[docs]defexecute_complete(self,context:Context,event:dict[str,Any]):""" Execute when the trigger fires - returns immediately. Relies on trigger to throw an exception, otherwise it assumes execution was successful. """ifevent["status"]=="error":raiseAirflowException(event["message"])self.log.info("%s completed with response %s ",self.task_id,event["message"],)return{"dataflow_job_id":self.dataflow_job_id}
[docs]classBeamRunPythonPipelineOperator(BeamBasePipelineOperator):""" Launch Apache Beam pipelines written in Python. Note that both ``default_pipeline_options`` and ``pipeline_options`` will be merged to specify pipeline execution parameter, and ``default_pipeline_options`` is expected to save high-level options, for instances, project and zone information, which apply to all beam operators in the DAG. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:BeamRunPythonPipelineOperator` .. seealso:: For more detail on Apache Beam have a look at the reference: https://beam.apache.org/documentation/ :param py_file: Reference to the python Apache Beam pipeline file.py, e.g., /some/local/file/path/to/your/python/pipeline/file. (templated) :param py_options: Additional python options, e.g., ["-m", "-v"]. :param py_interpreter: Python version of the beam pipeline. If None, this defaults to the python3. To track python versions supported by beam and related issues check: https://issues.apache.org/jira/browse/BEAM-1251 :param py_requirements: Additional python package(s) to install. If a value is passed to this parameter, a new virtual environment has been created with additional packages installed. You could also install the apache_beam package if it is not installed on your system or you want to use a different version. :param py_system_site_packages: Whether to include system_site_packages in your virtualenv. See virtualenv documentation for more information. This option is only relevant if the ``py_requirements`` parameter is not None. :param deferrable: Run operator in the deferrable mode: checks for the state using asynchronous calls. """
[docs]defexecute(self,context:Context):"""Execute the Apache Beam Python Pipeline."""self._execute_context=contextself._cast_dataflow_config()self.pipeline_options.setdefault("labels",{}).update({"airflow-version":"v"+version.replace(".","-").replace("+","-")})(self.is_dataflow,self.dataflow_job_name,self.snake_case_pipeline_options,self.process_line_callback,self.check_job_status_callback,)=self._init_pipeline_options(format_pipeline_options=True,job_name_variable_key="job_name")ifnotself.beam_hook:raiseAirflowException("Beam hook is not defined.")# Check deferrable parameter passed to the operator# to determine type of run - asynchronous or synchronousifself.deferrable:asyncio.run(self.execute_async(context))else:returnself.execute_sync(context)
[docs]asyncdefexecute_async(self,context:Context):# Creating a new event loop to manage I/O operations asynchronouslyloop=asyncio.get_event_loop()ifself.py_file.lower().startswith("gs://"):gcs_hook=GCSHook(gcp_conn_id=self.gcp_conn_id)# Running synchronous `enter_context()` method in a separate# thread using the default executor `None`. The `run_in_executor()` function returns the# file object, which is created using gcs function `provide_file()`, asynchronously.# This means we can perform asynchronous operations with this file.create_tmp_file_call=gcs_hook.provide_file(object_url=self.py_file)tmp_gcs_file:IO[str]=awaitloop.run_in_executor(None,contextlib.ExitStack().enter_context,# type: ignore[arg-type]create_tmp_file_call,)self.py_file=tmp_gcs_file.nameifself.is_dataflowandself.dataflow_hook:DataflowJobLink.persist(self,context,self.dataflow_config.project_id,self.dataflow_config.location,self.dataflow_job_id,)withself.dataflow_hook.provide_authorized_gcloud():self.defer(trigger=BeamPythonPipelineTrigger(variables=self.snake_case_pipeline_options,py_file=self.py_file,py_options=self.py_options,py_interpreter=self.py_interpreter,py_requirements=self.py_requirements,py_system_site_packages=self.py_system_site_packages,runner=self.runner,),method_name="execute_complete",)else:self.defer(trigger=BeamPythonPipelineTrigger(variables=self.snake_case_pipeline_options,py_file=self.py_file,py_options=self.py_options,py_interpreter=self.py_interpreter,py_requirements=self.py_requirements,py_system_site_packages=self.py_system_site_packages,runner=self.runner,),method_name="execute_complete",)
[docs]defon_kill(self)->None:ifself.dataflow_hookandself.dataflow_job_id:self.log.info("Dataflow job with id: `%s` was requested to be cancelled.",self.dataflow_job_id)self.dataflow_hook.cancel_job(job_id=self.dataflow_job_id,project_id=self.dataflow_config.project_id,location=self.dataflow_config.location,)
[docs]classBeamRunJavaPipelineOperator(BeamBasePipelineOperator):""" Launching Apache Beam pipelines written in Java. Note that both ``default_pipeline_options`` and ``pipeline_options`` will be merged to specify pipeline execution parameter, and ``default_pipeline_options`` is expected to save high-level pipeline_options, for instances, project and zone information, which apply to all Apache Beam operators in the DAG. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:BeamRunJavaPipelineOperator` .. seealso:: For more detail on Apache Beam have a look at the reference: https://beam.apache.org/documentation/ You need to pass the path to your jar file as a file reference with the ``jar`` parameter, the jar needs to be a self executing jar (see documentation here: https://beam.apache.org/documentation/runners/dataflow/#self-executing-jar). Use ``pipeline_options`` to pass on pipeline_options to your job. :param jar: The reference to a self executing Apache Beam jar (templated). :param job_class: The name of the Apache Beam pipeline class to be executed, it is often not the main class configured in the pipeline jar file. """
[docs]defexecute(self,context:Context):"""Execute the Apache Beam Python Pipeline."""self._execute_context=contextself._cast_dataflow_config()(self.is_dataflow,self.dataflow_job_name,self.pipeline_options,self.process_line_callback,_,)=self._init_pipeline_options()ifnotself.beam_hook:raiseAirflowException("Beam hook is not defined.")ifself.deferrable:self.execute_async(context)else:returnself.execute_sync(context)
[docs]defexecute_sync(self,context:Context):"""Execute the Apache Beam Pipeline."""withExitStack()asexit_stack:ifself.jar.lower().startswith("gs://"):gcs_hook=GCSHook(self.gcp_conn_id)tmp_gcs_file=exit_stack.enter_context(gcs_hook.provide_file(object_url=self.jar))self.jar=tmp_gcs_file.nameifself.is_dataflowandself.dataflow_hook:is_running=self.dataflow_config.check_if_running==CheckJobRunning.WaitForRunwhileis_runningandself.dataflow_config.check_if_running==CheckJobRunning.WaitForRun:# The reason for disable=no-value-for-parameter is that project_id parameter is# required but here is not passed, moreover it cannot be passed here.# This method is wrapped by @_fallback_to_project_id_from_variables decorator which# fallback project_id value from variables and raise error if project_id is# defined both in variables and as parameter (here is already defined in variables)is_running=self.dataflow_hook.is_job_dataflow_running(name=self.dataflow_config.job_name,variables=self.pipeline_options,location=self.dataflow_config.location,)ifnotis_running:self.pipeline_options["jobName"]=self.dataflow_job_namewithself.dataflow_hook.provide_authorized_gcloud():self.beam_hook.start_java_pipeline(variables=self.pipeline_options,jar=self.jar,job_class=self.job_class,process_line_callback=self.process_line_callback,)ifself.dataflow_job_nameandself.dataflow_config.location:multiple_jobs=self.dataflow_config.multiple_jobsorFalseDataflowJobLink.persist(self,context,self.dataflow_config.project_id,self.dataflow_config.location,self.dataflow_job_id,)self.dataflow_hook.wait_for_done(job_name=self.dataflow_job_name,location=self.dataflow_config.location,job_id=self.dataflow_job_id,multiple_jobs=multiple_jobs,project_id=self.dataflow_config.project_id,)return{"dataflow_job_id":self.dataflow_job_id}else:self.beam_hook.start_java_pipeline(variables=self.pipeline_options,jar=self.jar,job_class=self.job_class,process_line_callback=self.process_line_callback,)
[docs]defon_kill(self)->None:ifself.dataflow_hookandself.dataflow_job_id:self.log.info("Dataflow job with id: `%s` was requested to be cancelled.",self.dataflow_job_id)self.dataflow_hook.cancel_job(job_id=self.dataflow_job_id,project_id=self.dataflow_config.project_id,location=self.dataflow_config.location,)
[docs]classBeamRunGoPipelineOperator(BeamBasePipelineOperator):""" Launch Apache Beam pipelines written in Go. Note that both ``default_pipeline_options`` and ``pipeline_options`` will be merged to specify pipeline execution parameter, and ``default_pipeline_options`` is expected to save high-level options, for instances, project and zone information, which apply to all beam operators in the DAG. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:BeamRunGoPipelineOperator` .. seealso:: For more detail on Apache Beam have a look at the reference: https://beam.apache.org/documentation/ :param go_file: Reference to the Apache Beam pipeline Go source file, e.g. /local/path/to/main.go or gs://bucket/path/to/main.go. Exactly one of go_file and launcher_binary must be provided. :param launcher_binary: Reference to the Apache Beam pipeline Go binary compiled for the launching platform, e.g. /local/path/to/launcher-main or gs://bucket/path/to/launcher-main. Exactly one of go_file and launcher_binary must be provided. :param worker_binary: Reference to the Apache Beam pipeline Go binary compiled for the worker platform, e.g. /local/path/to/worker-main or gs://bucket/path/to/worker-main. Needed if the OS or architecture of the workers running the pipeline is different from that of the platform launching the pipeline. For more information, see the Apache Beam documentation for Go cross compilation: https://beam.apache.org/documentation/sdks/go-cross-compilation/. If launcher_binary is not set, providing a worker_binary will have no effect. If launcher_binary is set and worker_binary is not, worker_binary will default to the value of launcher_binary. """
[docs]defexecute(self,context:Context):"""Execute the Apache Beam Pipeline."""ifnotexactly_one(self.go_file,self.launcher_binary):raiseValueError("Exactly one of `go_file` and `launcher_binary` must be set")self._execute_context=contextself._cast_dataflow_config()ifself.dataflow_config.impersonation_chain:self.log.warning("Impersonation chain parameter is not supported for Apache Beam GO SDK and will be skipped ""in the execution")self.dataflow_support_impersonation=Falseself.pipeline_options.setdefault("labels",{}).update({"airflow-version":"v"+version.replace(".","-").replace("+","-")})(is_dataflow,dataflow_job_name,snake_case_pipeline_options,process_line_callback,_,)=self._init_pipeline_options(format_pipeline_options=True,job_name_variable_key="job_name")ifnotself.beam_hook:raiseAirflowException("Beam hook is not defined.")go_artifact:_GoArtifact=(_GoFile(file=self.go_file)ifself.go_fileelse_GoBinary(launcher=self.launcher_binary,worker=self.worker_binary))withExitStack()asexit_stack:ifgo_artifact.is_located_on_gcs():gcs_hook=GCSHook(self.gcp_conn_id)tmp_dir=exit_stack.enter_context(tempfile.TemporaryDirectory(prefix="apache-beam-go"))go_artifact.download_from_gcs(gcs_hook=gcs_hook,tmp_dir=tmp_dir)ifis_dataflowandself.dataflow_hook:withself.dataflow_hook.provide_authorized_gcloud():go_artifact.start_pipeline(beam_hook=self.beam_hook,variables=snake_case_pipeline_options,process_line_callback=process_line_callback,)DataflowJobLink.persist(self,context,self.dataflow_config.project_id,self.dataflow_config.location,self.dataflow_job_id,)ifdataflow_job_nameandself.dataflow_config.location:self.dataflow_hook.wait_for_done(job_name=dataflow_job_name,location=self.dataflow_config.location,job_id=self.dataflow_job_id,multiple_jobs=False,project_id=self.dataflow_config.project_id,)return{"dataflow_job_id":self.dataflow_job_id}else:go_artifact.start_pipeline(beam_hook=self.beam_hook,variables=snake_case_pipeline_options,process_line_callback=process_line_callback,)
[docs]defon_kill(self)->None:ifself.dataflow_hookandself.dataflow_job_id:self.log.info("Dataflow job with id: `%s` was requested to be cancelled.",self.dataflow_job_id)self.dataflow_hook.cancel_job(job_id=self.dataflow_job_id,project_id=self.dataflow_config.project_id,location=self.dataflow_config.location,)