blob: c5f13a7945ee0848e33c839cab922aca265901c3 [file] [log] [blame]
:mod:`airflow.providers.google.cloud.operators.dataflow`
========================================================
.. py:module:: airflow.providers.google.cloud.operators.dataflow
.. autoapi-nested-parse::
This module contains Google Dataflow operators.
Module Contents
---------------
.. py:class:: CheckJobRunning
Bases: :class:`enum.Enum`
Helper enum for choosing what to do if job is already running
IgnoreJob - do not check if running
FinishIfRunning - finish current dag run with no action
WaitForRun - wait for job to finish and then continue with new job
.. attribute:: IgnoreJob
:annotation: = 1
.. attribute:: FinishIfRunning
:annotation: = 2
.. attribute:: WaitForRun
:annotation: = 3
.. py:class:: DataflowCreateJavaJobOperator(*, jar: str, job_name: str = '{{task.task_id}}', dataflow_default_options: Optional[dict] = None, options: Optional[dict] = None, project_id: Optional[str] = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = 'google_cloud_default', delegate_to: Optional[str] = None, poll_sleep: int = 10, job_class: Optional[str] = None, check_if_running: CheckJobRunning = CheckJobRunning.WaitForRun, multiple_jobs: Optional[bool] = None, cancel_timeout: Optional[int] = 10 * 60, wait_until_finished: Optional[bool] = None, **kwargs)
Bases: :class:`airflow.models.BaseOperator`
Start a Java Cloud DataFlow batch job. The parameters of the operation
will be passed to the job.
**Example**: ::
default_args = {
'owner': 'airflow',
'depends_on_past': False,
'start_date':
(2016, 8, 1),
'email': ['alex@vanboxel.be'],
'email_on_failure': False,
'email_on_retry': False,
'retries': 1,
'retry_delay': timedelta(minutes=30),
'dataflow_default_options': {
'project': 'my-gcp-project',
'zone': 'us-central1-f',
'stagingLocation': 'gs://bucket/tmp/dataflow/staging/',
}
}
dag = DAG('test-dag', default_args=default_args)
task = DataFlowJavaOperator(
gcp_conn_id='gcp_default',
task_id='normalize-cal',
jar='{{var.value.gcp_dataflow_base}}pipeline-ingress-cal-normalize-1.0.jar',
options={
'autoscalingAlgorithm': 'BASIC',
'maxNumWorkers': '50',
'start': '{{ds}}',
'partitionType': 'DAY'
},
dag=dag)
.. seealso::
For more detail on job submission have a look at the reference:
https://cloud.google.com/dataflow/pipelines/specifying-exec-params
:param jar: The reference to a self executing DataFlow jar (templated).
:type jar: str
:param job_name: The 'jobName' to use when executing the DataFlow job
(templated). This ends up being set in the pipeline options, so any entry
with key ``'jobName'`` in ``options`` will be overwritten.
:type job_name: str
:param dataflow_default_options: Map of default job options.
:type dataflow_default_options: dict
:param options: Map of job specific 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.
:type options: dict
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
:type project_id: str
:param location: Job location.
:type location: str
:param gcp_conn_id: The connection ID to use connecting to Google Cloud.
:type gcp_conn_id: str
:param delegate_to: The account to impersonate using domain-wide delegation of authority,
if any. For this to work, the service account making the request must have
domain-wide delegation enabled.
:type delegate_to: str
:param poll_sleep: The time in seconds to sleep between polling Google
Cloud Platform for the dataflow job status while the job is in the
JOB_STATE_RUNNING state.
:type poll_sleep: int
:param job_class: The name of the dataflow job class to be executed, it
is often not the main class configured in the dataflow jar file.
:type job_class: str
:param multiple_jobs: If pipeline creates multiple jobs then monitor all jobs
:type multiple_jobs: boolean
:param check_if_running: before running job, validate that a previous run is not in process
:type check_if_running: CheckJobRunning(IgnoreJob = do not check if running, FinishIfRunning=
if job is running finish with nothing, WaitForRun= wait until job finished and the run job)
``jar``, ``options``, and ``job_name`` are templated so you can use variables in them.
:param cancel_timeout: How long (in seconds) operator should wait for the pipeline to be
successfully cancelled when task is being killed.
:type cancel_timeout: Optional[int]
:param wait_until_finished: (Optional)
If True, wait for the end of pipeline execution before exiting.
If False, only submits job.
If None, default behavior.
The default behavior depends on the type of pipeline:
* for the streaming pipeline, wait for jobs to start,
* for the batch pipeline, wait for the jobs to complete.
.. warning::
You cannot call ``PipelineResult.wait_until_finish`` method in your pipeline code for the operator
to work properly. i. e. you must use asynchronous execution. Otherwise, your pipeline will
always wait until finished. For more information, look at:
`Asynchronous execution
<https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#python_10>`__
The process of starting the Dataflow job in Airflow consists of two steps:
* running a subprocess and reading the stderr/stderr log for the job id.
* loop waiting for the end of the job ID from the previous step.
This loop checks the status of the job.
Step two is started just after step one has finished, so if you have wait_until_finished in your
pipeline code, step two will not start until the process stops. When this process stops,
steps two will run, but it will only execute one iteration as the job will be in a terminal state.
If you in your pipeline do not call the wait_for_pipeline method but pass wait_until_finish=True
to the operator, the second loop will wait for the job's terminal state.
If you in your pipeline do not call the wait_for_pipeline method, and pass wait_until_finish=False
to the operator, the second loop will check once is job not in terminal state and exit the loop.
:type wait_until_finished: Optional[bool]
Note that both
``dataflow_default_options`` and ``options`` will be merged to specify pipeline
execution parameter, and ``dataflow_default_options`` is expected to save
high-level options, for instances, project and zone information, which
apply to all dataflow operators in the DAG.
It's a good practice to define dataflow_* parameters in the default_args of the dag
like the project, zone and staging location.
.. code-block:: python
default_args = {
'dataflow_default_options': {
'zone': 'europe-west1-d',
'stagingLocation': 'gs://my-staging-bucket/staging/'
}
}
You need to pass the path to your dataflow 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 ``options`` to pass on options to your job.
.. code-block:: python
t1 = DataFlowJavaOperator(
task_id='dataflow_example',
jar='{{var.value.gcp_dataflow_base}}pipeline/build/libs/pipeline-example-1.0.jar',
options={
'autoscalingAlgorithm': 'BASIC',
'maxNumWorkers': '50',
'start': '{{ds}}',
'partitionType': 'DAY',
'labels': {'foo' : 'bar'}
},
gcp_conn_id='airflow-conn-id',
dag=my-dag)
.. attribute:: template_fields
:annotation: = ['options', 'jar', 'job_name']
.. attribute:: ui_color
:annotation: = #0273d4
.. method:: execute(self, context)
.. method:: on_kill(self)
.. py:class:: DataflowTemplatedJobStartOperator(*, template: str, job_name: str = '{{task.task_id}}', options: Optional[Dict[str, Any]] = None, dataflow_default_options: Optional[Dict[str, Any]] = None, parameters: Optional[Dict[str, str]] = None, project_id: Optional[str] = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = 'google_cloud_default', delegate_to: Optional[str] = None, poll_sleep: int = 10, impersonation_chain: Optional[Union[str, Sequence[str]]] = None, environment: Optional[Dict] = None, cancel_timeout: Optional[int] = 10 * 60, wait_until_finished: Optional[bool] = None, **kwargs)
Bases: :class:`airflow.models.BaseOperator`
Start a Templated Cloud DataFlow job. The parameters of the operation
will be passed to the job.
:param template: The reference to the DataFlow template.
:type template: str
:param job_name: The 'jobName' to use when executing the DataFlow template
(templated).
:param options: Map of job runtime environment options.
It will update environment argument if passed.
.. seealso::
For more information on possible configurations, look at the API documentation
`https://cloud.google.com/dataflow/pipelines/specifying-exec-params
<https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment>`__
:type options: dict
:param dataflow_default_options: Map of default job environment options.
:type dataflow_default_options: dict
:param parameters: Map of job specific parameters for the template.
:type parameters: dict
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
:type project_id: str
:param location: Job location.
:type location: str
:param gcp_conn_id: The connection ID to use connecting to Google Cloud.
:type gcp_conn_id: str
:param delegate_to: The account to impersonate using domain-wide delegation of authority,
if any. For this to work, the service account making the request must have
domain-wide delegation enabled.
:type delegate_to: str
:param poll_sleep: The time in seconds to sleep between polling Google
Cloud Platform for the dataflow job status while the job is in the
JOB_STATE_RUNNING state.
:type poll_sleep: int
:param impersonation_chain: Optional service account to impersonate using short-term
credentials, or chained list of accounts required to get the access_token
of the last account in the list, which will be impersonated in the request.
If set as a string, the account must grant the originating account
the Service Account Token Creator IAM role.
If set as a sequence, the identities from the list must grant
Service Account Token Creator IAM role to the directly preceding identity, with first
account from the list granting this role to the originating account (templated).
:type impersonation_chain: Union[str, Sequence[str]]
:type environment: Optional, Map of job runtime environment options.
.. seealso::
For more information on possible configurations, look at the API documentation
`https://cloud.google.com/dataflow/pipelines/specifying-exec-params
<https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment>`__
:type environment: Optional[dict]
:param cancel_timeout: How long (in seconds) operator should wait for the pipeline to be
successfully cancelled when task is being killed.
:type cancel_timeout: Optional[int]
:param wait_until_finished: (Optional)
If True, wait for the end of pipeline execution before exiting.
If False, only submits job.
If None, default behavior.
The default behavior depends on the type of pipeline:
* for the streaming pipeline, wait for jobs to start,
* for the batch pipeline, wait for the jobs to complete.
.. warning::
You cannot call ``PipelineResult.wait_until_finish`` method in your pipeline code for the operator
to work properly. i. e. you must use asynchronous execution. Otherwise, your pipeline will
always wait until finished. For more information, look at:
`Asynchronous execution
<https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#python_10>`__
The process of starting the Dataflow job in Airflow consists of two steps:
* running a subprocess and reading the stderr/stderr log for the job id.
* loop waiting for the end of the job ID from the previous step.
This loop checks the status of the job.
Step two is started just after step one has finished, so if you have wait_until_finished in your
pipeline code, step two will not start until the process stops. When this process stops,
steps two will run, but it will only execute one iteration as the job will be in a terminal state.
If you in your pipeline do not call the wait_for_pipeline method but pass wait_until_finish=True
to the operator, the second loop will wait for the job's terminal state.
If you in your pipeline do not call the wait_for_pipeline method, and pass wait_until_finish=False
to the operator, the second loop will check once is job not in terminal state and exit the loop.
:type wait_until_finished: Optional[bool]
It's a good practice to define dataflow_* parameters in the default_args of the dag
like the project, zone and staging location.
.. seealso::
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/LaunchTemplateParameters
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment
.. code-block:: python
default_args = {
'dataflow_default_options': {
'zone': 'europe-west1-d',
'tempLocation': 'gs://my-staging-bucket/staging/',
}
}
}
You need to pass the path to your dataflow template as a file reference with the
``template`` parameter. Use ``parameters`` to pass on parameters to your job.
Use ``environment`` to pass on runtime environment variables to your job.
.. code-block:: python
t1 = DataflowTemplateOperator(
task_id='dataflow_example',
template='{{var.value.gcp_dataflow_base}}',
parameters={
'inputFile': "gs://bucket/input/my_input.txt",
'outputFile': "gs://bucket/output/my_output.txt"
},
gcp_conn_id='airflow-conn-id',
dag=my-dag)
``template``, ``dataflow_default_options``, ``parameters``, and ``job_name`` are
templated so you can use variables in them.
Note that ``dataflow_default_options`` is expected to save high-level options
for project information, which apply to all dataflow operators in the DAG.
.. seealso::
https://cloud.google.com/dataflow/docs/reference/rest/v1b3
/LaunchTemplateParameters
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment
For more detail on job template execution have a look at the reference:
https://cloud.google.com/dataflow/docs/templates/executing-templates
.. attribute:: template_fields
:annotation: = ['template', 'job_name', 'options', 'parameters', 'project_id', 'location', 'gcp_conn_id', 'impersonation_chain', 'environment']
.. attribute:: ui_color
:annotation: = #0273d4
.. method:: execute(self, context)
.. method:: on_kill(self)
.. py:class:: DataflowStartFlexTemplateOperator(body: Dict, location: str, project_id: Optional[str] = None, gcp_conn_id: str = 'google_cloud_default', delegate_to: Optional[str] = None, drain_pipeline: bool = False, cancel_timeout: Optional[int] = 10 * 60, wait_until_finished: Optional[bool] = None, *args, **kwargs)
Bases: :class:`airflow.models.BaseOperator`
Starts flex templates with the Dataflow pipeline.
:param body: The request body. See:
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/projects.locations.flexTemplates/launch#request-body
:param location: The location of the Dataflow job (for example europe-west1)
:type location: str
:param project_id: The ID of the GCP project that owns the job.
If set to ``None`` or missing, the default project_id from the GCP connection is used.
:type project_id: Optional[str]
:param gcp_conn_id: The connection ID to use connecting to Google Cloud
Platform.
:type gcp_conn_id: str
:param delegate_to: The account to impersonate, if any.
For this to work, the service account making the request must have
domain-wide delegation enabled.
:type delegate_to: str
:param drain_pipeline: Optional, set to True if want to stop streaming job by draining it
instead of canceling during during killing task instance. See:
https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline
:type drain_pipeline: bool
:param cancel_timeout: How long (in seconds) operator should wait for the pipeline to be
successfully cancelled when task is being killed.
:type cancel_timeout: Optional[int]
:param wait_until_finished: (Optional)
If True, wait for the end of pipeline execution before exiting.
If False, only submits job.
If None, default behavior.
The default behavior depends on the type of pipeline:
* for the streaming pipeline, wait for jobs to start,
* for the batch pipeline, wait for the jobs to complete.
.. warning::
You cannot call ``PipelineResult.wait_until_finish`` method in your pipeline code for the operator
to work properly. i. e. you must use asynchronous execution. Otherwise, your pipeline will
always wait until finished. For more information, look at:
`Asynchronous execution
<https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#python_10>`__
The process of starting the Dataflow job in Airflow consists of two steps:
* running a subprocess and reading the stderr/stderr log for the job id.
* loop waiting for the end of the job ID from the previous step.
This loop checks the status of the job.
Step two is started just after step one has finished, so if you have wait_until_finished in your
pipeline code, step two will not start until the process stops. When this process stops,
steps two will run, but it will only execute one iteration as the job will be in a terminal state.
If you in your pipeline do not call the wait_for_pipeline method but pass wait_until_finish=True
to the operator, the second loop will wait for the job's terminal state.
If you in your pipeline do not call the wait_for_pipeline method, and pass wait_until_finish=False
to the operator, the second loop will check once is job not in terminal state and exit the loop.
:type wait_until_finished: Optional[bool]
.. attribute:: template_fields
:annotation: = ['body', 'location', 'project_id', 'gcp_conn_id']
.. method:: execute(self, context)
.. method:: on_kill(self)
.. py:class:: DataflowStartSqlJobOperator(job_name: str, query: str, options: Dict[str, Any], location: str = DEFAULT_DATAFLOW_LOCATION, project_id: Optional[str] = None, gcp_conn_id: str = 'google_cloud_default', delegate_to: Optional[str] = None, drain_pipeline: bool = False, *args, **kwargs)
Bases: :class:`airflow.models.BaseOperator`
Starts Dataflow SQL query.
:param job_name: The unique name to assign to the Cloud Dataflow job.
:type job_name: str
:param query: The SQL query to execute.
:type query: str
:param options: Job parameters to be executed. It can be a dictionary with the following keys.
For more information, look at:
`https://cloud.google.com/sdk/gcloud/reference/beta/dataflow/sql/query
<gcloud beta dataflow sql query>`__
command reference
:param options: dict
:param location: The location of the Dataflow job (for example europe-west1)
:type location: str
:param project_id: The ID of the GCP project that owns the job.
If set to ``None`` or missing, the default project_id from the GCP connection is used.
:type project_id: Optional[str]
:param gcp_conn_id: The connection ID to use connecting to Google Cloud
Platform.
:type gcp_conn_id: str
:param delegate_to: The account to impersonate, if any.
For this to work, the service account making the request must have
domain-wide delegation enabled.
:type delegate_to: str
:param drain_pipeline: Optional, set to True if want to stop streaming job by draining it
instead of canceling during during killing task instance. See:
https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline
:type drain_pipeline: bool
.. attribute:: template_fields
:annotation: = ['job_name', 'query', 'options', 'location', 'project_id', 'gcp_conn_id']
.. method:: execute(self, context)
.. method:: on_kill(self)
.. py:class:: DataflowCreatePythonJobOperator(*, py_file: str, job_name: str = '{{task.task_id}}', dataflow_default_options: Optional[dict] = None, options: Optional[dict] = None, py_interpreter: str = 'python3', py_options: Optional[List[str]] = None, py_requirements: Optional[List[str]] = None, py_system_site_packages: bool = False, project_id: Optional[str] = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = 'google_cloud_default', delegate_to: Optional[str] = None, poll_sleep: int = 10, drain_pipeline: bool = False, cancel_timeout: Optional[int] = 10 * 60, wait_until_finished: Optional[bool] = None, **kwargs)
Bases: :class:`airflow.models.BaseOperator`
Launching Cloud Dataflow jobs written in python. Note that both
dataflow_default_options and options will be merged to specify pipeline
execution parameter, and dataflow_default_options is expected to save
high-level options, for instances, project and zone information, which
apply to all dataflow operators in the DAG.
.. seealso::
For more detail on job submission have a look at the reference:
https://cloud.google.com/dataflow/pipelines/specifying-exec-params
:param py_file: Reference to the python dataflow pipeline file.py, e.g.,
/some/local/file/path/to/your/python/pipeline/file. (templated)
:type py_file: str
:param job_name: The 'job_name' to use when executing the DataFlow job
(templated). This ends up being set in the pipeline options, so any entry
with key ``'jobName'`` or ``'job_name'`` in ``options`` will be overwritten.
:type job_name: str
:param py_options: Additional python options, e.g., ["-m", "-v"].
:type py_options: list[str]
:param dataflow_default_options: Map of default job options.
:type dataflow_default_options: dict
:param options: Map of job specific 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.
:type options: dict
: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
:type py_interpreter: str
: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.
:type py_requirements: List[str]
: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 gcp_conn_id: The connection ID to use connecting to Google Cloud.
:type gcp_conn_id: str
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
:type project_id: str
:param location: Job location.
:type location: str
:param delegate_to: The account to impersonate using domain-wide delegation of authority,
if any. For this to work, the service account making the request must have
domain-wide delegation enabled.
:type delegate_to: str
:param poll_sleep: The time in seconds to sleep between polling Google
Cloud Platform for the dataflow job status while the job is in the
JOB_STATE_RUNNING state.
:type poll_sleep: int
:param drain_pipeline: Optional, set to True if want to stop streaming job by draining it
instead of canceling during during killing task instance. See:
https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline
:type drain_pipeline: bool
:param cancel_timeout: How long (in seconds) operator should wait for the pipeline to be
successfully cancelled when task is being killed.
:type cancel_timeout: Optional[int]
:param wait_until_finished: (Optional)
If True, wait for the end of pipeline execution before exiting.
If False, only submits job.
If None, default behavior.
The default behavior depends on the type of pipeline:
* for the streaming pipeline, wait for jobs to start,
* for the batch pipeline, wait for the jobs to complete.
.. warning::
You cannot call ``PipelineResult.wait_until_finish`` method in your pipeline code for the operator
to work properly. i. e. you must use asynchronous execution. Otherwise, your pipeline will
always wait until finished. For more information, look at:
`Asynchronous execution
<https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#python_10>`__
The process of starting the Dataflow job in Airflow consists of two steps:
* running a subprocess and reading the stderr/stderr log for the job id.
* loop waiting for the end of the job ID from the previous step.
This loop checks the status of the job.
Step two is started just after step one has finished, so if you have wait_until_finished in your
pipeline code, step two will not start until the process stops. When this process stops,
steps two will run, but it will only execute one iteration as the job will be in a terminal state.
If you in your pipeline do not call the wait_for_pipeline method but pass wait_until_finish=True
to the operator, the second loop will wait for the job's terminal state.
If you in your pipeline do not call the wait_for_pipeline method, and pass wait_until_finish=False
to the operator, the second loop will check once is job not in terminal state and exit the loop.
:type wait_until_finished: Optional[bool]
.. attribute:: template_fields
:annotation: = ['options', 'dataflow_default_options', 'job_name', 'py_file']
.. method:: execute(self, context)
Execute the python dataflow job.
.. method:: on_kill(self)