diff --git a/airflow/providers/google/cloud/operators/dataflow.py b/airflow/providers/google/cloud/operators/dataflow.py index 81ee281af06db..93816e7f2b5ef 100644 --- a/airflow/providers/google/cloud/operators/dataflow.py +++ b/airflow/providers/google/cloud/operators/dataflow.py @@ -53,7 +53,7 @@ class DataflowConfiguration: :py:class:`~airflow.providers.apache.beam.operators.beam.BeamRunJavaPipelineOperator` and :py:class:`~airflow.providers.apache.beam.operators.beam.BeamRunPythonPipelineOperator`. - :param job_name: The 'jobName' to use when executing the DataFlow job + :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'`` or ``'job_name'``in ``options`` will be overwritten. :type job_name: str @@ -173,7 +173,7 @@ def __init__( # pylint: disable=too-many-instance-attributes class DataflowCreateJavaJobOperator(BaseOperator): """ - Start a Java Cloud DataFlow batch job. The parameters of the operation + Start a Java Cloud Dataflow batch job. The parameters of the operation will be passed to the job. This class is deprecated. @@ -200,7 +200,7 @@ class DataflowCreateJavaJobOperator(BaseOperator): dag = DAG('test-dag', default_args=default_args) - task = DataFlowJavaOperator( + task = DataflowCreateJavaJobOperator( gcp_conn_id='gcp_default', task_id='normalize-cal', jar='{{var.value.gcp_dataflow_base}}pipeline-ingress-cal-normalize-1.0.jar', @@ -221,9 +221,9 @@ class DataflowCreateJavaJobOperator(BaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:DataflowCreateJavaJobOperator` - :param jar: The reference to a self executing DataFlow jar (templated). + :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 + :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 @@ -330,7 +330,7 @@ class DataflowCreateJavaJobOperator(BaseOperator): .. code-block:: python - t1 = DataFlowJavaOperator( + t1 = DataflowCreateJavaJobOperator( task_id='dataflow_example', jar='{{var.value.gcp_dataflow_base}}pipeline/build/libs/pipeline-example-1.0.jar', options={ @@ -481,16 +481,16 @@ def on_kill(self) -> None: # pylint: disable=too-many-instance-attributes class DataflowTemplatedJobStartOperator(BaseOperator): """ - Start a Templated Cloud DataFlow job. The parameters of the operation + Start a Templated Cloud Dataflow job. The parameters of the operation will be passed to the job. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:DataflowTemplatedJobStartOperator` - :param template: The reference to the DataFlow template. + :param template: The reference to the Dataflow template. :type template: str - :param job_name: The 'jobName' to use when executing the DataFlow template + :param job_name: The 'jobName' to use when executing the Dataflow template (templated). :type job_name: Optional[str] :param options: Map of job runtime environment options. @@ -598,7 +598,7 @@ class DataflowTemplatedJobStartOperator(BaseOperator): .. code-block:: python - t1 = DataflowTemplateOperator( + t1 = DataflowTemplatedJobStartOperator( task_id='dataflow_example', template='{{var.value.gcp_dataflow_base}}', parameters={ @@ -954,7 +954,7 @@ class DataflowCreatePythonJobOperator(BaseOperator): :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 + :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