You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/12/04 16:36:57 UTC

[GitHub] [airflow] TobKed opened a new pull request #12814: Add Apache Beam operators

TobKed opened a new pull request #12814:
URL: https://github.com/apache/airflow/pull/12814


   Simple [Apache Beam](https://github.com/apache/beam) operators to run Python and Java pipelines with DirectRunner (DirectRunner is used by default but other runners are supported as well).
   
   It will allow to test DAGs with pipelines which uses DirectRunner for faster feedback, then using e.g. Dataflow operators.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563144866



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,305 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.hooks.dataflow import DataflowJobStatus
+from airflow.providers.google.cloud.operators.dataflow import DataflowPythonConfiguration
+from airflow.providers.google.cloud.sensors.dataflow import DataflowJobStatusSensor
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+GCS_PYTHON_DATAFLOW_ASYNC = os.environ.get(
+    'APACHE_BEAM_PYTHON_DATAFLOW_ASYNC', 'gs://test-dataflow-example/wordcount_debugging.py'
+)
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_pipeline_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_pipeline_direct_runner",
+        runner="DirectRunner",

Review comment:
       Yes! Thank you. I will commit this args.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-766002219


   [The Workflow run](https://github.com/apache/airflow/actions/runs/505778507) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565409702



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():

Review comment:
       I made variable `is_dataflow` to store check value.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565410115



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)

Review comment:
       I used `convert_camel_to_snake` here. Thank you!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565226698



##########
File path: airflow/providers/apache/beam/README.md
##########
@@ -0,0 +1,75 @@
+<!--
+ 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.
+ -->
+
+
+# Package apache-airflow-providers-apache-beam
+
+Release: 0.0.1
+
+**Table of contents**
+
+- [Provider package](#provider-package)
+- [Installation](#installation)
+- [PIP requirements](#pip-requirements)
+- [Cross provider package dependencies](#cross-provider-package-dependencies)
+- [Provider class summary](#provider-classes-summary)
+    - [Operators](#operators)
+    - [Transfer operators](#transfer-operators)
+    - [Hooks](#hooks)
+- [Releases](#releases)
+
+
+## Provider package
+
+This is a provider package for `apache.beam` provider. All classes for this provider package
+are in `airflow.providers.apache.beam` python package.
+
+
+## Installation
+
+You can install this package on top of an existing airflow 2.* installation via
+`pip install apache-airflow-providers-apache-beam`
+
+
+## PIP requirements
+
+
+## Cross provider package dependencies
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+
+# Provider classes summary
+
+In Airflow 2.0, all operators, transfers, hooks, sensors, secrets for the `apache.beam` provider
+are in the `airflow.providers.apache.beam` package. You can read more about the naming conventions used
+in [Naming conventions for provider packages](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#naming-conventions-for-provider-packages)
+
+
+## Operators
+

Review comment:
       Should we add info here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-762385570


   [The Workflow run](https://github.com/apache/airflow/actions/runs/494069782) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-770416856


   [The Workflow run](https://github.com/apache/airflow/actions/runs/525614277) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r536431621



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,234 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class _BeamRunner(LoggingMixin):
+    def __init__(
+        self,
+        cmd: List[str],
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code {self._proc.returncode}")
+
+
+class BeamHook(BaseHook):
+    """
+    Hook for Apache Beam.
+
+    All the methods in the hook where project_id is used must be called with
+    keyword arguments rather than positional.
+    """
+
+    def __init__(
+        self,
+        runner: str,
+    ) -> None:
+        self.runner = runner
+        super().__init__()
+
+    def _start_pipeline(
+        self,
+        variables: dict,
+        command_prefix: List[str],
+    ) -> None:
+        cmd = command_prefix + [
+            f"--runner={self.runner}",
+        ]
+        if variables:
+            cmd.extend(self._options_to_args(variables))
+        _BeamRunner(cmd=cmd).wait_for_done()

Review comment:
       Should we allow async mode (op + sensor)? So the running pipeline is not blocking worker slot? I suppose this may be hard as the subprocess may be running in worker other than the sensor is using. However we may consider load balancing task to make sure the sensor will be scheduled on proper worker. See https://github.com/apache/airflow/issues/12519#issuecomment-733118659
   
   Although, this is a bigger change than this PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r536438411



##########
File path: airflow/providers/apache/beam/provider.yaml
##########
@@ -0,0 +1,40 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-apache-beam
+name: Apache Beam
+description: |
+    `Apache Druid <https://druid.apache.org/>`__.
+
+versions:
+  - 0.0.1
+
+integrations:
+  - integration-name: Apache Beam
+    external-doc-url: https://beam.apache.org/
+    tags: [apache]
+
+operators:
+  - integration-name: Apache Druid
+    python-modules:
+      - airflow.providers.apache.beam.operators.beam
+
+hooks:
+  - integration-name: Apache Druid

Review comment:
       ```suggestion
     - integration-name: Apache Beam
   ```

##########
File path: airflow/providers/apache/beam/provider.yaml
##########
@@ -0,0 +1,40 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-apache-beam
+name: Apache Beam
+description: |
+    `Apache Druid <https://druid.apache.org/>`__.
+
+versions:
+  - 0.0.1
+
+integrations:
+  - integration-name: Apache Beam
+    external-doc-url: https://beam.apache.org/
+    tags: [apache]
+
+operators:
+  - integration-name: Apache Druid

Review comment:
       ```suggestion
     - integration-name: Apache Beam
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-762815248


   [The Workflow run](https://github.com/apache/airflow/actions/runs/495834850) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565349691



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,295 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.info(line.rstrip("\n"))

Review comment:
       Sure. I made small refactor here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565356744



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,295 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code {self._proc.returncode}")

Review comment:
       Could you explain why? I didn't see `SystemExit` in other hooks or operators. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-772310082


   thanks for all reviewers! I rebased on the latest master and squashed fixups


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-768271739


   [The Workflow run](https://github.com/apache/airflow/actions/runs/514938589) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567432715



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        with ExitStack() as exit_stack:
+            if self.jar.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.jar)
+                )
+                self.jar = tmp_gcs_file.name
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                is_running = False
+                if self.dataflow_config.check_if_running != CheckJobRunning.IgnoreJob:
+                    is_running = (
+                        self.dataflow_hook.is_job_dataflow_running(  # pylint: disable=no-value-for-parameter

Review comment:
       Good point. I will add comment




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-772430157


   Transient doc error :(
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r537585087



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_direct_runner",
+        py_file='apache_beam.examples.wordcount',
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_direct_runner",
+        py_file=GCS_PYTHON,
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_dataflow_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_dataflow_runner",
+        runner="DataflowRunner",
+        py_file=GCS_PYTHON,
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],

Review comment:
       Thanks. I will update to `2.25.0` for now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed edited a comment on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed edited a comment on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-762284036


   PTAL @aaltay @mik-laj @turbaszek 
   I updated PR with refactor which moves common logic between Dataflow and Beam to Beam.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567659095



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,292 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd not in (self._proc.stdout, self._proc.stderr):
+            raise Exception("No data in stderr or in stdout.")
+
+        func_proc, func_log = (
+            (self._proc.stderr, self.log.warning)
+            if fd == self._proc.stderr
+            else (self._proc.stdout, self.log.info)
+        )
+
+        while True:
+            line = func_proc.readline().decode()
+            if not line:
+                return
+            if self.process_line_callback:
+                self.process_line_callback(line)
+            func_log(line.rstrip("\n"))

Review comment:
       ```suggestion
           fd_to_log = {self._proc.stderr: self.log.warnings, self._proc.stdout, self.log.info}
           func_log = fd_to_log[fd]
   
           while True:
               line = fd.readline().decode()
               if not line:
                   return
               if self.process_line_callback:
                   self.process_line_callback(line)
               func_log(line.rstrip("\n"))
   ```
   
   I think this is easier to read and possibly extend in future




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567436289



##########
File path: airflow/providers/google/cloud/operators/dataflow.py
##########
@@ -43,6 +48,214 @@ class CheckJobRunning(Enum):
     WaitForRun = 3
 
 
+class DataflowConfiguration(metaclass=ABCMeta):
+    """Abstract class for Dataflow configuration to be passed to Beam operators"""
+
+    template_fields = ["job_name", "location"]
+
+    def __init__(
+        self,
+        *,
+        job_name: Optional[str] = "{{task.task_id}}",
+        append_job_name: bool = True,
+        project_id: Optional[str] = None,
+        location: Optional[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,
+        drain_pipeline: bool = False,
+        cancel_timeout: Optional[int] = 5 * 60,
+        wait_until_finished: Optional[bool] = None,
+    ) -> None:
+        self.job_name = job_name
+        self.append_job_name = append_job_name
+        self.project_id = project_id
+        self.location = location
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.poll_sleep = poll_sleep
+        self.impersonation_chain = impersonation_chain
+        self.drain_pipeline = drain_pipeline
+        self.cancel_timeout = cancel_timeout
+        self.wait_until_finished = wait_until_finished
+
+
+class DataflowPythonConfiguration(DataflowConfiguration):
+    """
+    Dataflow configuration that can be passed to
+    :py:class:`~airflow.providers.apache.beam.operators.beam.BeamRunPythonPipelineOperator`
+
+    :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
+    :param append_job_name: True if unique suffix has to be appended to job name.
+    :type append_job_name: bool
+    :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]]
+    :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]
+    """
+
+
+class DataflowJavaConfiguration(DataflowConfiguration):
+    """
+    Dataflow configuration that can be passed to
+    :py:class:`~airflow.providers.apache.beam.operators.beam.BeamRunJavaPipelineOperator`
+
+    :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
+    :param append_job_name: True if unique suffix has to be appended to job name.
+    :type append_job_name: bool
+    :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]]
+    :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]
+    :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.
+    """
+
+    def __init__(
+        self,
+        *,
+        multiple_jobs: Optional[bool] = None,
+        check_if_running: CheckJobRunning = CheckJobRunning.WaitForRun,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.multiple_jobs = multiple_jobs
+        self.check_if_running = check_if_running
+
+
 # pylint: disable=too-many-instance-attributes
 class DataflowCreateJavaJobOperator(BaseOperator):

Review comment:
       I added deprecation warnings in docstrings and logs.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r536440661



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,407 @@
+#
+# 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."""
+import copy
+import re
+from contextlib import ExitStack
+from typing import List, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook
+from airflow.providers.google.cloud.hooks.dataflow import DataflowHook
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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 project_id: (DataflowRunner) 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 gcp_conn_id: (DataflowRunner) Optional.
+        The connection ID to use connecting to Google Cloud.
+    :type gcp_conn_id: str
+    :param job_name: (DataflowRunner) Optional.
+        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 delegate_to: (DataflowRunner) Optional.
+        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
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "job_name"]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        project_id: Optional[str] = None,
+        job_name: str = "{{task.task_id}}",
+        delegate_to: Optional[str] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.job_name = job_name
+        self.project_id = project_id
+        self.job_id = None
+        self.delegate_to = delegate_to
+        self.hook: Optional[DataflowHook] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        pipeline_options = self.default_pipeline_options.copy()
+        pipeline_options.update(self.pipeline_options)
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            if self.runner == "DataflowRunner":

Review comment:
       While I agree that logging a warning or even using a `UserWarning` for DataflowRunner is a good thing, I would say that in case of operator behaviour we should have only one, consistent behaviour. That's what I would at least expect as a user. Mind I'm not beam expert




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565244829



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():

Review comment:
       Wouldn't it be feasible to implement, `BeamRunnerType.isinstance(self.runner, BeamRunner.DataflowRunner)` to avoid duplication? 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567660840



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,446 @@
+#
+# 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 contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.utils.helpers import convert_camel_to_snake
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if python file is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )

Review comment:
       Should we consider encapsulating this logic into some `dataflow_method` to reduce size of `execute` definition. WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r537729966



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,234 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class _BeamRunner(LoggingMixin):
+    def __init__(
+        self,
+        cmd: List[str],
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)

Review comment:
       printing stdout/stderr to logs is handled by `_process_fd` function two lines above :) I used logs produced by it extensively during debugging of example dag and I can confirm they provide helpful information about errors.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-772709932


   [The Workflow run](https://github.com/apache/airflow/actions/runs/534468261) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk merged pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #12814:
URL: https://github.com/apache/airflow/pull/12814


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r537583810



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')

Review comment:
       It is not publicly writable location.  This example DAG is used in system tests (`tests/providers/apache/beam/operators/test_beam_system.py`) which is being run by pytest with flag `--system` e.g. `--system=google` and when system test is executed user should provide GCP connection with proper permissions to storage and Dataflow resources. Environmental variables with resources should be adjusted as well if necessary.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-741865234


   [The Workflow run](https://github.com/apache/airflow/actions/runs/410927702) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563147254



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+        self._dataflow_job_name: Optional[str] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowPythonConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = self._dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=self._dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowJavaConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowJavaConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowJavaConfiguration(**self.dataflow_config)

Review comment:
       I added warning in logs




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567482118



##########
File path: setup.py
##########
@@ -522,6 +522,7 @@ def get_sphinx_theme_version() -> str:
 # Dict of all providers which are part of the Apache Airflow repository together with their requirements
 PROVIDERS_REQUIREMENTS: Dict[str, List[str]] = {
     'amazon': amazon,
+    'apache.beam': [],

Review comment:
       Thank you!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-762890413


   [The Workflow run](https://github.com/apache/airflow/actions/runs/496161257) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565250645



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        with ExitStack() as exit_stack:
+            if self.jar.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.jar)
+                )
+                self.jar = tmp_gcs_file.name
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                is_running = False
+                if self.dataflow_config.check_if_running != CheckJobRunning.IgnoreJob:
+                    is_running = (
+                        self.dataflow_hook.is_job_dataflow_running(  # pylint: disable=no-value-for-parameter
+                            name=self.dataflow_config.job_name,
+                            variables=pipeline_options,
+                        )
+                    )
+                    while is_running and self.dataflow_config.check_if_running == CheckJobRunning.WaitForRun:
+                        # pylint: disable=no-value-for-parameter
+                        is_running = self.dataflow_hook.is_job_dataflow_running(
+                            name=self.dataflow_config.job_name,
+                            variables=pipeline_options,
+                        )
+                if not is_running:
+                    pipeline_options["jobName"] = self._dataflow_job_name
+                    self.beam_hook.start_java_pipeline(
+                        variables=pipeline_options,
+                        jar=self.jar,
+                        job_class=self.job_class,
+                        process_line_callback=process_line_callback,
+                    )
+                    self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                        job_name=self._dataflow_job_name,
+                        location=self.dataflow_config.location,
+                        job_id=self.dataflow_job_id,
+                        multiple_jobs=self.dataflow_config.multiple_jobs,
+                    )
+
+            else:
+                self.beam_hook.start_java_pipeline(
+                    variables=pipeline_options,
+                    jar=self.jar,
+                    job_class=self.job_class,
+                    process_line_callback=process_line_callback,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")

Review comment:
       Should we make this message more meaningful? Also shouldn't it be within the if clause?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567453177



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )

Review comment:
       I agree. I removed examples from docstrings and added how to guide.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-766001606


   [The Workflow run](https://github.com/apache/airflow/actions/runs/505750152) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r594540730



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,289 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook

Review comment:
       Solved in https://github.com/apache/airflow/pull/14554




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r538230313



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,407 @@
+#
+# 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."""
+import copy
+import re
+from contextlib import ExitStack
+from typing import List, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook
+from airflow.providers.google.cloud.hooks.dataflow import DataflowHook
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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 project_id: (DataflowRunner) 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 gcp_conn_id: (DataflowRunner) Optional.
+        The connection ID to use connecting to Google Cloud.
+    :type gcp_conn_id: str
+    :param job_name: (DataflowRunner) Optional.
+        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 delegate_to: (DataflowRunner) Optional.
+        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
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "job_name"]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        project_id: Optional[str] = None,
+        job_name: str = "{{task.task_id}}",
+        delegate_to: Optional[str] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.job_name = job_name
+        self.project_id = project_id
+        self.job_id = None
+        self.delegate_to = delegate_to
+        self.hook: Optional[DataflowHook] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        pipeline_options = self.default_pipeline_options.copy()
+        pipeline_options.update(self.pipeline_options)
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            if self.runner == "DataflowRunner":

Review comment:
       Correct me if I'm wrong, but we use two different hooks depending if it's dataflow or not. So at least at this point the behaviour is not identical (unless the hooks implements exactly the same logic but then - why to have two different hooks?). So I would be in favour of using only `BeamHook` and rising a warning if users use DataflowRunner. WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563718589



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+        self._dataflow_job_name: Optional[str] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowPythonConfiguration(**self.dataflow_config)

Review comment:
       I wanted to distinguish them because `DataflowJavaConfiguration` supports additional parameters (`multiple_jobs`, `check_if_running`) but for sake of simplicity i made only `DataflowConfiguration` and added information in docstring to these two parameters that they are supported only by Java.  Thanks for suggestions. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-768221322


   [The Workflow run](https://github.com/apache/airflow/actions/runs/514885104) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-759419276


   [The Workflow run](https://github.com/apache/airflow/actions/runs/482655296) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-769955614


   [The Workflow run](https://github.com/apache/airflow/actions/runs/521450745) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563719826



##########
File path: airflow/providers/google/cloud/operators/dataflow.py
##########
@@ -43,6 +48,214 @@ class CheckJobRunning(Enum):
     WaitForRun = 3
 
 
+class DataflowConfiguration(metaclass=ABCMeta):
+    """Abstract class for Dataflow configuration to be passed to Beam operators"""
+
+    template_fields = ["job_name", "location"]
+
+    def __init__(
+        self,
+        *,
+        job_name: Optional[str] = "{{task.task_id}}",
+        append_job_name: bool = True,
+        project_id: Optional[str] = None,
+        location: Optional[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,
+        drain_pipeline: bool = False,
+        cancel_timeout: Optional[int] = 5 * 60,
+        wait_until_finished: Optional[bool] = None,
+    ) -> None:
+        self.job_name = job_name
+        self.append_job_name = append_job_name
+        self.project_id = project_id
+        self.location = location
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.poll_sleep = poll_sleep
+        self.impersonation_chain = impersonation_chain
+        self.drain_pipeline = drain_pipeline
+        self.cancel_timeout = cancel_timeout
+        self.wait_until_finished = wait_until_finished
+
+
+class DataflowPythonConfiguration(DataflowConfiguration):

Review comment:
       Sure. As i mentioned [here](https://github.com/apache/airflow/pull/12814#discussion_r563718589) I made only base version.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] aaltay commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
aaltay commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r541458711



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,234 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class _BeamRunner(LoggingMixin):
+    def __init__(
+        self,
+        cmd: List[str],
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)

Review comment:
       Nice!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r568335069



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,446 @@
+#
+# 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 contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.utils.helpers import convert_camel_to_snake
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if python file is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        formatted_pipeline_options = {
+            convert_camel_to_snake(key): pipeline_options[key] for key in pipeline_options
+        }
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if is_dataflow:
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        if self.dataflow_hook and self.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,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )

Review comment:
       I made PoC of refactor. PTAL: https://github.com/PolideaInternal/airflow/pull/976

##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,446 @@
+#
+# 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 contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.utils.helpers import convert_camel_to_snake
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if python file is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        formatted_pipeline_options = {
+            convert_camel_to_snake(key): pipeline_options[key] for key in pipeline_options
+        }
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if is_dataflow:
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        if self.dataflow_hook and self.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,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )

Review comment:
       @turbaszek thanks for suggestion. I made PoC of refactor. PTAL: https://github.com/PolideaInternal/airflow/pull/976




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-769768419


   [The Workflow run](https://github.com/apache/airflow/actions/runs/520697213) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565247697



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        with ExitStack() as exit_stack:
+            if self.jar.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.jar)
+                )
+                self.jar = tmp_gcs_file.name
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                is_running = False
+                if self.dataflow_config.check_if_running != CheckJobRunning.IgnoreJob:
+                    is_running = (
+                        self.dataflow_hook.is_job_dataflow_running(  # pylint: disable=no-value-for-parameter

Review comment:
       Can you explain the pylint comment here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-763808290


   [The Workflow run](https://github.com/apache/airflow/actions/runs/498903306) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-740600403


   [The Workflow run](https://github.com/apache/airflow/actions/runs/408233015) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r568019985



##########
File path: airflow/providers/apache/beam/README.md
##########
@@ -0,0 +1,75 @@
+<!--
+ 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.
+ -->
+
+
+# Package apache-airflow-providers-apache-beam
+
+Release: 0.0.1
+
+**Table of contents**
+
+- [Provider package](#provider-package)
+- [Installation](#installation)
+- [PIP requirements](#pip-requirements)
+- [Cross provider package dependencies](#cross-provider-package-dependencies)
+- [Provider class summary](#provider-classes-summary)
+    - [Operators](#operators)
+    - [Transfer operators](#transfer-operators)
+    - [Hooks](#hooks)
+- [Releases](#releases)
+
+
+## Provider package
+
+This is a provider package for `apache.beam` provider. All classes for this provider package
+are in `airflow.providers.apache.beam` python package.
+
+
+## Installation
+
+You can install this package on top of an existing airflow 2.* installation via
+`pip install apache-airflow-providers-apache-beam`
+
+
+## PIP requirements
+
+
+## Cross provider package dependencies
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+
+# Provider classes summary
+
+In Airflow 2.0, all operators, transfers, hooks, sensors, secrets for the `apache.beam` provider
+are in the `airflow.providers.apache.beam` package. You can read more about the naming conventions used
+in [Naming conventions for provider packages](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#naming-conventions-for-provider-packages)
+
+
+## Operators
+

Review comment:
       Sure. I improved Readme




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r536431621



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,234 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class _BeamRunner(LoggingMixin):
+    def __init__(
+        self,
+        cmd: List[str],
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code {self._proc.returncode}")
+
+
+class BeamHook(BaseHook):
+    """
+    Hook for Apache Beam.
+
+    All the methods in the hook where project_id is used must be called with
+    keyword arguments rather than positional.
+    """
+
+    def __init__(
+        self,
+        runner: str,
+    ) -> None:
+        self.runner = runner
+        super().__init__()
+
+    def _start_pipeline(
+        self,
+        variables: dict,
+        command_prefix: List[str],
+    ) -> None:
+        cmd = command_prefix + [
+            f"--runner={self.runner}",
+        ]
+        if variables:
+            cmd.extend(self._options_to_args(variables))
+        _BeamRunner(cmd=cmd).wait_for_done()

Review comment:
       Should we allow async mode (op + sensor)? So the running pipeline is not blocking worker slot? I suppose this may be hard as we are running a subprocess




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-739001072


   [The Workflow run](https://github.com/apache/airflow/actions/runs/401324946) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r543206492



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_direct_runner",
+        py_file='apache_beam.examples.wordcount',
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_direct_runner",
+        py_file=GCS_PYTHON,
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_dataflow_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_dataflow_runner",
+        runner="DataflowRunner",
+        py_file=GCS_PYTHON,
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],

Review comment:
       Thanks. I update version to the latest one :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565408641



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        with ExitStack() as exit_stack:
+            if self.jar.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.jar)
+                )
+                self.jar = tmp_gcs_file.name
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                is_running = False
+                if self.dataflow_config.check_if_running != CheckJobRunning.IgnoreJob:
+                    is_running = (
+                        self.dataflow_hook.is_job_dataflow_running(  # pylint: disable=no-value-for-parameter
+                            name=self.dataflow_config.job_name,
+                            variables=pipeline_options,
+                        )
+                    )
+                    while is_running and self.dataflow_config.check_if_running == CheckJobRunning.WaitForRun:
+                        # pylint: disable=no-value-for-parameter
+                        is_running = self.dataflow_hook.is_job_dataflow_running(
+                            name=self.dataflow_config.job_name,
+                            variables=pipeline_options,
+                        )
+                if not is_running:
+                    pipeline_options["jobName"] = self._dataflow_job_name
+                    self.beam_hook.start_java_pipeline(
+                        variables=pipeline_options,
+                        jar=self.jar,
+                        job_class=self.job_class,
+                        process_line_callback=process_line_callback,
+                    )
+                    self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                        job_name=self._dataflow_job_name,
+                        location=self.dataflow_config.location,
+                        job_id=self.dataflow_job_id,
+                        multiple_jobs=self.dataflow_config.multiple_jobs,
+                    )
+
+            else:
+                self.beam_hook.start_java_pipeline(
+                    variables=pipeline_options,
+                    jar=self.jar,
+                    job_class=self.job_class,
+                    process_line_callback=process_line_callback,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")

Review comment:
       Yes! I made change to it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563146425



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+        self._dataflow_job_name: Optional[str] = None

Review comment:
       You are right. I made it local. Thank you




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565408851



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}

Review comment:
       Thanks! Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-772475101


   [The Workflow run](https://github.com/apache/airflow/actions/runs/533589113) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563147783



##########
File path: airflow/providers/google/cloud/hooks/dataflow.py
##########
@@ -50,6 +47,35 @@
 T = TypeVar("T", bound=Callable)  # pylint: disable=invalid-name
 
 
+def process_line_and_extract_dataflow_job_id_callback(
+    on_new_job_id_callback: Optional[Callable[[str], None]]
+) -> Callable[[str], None]:
+    """
+    Returns callback which triggers function passed as `on_new_job_id_callback` when Dataflow job_id is found.
+    To be used for `process_line_callback` in
+    :py:class:`~airflow.providers.apache.beam.hooks.beam.BeamCommandRunner`
+
+    :param on_new_job_id_callback: Callback called when the job ID is known
+    :type on_new_job_id_callback: callback
+    """
+
+    def _process_line_and_extract_job_id(
+        line: str,
+        # on_new_job_id_callback: Optional[Callable[[str], None]]
+    ) -> None:
+        # Job id info: https://goo.gl/SE29y9.
+        matched_job = JOB_ID_PATTERN.search(line)
+        if matched_job:
+            job_id = matched_job.group("job_id_java") or matched_job.group("job_id_python")

Review comment:
       It is different matching group because stdout for Java and Python commands shows job id in different way:
   ```
   JOB_ID_PATTERN = re.compile(
       r"Submitted job: (?P<job_id_java>.*)|Created job with id: \[(?P<job_id_python>.*)\]"
   )
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-748020425


   [The Workflow run](https://github.com/apache/airflow/actions/runs/430277698) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-770172347


   [The Workflow run](https://github.com/apache/airflow/actions/runs/522895061) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r536431621



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,234 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class _BeamRunner(LoggingMixin):
+    def __init__(
+        self,
+        cmd: List[str],
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code {self._proc.returncode}")
+
+
+class BeamHook(BaseHook):
+    """
+    Hook for Apache Beam.
+
+    All the methods in the hook where project_id is used must be called with
+    keyword arguments rather than positional.
+    """
+
+    def __init__(
+        self,
+        runner: str,
+    ) -> None:
+        self.runner = runner
+        super().__init__()
+
+    def _start_pipeline(
+        self,
+        variables: dict,
+        command_prefix: List[str],
+    ) -> None:
+        cmd = command_prefix + [
+            f"--runner={self.runner}",
+        ]
+        if variables:
+            cmd.extend(self._options_to_args(variables))
+        _BeamRunner(cmd=cmd).wait_for_done()

Review comment:
       Should we allow async mode (op + sensor)? So the running pipeline is not blocking worker slot? I suppose this may be hard as the subprocess may be running in worker other than the sensor is using. However we may consider load balancing task to make sure the sensor will be scheduled on proper worker. See https://github.com/apache/airflow/issues/12519#issuecomment-733118659 for ideas
   
   Although, this is a bigger change than this PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-763723606


   > Is there a commit of the new changes I can review, or do I need to review the whole PR? I do not recall much from my previous review. What are the new changes?
   
   @aaltay I squashed commits so first one represents old changes, however I think review of all changes is required. The most important files are hooks and operators for Beam and Dataflow:
   
    * `airflow/providers/google/cloud/hooks/dataflow.py`
    * `airflow/providers/google/cloud/operators/dataflow.py`
    * `airflow/providers/apache/beam/operators/beam.py`
    * `airflow/providers/apache/beam/hooks/beam.py`
   
   After change Beam hook handles the logic responsible for **starting** pipelines for all runners.  If runner is Dataflow then Dataflow specific logic is responsible for handling a waiting for the proper status, cancelling etc.
   
   The background for this change is that user may test pipeline with `DirectRunner`, then just change runner parameter to `DataflowRunner` to run it on GCP.  Previously he had to change operator from Beam to Dataflow. 
   
   All changes are backward compatible so there should be no change in use of Dataflow operators and hooks for the users.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565383776



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        with ExitStack() as exit_stack:
+            if self.jar.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.jar)
+                )
+                self.jar = tmp_gcs_file.name
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                is_running = False
+                if self.dataflow_config.check_if_running != CheckJobRunning.IgnoreJob:
+                    is_running = (
+                        self.dataflow_hook.is_job_dataflow_running(  # pylint: disable=no-value-for-parameter

Review comment:
       `project_id` parameter is required for this method and not passed, however it is bit truly required because it is handled by `@_fallback_to_project_id_from_variables` decorator.
   Moreover, in this case `project_id` cannot be passed as parameter because it is already present as key-value in `variables`.  The `@_fallback_to_project_id_from_variables`decorator ensures that project_id cannot be passed both in `variables` and as the `project_id` parameter (raises error if both present).
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] aaltay commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
aaltay commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-763235575


   > PTAL @aaltay @mik-laj @turbaszek
   > I updated PR with refactor which moves common logic between Dataflow and Beam to Beam.
   
   Is there a commit of the new changes I can review, or do I need to review the whole PR? I do not recall much from my previous review. What are the new changes?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] aaltay commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
aaltay commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r541458550



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_direct_runner",
+        py_file='apache_beam.examples.wordcount',
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_direct_runner",
+        py_file=GCS_PYTHON,
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_dataflow_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_dataflow_runner",
+        runner="DataflowRunner",
+        py_file=GCS_PYTHON,
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],

Review comment:
       Cool. (Btw 2.26 is available as of today.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r536440661



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,407 @@
+#
+# 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."""
+import copy
+import re
+from contextlib import ExitStack
+from typing import List, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook
+from airflow.providers.google.cloud.hooks.dataflow import DataflowHook
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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 project_id: (DataflowRunner) 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 gcp_conn_id: (DataflowRunner) Optional.
+        The connection ID to use connecting to Google Cloud.
+    :type gcp_conn_id: str
+    :param job_name: (DataflowRunner) Optional.
+        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 delegate_to: (DataflowRunner) Optional.
+        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
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "job_name"]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        project_id: Optional[str] = None,
+        job_name: str = "{{task.task_id}}",
+        delegate_to: Optional[str] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.job_name = job_name
+        self.project_id = project_id
+        self.job_id = None
+        self.delegate_to = delegate_to
+        self.hook: Optional[DataflowHook] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        pipeline_options = self.default_pipeline_options.copy()
+        pipeline_options.update(self.pipeline_options)
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            if self.runner == "DataflowRunner":

Review comment:
       While I agree that logging a warning or even using a `UserWarning` for DataflowRunner is a good thing, I would say that in case of operator behaviour we should have only one, consistent behaviour. That'w what I would at least expect as a user. Mind I'm not beam expert




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-748020138


   [The Workflow run](https://github.com/apache/airflow/actions/runs/430274645) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563668817



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}

Review comment:
       I don't think it is possible. WDYT @potiuk ?

##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+        self._dataflow_job_name: Optional[str] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowPythonConfiguration(**self.dataflow_config)

Review comment:
       I wanted to distinguish them because `DataflowJavaConfiguration` supports additional parameters (`multiple_jobs`, `check_if_running`) but for sake of simplicity i made only `DataflowConfiguration` and added information in docstring to these two parameters that they are supported only by Java.  Thanks for suggestions. 

##########
File path: airflow/providers/google/cloud/operators/dataflow.py
##########
@@ -43,6 +48,214 @@ class CheckJobRunning(Enum):
     WaitForRun = 3
 
 
+class DataflowConfiguration(metaclass=ABCMeta):
+    """Abstract class for Dataflow configuration to be passed to Beam operators"""
+
+    template_fields = ["job_name", "location"]
+
+    def __init__(
+        self,
+        *,
+        job_name: Optional[str] = "{{task.task_id}}",
+        append_job_name: bool = True,
+        project_id: Optional[str] = None,
+        location: Optional[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,
+        drain_pipeline: bool = False,
+        cancel_timeout: Optional[int] = 5 * 60,
+        wait_until_finished: Optional[bool] = None,
+    ) -> None:
+        self.job_name = job_name
+        self.append_job_name = append_job_name
+        self.project_id = project_id
+        self.location = location
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.poll_sleep = poll_sleep
+        self.impersonation_chain = impersonation_chain
+        self.drain_pipeline = drain_pipeline
+        self.cancel_timeout = cancel_timeout
+        self.wait_until_finished = wait_until_finished
+
+
+class DataflowPythonConfiguration(DataflowConfiguration):

Review comment:
       Sure. As i mentioned [here](https://github.com/apache/airflow/pull/12814#discussion_r563718589) I made only base version.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r537742145



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,407 @@
+#
+# 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."""
+import copy
+import re
+from contextlib import ExitStack
+from typing import List, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook
+from airflow.providers.google.cloud.hooks.dataflow import DataflowHook
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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 project_id: (DataflowRunner) 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 gcp_conn_id: (DataflowRunner) Optional.
+        The connection ID to use connecting to Google Cloud.
+    :type gcp_conn_id: str
+    :param job_name: (DataflowRunner) Optional.
+        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 delegate_to: (DataflowRunner) Optional.
+        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
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "job_name"]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        project_id: Optional[str] = None,
+        job_name: str = "{{task.task_id}}",
+        delegate_to: Optional[str] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.job_name = job_name
+        self.project_id = project_id
+        self.job_id = None
+        self.delegate_to = delegate_to
+        self.hook: Optional[DataflowHook] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        pipeline_options = self.default_pipeline_options.copy()
+        pipeline_options.update(self.pipeline_options)
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            if self.runner == "DataflowRunner":

Review comment:
       I am not sure did I understand your comment correctly. AFAIK the behavior from user perspective will be the same, just if user wants to use more detaflow specific features available in DataflowOperator will be informed about it. Could you elaborate more?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565245551



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}

Review comment:
       ```suggestion
       template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-741969583


   [The Workflow run](https://github.com/apache/airflow/actions/runs/411305222) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-768541676


   [The Workflow run](https://github.com/apache/airflow/actions/runs/515844283) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563668817



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}

Review comment:
       I don't think it is possible. WDYT @potiuk ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-742038720


   [The Workflow run](https://github.com/apache/airflow/actions/runs/411526508) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r537712551



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(

Review comment:
       Another role of example dags is bo be part of documentation and cen be visible from docs (example from PR for [databricks](https://github.com/apache/airflow/pull/12175/files#diff-e287093666061b3eb90c0c75ea0bf551563bcb6ee00d3bf6028b0239cade2f34R64)).
   
   I refactored it slightly so local task is "fully" local and in second task both input and output will be on GCS. My idea was to show working example showing usage different sources/outputs. Also executing system tests will confirm that both tasks works.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563147948



##########
File path: airflow/providers/google/cloud/hooks/dataflow.py
##########
@@ -680,22 +582,36 @@ def start_java_dataflow(
         :param location: Job location.
         :type location: str
         """
-        name = self._build_dataflow_job_name(job_name, append_job_name)
+        warnings.warn(
+            """"This method is deprecated.

Review comment:
       Yes. If someone uses this hook method the only difference should be this deprecation warning.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-772803991


   Passing in my fork:  https://github.com/potiuk/airflow/runs/1825273695  merging it :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-768944186


   [The Workflow run](https://github.com/apache/airflow/actions/runs/517411891) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-771658629


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest master at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565236458



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,295 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.info(line.rstrip("\n"))

Review comment:
       Should we introduce some dryness here? Both loops are very similar up to the source and log level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r537734616



##########
File path: tests/providers/apache/beam/hooks/test_beam.py
##########
@@ -0,0 +1,233 @@
+# 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.
+#
+
+import copy
+import unittest
+from unittest import mock
+from unittest.mock import MagicMock
+
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.providers.apache.beam.hooks.beam import BeamHook, _BeamRunner
+
+PY_FILE = 'apache_beam.examples.wordcount'
+JAR_FILE = 'unitest.jar'
+JOB_CLASS = 'com.example.UnitTest'
+PY_OPTIONS = ['-m']
+TEST_JOB_ID = 'test-job-id'
+
+DEFAULT_RUNNER = "DirectRunner"
+BEAM_STRING = 'airflow.providers.apache.beam.hooks.beam.{}'
+BEAM_VARIABLES_PY = {'output': 'gs://test/output', 'labels': {'foo': 'bar'}}
+BEAM_VARIABLES_JAVA = {
+    'output': 'gs://test/output',
+    'labels': {'foo': 'bar'},
+}
+
+APACHE_BEAM_V_2_14_0_JAVA_SDK_LOG = f""""\
+Dataflow SDK version: 2.14.0
+Jun 15, 2020 2:57:28 PM org.apache.beam.runners.dataflow.DataflowRunner run
+INFO: To access the Dataflow monitoring console, please navigate to https://console.cloud.google.com/dataflow\
+/jobsDetail/locations/europe-west3/jobs/{TEST_JOB_ID}?project=XXX
+Submitted job: {TEST_JOB_ID}
+Jun 15, 2020 2:57:28 PM org.apache.beam.runners.dataflow.DataflowRunner run
+INFO: To cancel the job using the 'gcloud' tool, run:
+> gcloud dataflow jobs --project=XXX cancel --region=europe-west3 {TEST_JOB_ID}
+"""
+
+
+class TestBeamHook(unittest.TestCase):
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+            variables=copy.deepcopy(BEAM_VARIABLES_PY),
+            py_file=PY_FILE,
+            py_options=PY_OPTIONS,
+        )
+
+        expected_cmd = [
+            "python3",
+            '-m',
+            PY_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels=foo=bar',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+    @parameterized.expand(
+        [
+            ('default_to_python3', 'python3'),
+            ('major_version_2', 'python2'),
+            ('major_version_3', 'python3'),
+            ('minor_version', 'python3.6'),
+        ]
+    )
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline_with_custom_interpreter(self, _, py_interpreter, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+            variables=copy.deepcopy(BEAM_VARIABLES_PY),
+            py_file=PY_FILE,
+            py_options=PY_OPTIONS,
+            py_interpreter=py_interpreter,
+        )
+
+        expected_cmd = [
+            py_interpreter,
+            '-m',
+            PY_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels=foo=bar',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+    @parameterized.expand(
+        [
+            (['foo-bar'], False),
+            (['foo-bar'], True),
+            ([], True),
+        ]
+    )
+    @mock.patch(BEAM_STRING.format('prepare_virtualenv'))
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline_with_non_empty_py_requirements_and_without_system_packages(
+        self, current_py_requirements, current_py_system_site_packages, mock_runner, mock_virtualenv
+    ):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+        mock_virtualenv.return_value = '/dummy_dir/bin/python'
+
+        hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+            variables=copy.deepcopy(BEAM_VARIABLES_PY),
+            py_file=PY_FILE,
+            py_options=PY_OPTIONS,
+            py_requirements=current_py_requirements,
+            py_system_site_packages=current_py_system_site_packages,
+        )
+
+        expected_cmd = [
+            '/dummy_dir/bin/python',
+            '-m',
+            PY_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels=foo=bar',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+        mock_virtualenv.assert_called_once_with(
+            venv_directory=mock.ANY,
+            python_bin="python3",
+            system_site_packages=current_py_system_site_packages,
+            requirements=current_py_requirements,
+        )
+
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline_with_empty_py_requirements_and_without_system_packages(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        with self.assertRaisesRegex(AirflowException, "Invalid method invocation."):
+            hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+                variables=copy.deepcopy(BEAM_VARIABLES_PY),
+                py_file=PY_FILE,
+                py_options=PY_OPTIONS,
+                py_requirements=[],
+            )
+
+        mock_runner.assert_not_called()
+        wait_for_done.assert_not_called()
+
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_java_pipeline(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_java_pipeline(  # pylint: disable=no-value-for-parameter
+            jar=JAR_FILE,
+            variables=copy.deepcopy(BEAM_VARIABLES_JAVA),
+        )
+
+        expected_cmd = [
+            'java',
+            '-jar',
+            JAR_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels={"foo":"bar"}',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_java_pipeline_with_job_class(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_java_pipeline(  # pylint: disable=no-value-for-parameter
+            jar=JAR_FILE, variables=copy.deepcopy(BEAM_VARIABLES_JAVA), job_class=JOB_CLASS
+        )
+
+        expected_cmd = [
+            'java',
+            '-cp',
+            JAR_FILE,
+            JOB_CLASS,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels={"foo":"bar"}',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+
+class TestDataflow(unittest.TestCase):

Review comment:
       I found there is mistake in the title (should be `TestBeamRunner`) and it is purpose is to test `providers.apache.beam.hooks.beam._BeamRunner` which is responsible for proper executing command in subprocess, printing stdout/stderr to logs and and handling exit code. In other tests this class is mocked and if runner is Dataflow slightly different command runner will be used: `airflow.providers.google.cloud.hooks.dataflow._DataflowRunner.`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565243755



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)

Review comment:
       Should we add this function to airflow.utils, especially that we already have `convert_camel_to_snake` there?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-763083110


   [The Workflow run](https://github.com/apache/airflow/actions/runs/496748465) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-771388508


   [The Workflow run](https://github.com/apache/airflow/actions/runs/529886097) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565242502



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )

Review comment:
       I'm not a fan of examples in docstrings, especially if we have nice way to use examples in docs. It often happens that the docs examples are not updated when something changes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567662732



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,446 @@
+#
+# 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 contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.utils.helpers import convert_camel_to_snake
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if python file is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        formatted_pipeline_options = {
+            convert_camel_to_snake(key): pipeline_options[key] for key in pipeline_options
+        }
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if is_dataflow:
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        if self.dataflow_hook and self.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,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )

Review comment:
       Same here, should we consider encapsulating this logic into separate method? PosAnd if possible extracting some common logic between Python and Java may help too 🚀 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-772710474


   [The Workflow run](https://github.com/apache/airflow/actions/runs/534528544) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-762284036


   PTAL @aaltay @mik-laj @turbaszek 
   I updated PR with refactor which moves common between Dataflow and Beam to Beam.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] aaltay commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
aaltay commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r561159658



##########
File path: airflow/providers/apache/beam/BACKPORT_PROVIDER_README.md
##########
@@ -0,0 +1,101 @@
+<!--
+ 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.
+ -->
+
+
+# Package apache-airflow-backport-providers-apache-beam
+
+Release:
+
+**Table of contents**
+
+- [Backport package](#backport-package)
+- [Installation](#installation)
+- [PIP requirements](#pip-requirements)
+- [Cross provider package dependencies](#cross-provider-package-dependencies)
+- [Provider class summary](#provider-classes-summary)
+    - [Operators](#operators)
+        - [Moved operators](#moved-operators)
+    - [Transfer operators](#transfer-operators)
+        - [Moved transfer operators](#moved-transfer-operators)
+    - [Hooks](#hooks)
+        - [Moved hooks](#moved-hooks)
+- [Releases](#releases)
+    - [Release](#release)
+
+## Backport package
+
+This is a backport providers package for `apache.beam` provider. All classes for this provider package
+are in `airflow.providers.apache.beam` python package.
+
+**Only Python 3.6+ is supported for this backport package.**
+
+While Airflow 1.10.* continues to support Python 2.7+ - you need to upgrade python to 3.6+ if you
+want to use this backport package.
+
+
+
+## Installation
+
+You can install this package on top of an existing airflow 1.10.* installation via
+`pip install apache-airflow-backport-providers-apache-beam`
+
+## PIP requirements
+
+
+## Cross provider package dependencies
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+```bash
+pip install apache-airflow-backport-providers-apache-beam[google]
+```
+
+| Dependent package                                                                                                   | Extra  |
+|:--------------------------------------------------------------------------------------------------------------------|:-------|
+| [apache-airflow-backport-providers-google](https://github.com/apache/airflow/tree/master/airflow/providers/google/) | google |
+
+# Provider classes summary
+
+In Airflow 2.0, all operators, transfers, hooks, sensors, secrets for the `apache.beam` provider
+are in the `airflow.providers.apache.beam` package. You can read more about the naming conventions used
+in [Naming conventions for provider packages](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#naming-conventions-for-provider-packages)
+
+
+## Operators

Review comment:
       There is more that could be added here related to hooks, and operators maybe?

##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,287 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:

Review comment:
       Why is this needed? Ideally any Beam supported runners should just work, and there is no need for this duplication.

##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,305 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.hooks.dataflow import DataflowJobStatus
+from airflow.providers.google.cloud.operators.dataflow import DataflowPythonConfiguration
+from airflow.providers.google.cloud.sensors.dataflow import DataflowJobStatusSensor
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+GCS_PYTHON_DATAFLOW_ASYNC = os.environ.get(
+    'APACHE_BEAM_PYTHON_DATAFLOW_ASYNC', 'gs://test-dataflow-example/wordcount_debugging.py'
+)
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_pipeline_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_pipeline_direct_runner",
+        runner="DirectRunner",

Review comment:
       Is DirectRunner the default option? (If yes, you can omit this arg.)

##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}

Review comment:
       Does the operator always have the same version as airflow? Would it make sense to include what is the operator version that is used?

##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+        self._dataflow_job_name: Optional[str] = None

Review comment:
       _dataflow_job_name - I think this does not need to be a member. It seems to be local only to execute().

##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+        self._dataflow_job_name: Optional[str] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowPythonConfiguration(**self.dataflow_config)

Review comment:
       DataflowPythonConfiguration is getting a bit strange in Beam operator. I wonder if it can be merged into regular options?
   
   Another question is there a reason for DataflowPythonConfiguration / and its java equivalent. Presumably they are similar and this is more like "DataflowConfiguration"

##########
File path: airflow/providers/google/cloud/hooks/dataflow.py
##########
@@ -50,6 +47,35 @@
 T = TypeVar("T", bound=Callable)  # pylint: disable=invalid-name
 
 
+def process_line_and_extract_dataflow_job_id_callback(
+    on_new_job_id_callback: Optional[Callable[[str], None]]
+) -> Callable[[str], None]:
+    """
+    Returns callback which triggers function passed as `on_new_job_id_callback` when Dataflow job_id is found.
+    To be used for `process_line_callback` in
+    :py:class:`~airflow.providers.apache.beam.hooks.beam.BeamCommandRunner`
+
+    :param on_new_job_id_callback: Callback called when the job ID is known
+    :type on_new_job_id_callback: callback
+    """
+
+    def _process_line_and_extract_job_id(
+        line: str,
+        # on_new_job_id_callback: Optional[Callable[[str], None]]
+    ) -> None:
+        # Job id info: https://goo.gl/SE29y9.
+        matched_job = JOB_ID_PATTERN.search(line)
+        if matched_job:
+            job_id = matched_job.group("job_id_java") or matched_job.group("job_id_python")

Review comment:
       What is the difference between job_id_java and job_id_python?

##########
File path: airflow/providers/google/cloud/operators/dataflow.py
##########
@@ -43,6 +48,214 @@ class CheckJobRunning(Enum):
     WaitForRun = 3
 
 
+class DataflowConfiguration(metaclass=ABCMeta):
+    """Abstract class for Dataflow configuration to be passed to Beam operators"""
+
+    template_fields = ["job_name", "location"]
+
+    def __init__(
+        self,
+        *,
+        job_name: Optional[str] = "{{task.task_id}}",
+        append_job_name: bool = True,
+        project_id: Optional[str] = None,
+        location: Optional[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,
+        drain_pipeline: bool = False,
+        cancel_timeout: Optional[int] = 5 * 60,
+        wait_until_finished: Optional[bool] = None,
+    ) -> None:
+        self.job_name = job_name
+        self.append_job_name = append_job_name
+        self.project_id = project_id
+        self.location = location
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.poll_sleep = poll_sleep
+        self.impersonation_chain = impersonation_chain
+        self.drain_pipeline = drain_pipeline
+        self.cancel_timeout = cancel_timeout
+        self.wait_until_finished = wait_until_finished
+
+
+class DataflowPythonConfiguration(DataflowConfiguration):

Review comment:
       I wonder if you can remove DataflowPythonConfiguration and java version, in favor of only keeping the base version. (E.g. we could avoid adding another one when Go is available.)

##########
File path: airflow/providers/google/cloud/hooks/dataflow.py
##########
@@ -680,22 +582,36 @@ def start_java_dataflow(
         :param location: Job location.
         :type location: str
         """
-        name = self._build_dataflow_job_name(job_name, append_job_name)
+        warnings.warn(
+            """"This method is deprecated.

Review comment:
       Are the new versions fully backward compatible with these ones?

##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+        self._dataflow_job_name: Optional[str] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowPythonConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = self._dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=self._dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowJavaConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowJavaConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowJavaConfiguration(**self.dataflow_config)

Review comment:
       what happens if dataflow config is not none, but the runner is something other than dataflow runner. Should it be an error/ warning? 

##########
File path: airflow/providers/google/cloud/operators/dataflow.py
##########
@@ -43,6 +48,214 @@ class CheckJobRunning(Enum):
     WaitForRun = 3
 
 
+class DataflowConfiguration(metaclass=ABCMeta):
+    """Abstract class for Dataflow configuration to be passed to Beam operators"""
+
+    template_fields = ["job_name", "location"]
+
+    def __init__(
+        self,
+        *,
+        job_name: Optional[str] = "{{task.task_id}}",
+        append_job_name: bool = True,
+        project_id: Optional[str] = None,
+        location: Optional[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,
+        drain_pipeline: bool = False,
+        cancel_timeout: Optional[int] = 5 * 60,
+        wait_until_finished: Optional[bool] = None,
+    ) -> None:
+        self.job_name = job_name
+        self.append_job_name = append_job_name
+        self.project_id = project_id
+        self.location = location
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.poll_sleep = poll_sleep
+        self.impersonation_chain = impersonation_chain
+        self.drain_pipeline = drain_pipeline
+        self.cancel_timeout = cancel_timeout
+        self.wait_until_finished = wait_until_finished
+
+
+class DataflowPythonConfiguration(DataflowConfiguration):
+    """
+    Dataflow configuration that can be passed to
+    :py:class:`~airflow.providers.apache.beam.operators.beam.BeamRunPythonPipelineOperator`
+
+    :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
+    :param append_job_name: True if unique suffix has to be appended to job name.
+    :type append_job_name: bool
+    :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]]
+    :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]
+    """
+
+
+class DataflowJavaConfiguration(DataflowConfiguration):
+    """
+    Dataflow configuration that can be passed to
+    :py:class:`~airflow.providers.apache.beam.operators.beam.BeamRunJavaPipelineOperator`
+
+    :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
+    :param append_job_name: True if unique suffix has to be appended to job name.
+    :type append_job_name: bool
+    :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]]
+    :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]
+    :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.
+    """
+
+    def __init__(
+        self,
+        *,
+        multiple_jobs: Optional[bool] = None,
+        check_if_running: CheckJobRunning = CheckJobRunning.WaitForRun,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.multiple_jobs = multiple_jobs
+        self.check_if_running = check_if_running
+
+
 # pylint: disable=too-many-instance-attributes
 class DataflowCreateJavaJobOperator(BaseOperator):

Review comment:
       Should this have a deprecation warning?

##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}

Review comment:
       why not use version as is, instead of adding a "v" and changing . and +'s to dashes?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567435866



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,295 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code {self._proc.returncode}")

Review comment:
       I put `AirflowException` here. I have doubts about `SystemExit` because it requests exit from interpreter. I am not sure what are the consequences of exiting interpreter on the worker.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-772471044


   [The Workflow run](https://github.com/apache/airflow/actions/runs/533218145) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567469369



##########
File path: setup.py
##########
@@ -522,6 +522,7 @@ def get_sphinx_theme_version() -> str:
 # Dict of all providers which are part of the Apache Airflow repository together with their requirements
 PROVIDERS_REQUIREMENTS: Dict[str, List[str]] = {
     'amazon': amazon,
+    'apache.beam': [],

Review comment:
       ```suggestion
       'apache.beam': apache_beam,
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-763006552


   [The Workflow run](https://github.com/apache/airflow/actions/runs/496553448) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563146228



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}

Review comment:
       It is line moved from Dataflow hook.
   The reasons for such formatting are restrictions ([Labeling resources - label format](https://cloud.google.com/compute/docs/labeling-resources#label_format)):
   - `v` solves `Label keys must start with a lowercase letter.`
   - `.` and `+` are not allowed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-768996473


   [The Workflow run](https://github.com/apache/airflow/actions/runs/517748001) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] aaltay commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
aaltay commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r536302547



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')

Review comment:
       Curiosity question: Does `gs://test-dataflow-example/` exist as a publicly writable location?

##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_direct_runner",
+        py_file='apache_beam.examples.wordcount',
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_direct_runner",
+        py_file=GCS_PYTHON,
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_dataflow_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_dataflow_runner",
+        runner="DataflowRunner",
+        py_file=GCS_PYTHON,
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],

Review comment:
       Use a newer version in examples? 2.25.0 is the latest, 2.26.0 is imminent.

##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(

Review comment:
       I think this and start_python_job_direct_runner are a bit duplicative. They more or less do the same exact thing except for the location of module to run.

##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,234 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class _BeamRunner(LoggingMixin):
+    def __init__(
+        self,
+        cmd: List[str],
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)

Review comment:
       Is it possible to capture stdout or stderr, that usually has the real relevant errors.

##########
File path: tests/providers/apache/beam/hooks/test_beam.py
##########
@@ -0,0 +1,233 @@
+# 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.
+#
+
+import copy
+import unittest
+from unittest import mock
+from unittest.mock import MagicMock
+
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.providers.apache.beam.hooks.beam import BeamHook, _BeamRunner
+
+PY_FILE = 'apache_beam.examples.wordcount'
+JAR_FILE = 'unitest.jar'
+JOB_CLASS = 'com.example.UnitTest'
+PY_OPTIONS = ['-m']
+TEST_JOB_ID = 'test-job-id'
+
+DEFAULT_RUNNER = "DirectRunner"
+BEAM_STRING = 'airflow.providers.apache.beam.hooks.beam.{}'
+BEAM_VARIABLES_PY = {'output': 'gs://test/output', 'labels': {'foo': 'bar'}}
+BEAM_VARIABLES_JAVA = {
+    'output': 'gs://test/output',
+    'labels': {'foo': 'bar'},
+}
+
+APACHE_BEAM_V_2_14_0_JAVA_SDK_LOG = f""""\
+Dataflow SDK version: 2.14.0
+Jun 15, 2020 2:57:28 PM org.apache.beam.runners.dataflow.DataflowRunner run
+INFO: To access the Dataflow monitoring console, please navigate to https://console.cloud.google.com/dataflow\
+/jobsDetail/locations/europe-west3/jobs/{TEST_JOB_ID}?project=XXX
+Submitted job: {TEST_JOB_ID}
+Jun 15, 2020 2:57:28 PM org.apache.beam.runners.dataflow.DataflowRunner run
+INFO: To cancel the job using the 'gcloud' tool, run:
+> gcloud dataflow jobs --project=XXX cancel --region=europe-west3 {TEST_JOB_ID}
+"""
+
+
+class TestBeamHook(unittest.TestCase):
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+            variables=copy.deepcopy(BEAM_VARIABLES_PY),
+            py_file=PY_FILE,
+            py_options=PY_OPTIONS,
+        )
+
+        expected_cmd = [
+            "python3",
+            '-m',
+            PY_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels=foo=bar',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+    @parameterized.expand(
+        [
+            ('default_to_python3', 'python3'),
+            ('major_version_2', 'python2'),
+            ('major_version_3', 'python3'),
+            ('minor_version', 'python3.6'),
+        ]
+    )
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline_with_custom_interpreter(self, _, py_interpreter, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+            variables=copy.deepcopy(BEAM_VARIABLES_PY),
+            py_file=PY_FILE,
+            py_options=PY_OPTIONS,
+            py_interpreter=py_interpreter,
+        )
+
+        expected_cmd = [
+            py_interpreter,
+            '-m',
+            PY_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels=foo=bar',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+    @parameterized.expand(
+        [
+            (['foo-bar'], False),
+            (['foo-bar'], True),
+            ([], True),
+        ]
+    )
+    @mock.patch(BEAM_STRING.format('prepare_virtualenv'))
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline_with_non_empty_py_requirements_and_without_system_packages(
+        self, current_py_requirements, current_py_system_site_packages, mock_runner, mock_virtualenv
+    ):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+        mock_virtualenv.return_value = '/dummy_dir/bin/python'
+
+        hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+            variables=copy.deepcopy(BEAM_VARIABLES_PY),
+            py_file=PY_FILE,
+            py_options=PY_OPTIONS,
+            py_requirements=current_py_requirements,
+            py_system_site_packages=current_py_system_site_packages,
+        )
+
+        expected_cmd = [
+            '/dummy_dir/bin/python',
+            '-m',
+            PY_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels=foo=bar',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+        mock_virtualenv.assert_called_once_with(
+            venv_directory=mock.ANY,
+            python_bin="python3",
+            system_site_packages=current_py_system_site_packages,
+            requirements=current_py_requirements,
+        )
+
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_python_pipeline_with_empty_py_requirements_and_without_system_packages(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        with self.assertRaisesRegex(AirflowException, "Invalid method invocation."):
+            hook.start_python_pipeline(  # pylint: disable=no-value-for-parameter
+                variables=copy.deepcopy(BEAM_VARIABLES_PY),
+                py_file=PY_FILE,
+                py_options=PY_OPTIONS,
+                py_requirements=[],
+            )
+
+        mock_runner.assert_not_called()
+        wait_for_done.assert_not_called()
+
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_java_pipeline(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_java_pipeline(  # pylint: disable=no-value-for-parameter
+            jar=JAR_FILE,
+            variables=copy.deepcopy(BEAM_VARIABLES_JAVA),
+        )
+
+        expected_cmd = [
+            'java',
+            '-jar',
+            JAR_FILE,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels={"foo":"bar"}',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+    @mock.patch(BEAM_STRING.format('_BeamRunner'))
+    def test_start_java_pipeline_with_job_class(self, mock_runner):
+        hook = BeamHook(runner=DEFAULT_RUNNER)
+        wait_for_done = mock_runner.return_value.wait_for_done
+
+        hook.start_java_pipeline(  # pylint: disable=no-value-for-parameter
+            jar=JAR_FILE, variables=copy.deepcopy(BEAM_VARIABLES_JAVA), job_class=JOB_CLASS
+        )
+
+        expected_cmd = [
+            'java',
+            '-cp',
+            JAR_FILE,
+            JOB_CLASS,
+            f'--runner={DEFAULT_RUNNER}',
+            '--output=gs://test/output',
+            '--labels={"foo":"bar"}',
+        ]
+        mock_runner.assert_called_once_with(cmd=expected_cmd)
+        wait_for_done.assert_called_once_with()
+
+
+class TestDataflow(unittest.TestCase):

Review comment:
       You can also run a real Beam pipeline with DirectRunner as another test.

##########
File path: airflow/providers/apache/beam/provider.yaml
##########
@@ -0,0 +1,40 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-apache-beam
+name: Apache Beam
+description: |
+    `Apache Druid <https://druid.apache.org/>`__.

Review comment:
       Apache Beam ... ?

##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_direct_runner",
+        py_file='apache_beam.examples.wordcount',
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_direct_runner",
+        py_file=GCS_PYTHON,
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_dataflow_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_dataflow_runner",
+        runner="DataflowRunner",
+        py_file=GCS_PYTHON,
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_local_spark_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_spark_runner",
+        py_file='apache_beam.examples.wordcount',
+        runner="SparkRunner",
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_local_flink_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_flink_runner",
+        py_file='apache_beam.examples.wordcount',
+        runner="FlinkRunner",
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_python_job_local_flink_runner',
+        },
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_local_direct_runner >> start_python_job_local_flink_runner

Review comment:
       Are these two lines equivalent to [start_python_job_local_direct_runner, start_python_job_direct_runner] >> start_python_job_local_flink_runner




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-771388508






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-771151611


   [The Workflow run](https://github.com/apache/airflow/actions/runs/528875448) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567254021



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,295 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code {self._proc.returncode}")

Review comment:
       First, that's my feeling for an exception with information "return code...". Second, specifying the exception type make it easier to catch it without need of broad except clause.
   (I thought I answered to this 2 days ago 👀 )




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r568335069



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,446 @@
+#
+# 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 contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.utils.helpers import convert_camel_to_snake
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if python file is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        formatted_pipeline_options = {
+            convert_camel_to_snake(key): pipeline_options[key] for key in pipeline_options
+        }
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if is_dataflow:
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        if self.dataflow_hook and self.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,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )

Review comment:
       @turbaszek thanks for suggestion. I made PoC of refactor. PTAL: https://github.com/PolideaInternal/airflow/pull/976




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r567254201



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,474 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        camel_to_snake = lambda name: re.sub(r"[A-Z]", lambda x: "_" + x.group(0).lower(), name)
+        formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options}
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        self.log.info("On kill.")
+        if self.dataflow_hook and self.dataflow_job_id:
+            self.dataflow_hook.cancel_job(
+                job_id=self.dataflow_job_id,
+                project_id=self.dataflow_config.project_id,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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.
+
+    It's a good practice to define parameters in the default_args of the dag
+    like the project, zone and staging location.
+
+    .. code-block:: python
+
+       default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+    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.
+
+    .. code-block:: python
+
+       t1 = BeamRunJavaPipelineOperator(
+           task_id='start_java_job_spark_runner',
+           jar='{{var.value.spark_runner_jar_base}}pipeline/build/libs/pipeline-example-1.0.jar',
+           pipeline_options={
+               'output': '/tmp/start_java_job_spark_runner',
+               'inputFile': 'gs://apache-beam-samples/shakespeare/kinglear.txt,
+           },
+           dag=my-dag)
+
+    .. seealso::
+        For more detail on Apache Beam have a look at the reference:
+        https://beam.apache.org/documentation/
+
+    :param jar: The reference to a self executing Apache Beam jar (templated).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        with ExitStack() as exit_stack:
+            if self.jar.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.jar)
+                )
+                self.jar = tmp_gcs_file.name
+
+            if self.runner.lower() == BeamRunnerType.DataflowRunner.lower():
+                is_running = False
+                if self.dataflow_config.check_if_running != CheckJobRunning.IgnoreJob:
+                    is_running = (
+                        self.dataflow_hook.is_job_dataflow_running(  # pylint: disable=no-value-for-parameter

Review comment:
       Would you mind adding small comment in code? I think it may help others in future




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-770399087


   [The Workflow run](https://github.com/apache/airflow/actions/runs/525446718) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r568335069



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,446 @@
+#
+# 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 contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import CheckJobRunning, DataflowConfiguration
+from airflow.utils.decorators import apply_defaults
+from airflow.utils.helpers import convert_camel_to_snake
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if python file is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+        )
+        self.py_interpreter = py_interpreter
+        self.py_requirements = py_requirements
+        self.py_system_site_packages = py_system_site_packages
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_config = dataflow_config or {}
+        self.beam_hook: Optional[BeamHook] = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.dataflow_job_id: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["job_name"] = dataflow_job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )
+
+        pipeline_options.update(self.pipeline_options)
+
+        # Convert argument names from lowerCamelCase to snake case.
+        formatted_pipeline_options = {
+            convert_camel_to_snake(key): pipeline_options[key] for key in pipeline_options
+        }
+
+        with ExitStack() as exit_stack:
+            if self.py_file.lower().startswith("gs://"):
+                gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to)
+                tmp_gcs_file = exit_stack.enter_context(  # pylint: disable=no-member
+                    gcs_hook.provide_file(object_url=self.py_file)
+                )
+                self.py_file = tmp_gcs_file.name
+
+            self.beam_hook.start_python_pipeline(
+                variables=formatted_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,
+                process_line_callback=process_line_callback,
+            )
+
+            if is_dataflow:
+                self.dataflow_hook.wait_for_done(  # pylint: disable=no-value-for-parameter
+                    job_name=dataflow_job_name,
+                    location=self.dataflow_config.location,
+                    job_id=self.dataflow_job_id,
+                    multiple_jobs=False,
+                )
+
+        return {"dataflow_job_id": self.dataflow_job_id}
+
+    def on_kill(self) -> None:
+        if self.dataflow_hook and self.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,
+            )
+
+
+# pylint: disable=too-many-instance-attributes
+class BeamRunJavaPipelineOperator(BaseOperator):
+    """
+    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).
+    :type jar: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        See:
+        https://beam.apache.org/documentation/runners/capability-matrix/
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+    :type runner: str
+    :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.
+    :type job_class: str
+    :param default_pipeline_options: Map of default job pipeline_options.
+    :type default_pipeline_options: dict
+    :param pipeline_options: Map of job specific 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 pipeline_options will be added for each key.
+          If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key-B`` pipeline_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 pipeline_options: dict
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Storage if jar is on GCS
+    :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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowConfiguration]
+    """
+
+    template_fields = [
+        "jar",
+        "runner",
+        "job_class",
+        "pipeline_options",
+        "default_pipeline_options",
+        "dataflow_config",
+    ]
+    template_fields_renderers = {'dataflow_config': 'json', 'pipeline_options': 'json'}
+    ui_color = "#0273d4"
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        jar: str,
+        runner: str = "DirectRunner",
+        job_class: Optional[str] = None,
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_options: Optional[dict] = None,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.jar = jar
+        self.runner = runner
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.job_class = job_class
+        self.dataflow_config = dataflow_config or {}
+        self.gcp_conn_id = gcp_conn_id
+        self.delegate_to = delegate_to
+        self.dataflow_job_id = None
+        self.dataflow_hook: Optional[DataflowHook] = None
+        self.beam_hook: Optional[BeamHook] = None
+        self._dataflow_job_name: Optional[str] = None
+
+        if self.dataflow_config and self.runner.lower() != BeamRunnerType.DataflowRunner.lower():
+            self.log.warning(
+                "dataflow_config is defined but runner is different than DataflowRunner (%s)", self.runner
+            )
+
+    def execute(self, context):
+        """Execute the Apache Beam Pipeline."""
+        self.beam_hook = BeamHook(runner=self.runner)
+        pipeline_options = self.default_pipeline_options.copy()
+        process_line_callback: Optional[Callable] = None
+        is_dataflow = self.runner.lower() == BeamRunnerType.DataflowRunner.lower()
+
+        if isinstance(self.dataflow_config, dict):
+            self.dataflow_config = DataflowConfiguration(**self.dataflow_config)
+
+        if is_dataflow:
+            self.dataflow_hook = DataflowHook(
+                gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id,
+                delegate_to=self.dataflow_config.delegate_to or self.delegate_to,
+                poll_sleep=self.dataflow_config.poll_sleep,
+                impersonation_chain=self.dataflow_config.impersonation_chain,
+                drain_pipeline=self.dataflow_config.drain_pipeline,
+                cancel_timeout=self.dataflow_config.cancel_timeout,
+                wait_until_finished=self.dataflow_config.wait_until_finished,
+            )
+            self.dataflow_config.project_id = self.dataflow_config.project_id or self.dataflow_hook.project_id
+
+            self._dataflow_job_name = DataflowHook.build_dataflow_job_name(
+                self.dataflow_config.job_name, self.dataflow_config.append_job_name
+            )
+            pipeline_options["jobName"] = self.dataflow_config.job_name
+            pipeline_options["project"] = self.dataflow_config.project_id
+            pipeline_options["region"] = self.dataflow_config.location
+            pipeline_options.setdefault("labels", {}).update(
+                {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}
+            )
+
+            def set_current_dataflow_job_id(job_id):
+                self.dataflow_job_id = job_id
+
+            process_line_callback = process_line_and_extract_dataflow_job_id_callback(
+                on_new_job_id_callback=set_current_dataflow_job_id
+            )

Review comment:
       I made PoC of refactor. PTAL: https://github.com/PolideaInternal/airflow/pull/976




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r563145560



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,287 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:

Review comment:
       I added it because I wanted to avoid typing string "DirectRunner" in many places.
   I extended list of runner types and added link to the documentation. What do you think about such solution?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] RosterIn commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
RosterIn commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r571811592



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,289 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook

Review comment:
       @TobKed This raises warning that spams the logs:
   ```
   /opt/airflow/airflow/providers/apache/beam/hooks/beam.py:28: DeprecationWarning: This module is deprecated. Please use `airflow.hooks.base`. 
   from airflow.hooks.base_hook import BaseHook  
   ```
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565243038



##########
File path: airflow/providers/apache/beam/operators/beam.py
##########
@@ -0,0 +1,469 @@
+#
+# 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."""
+import re
+from contextlib import ExitStack
+from typing import Callable, List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType
+from airflow.providers.google.cloud.hooks.dataflow import (
+    DataflowHook,
+    process_line_and_extract_dataflow_job_id_callback,
+)
+from airflow.providers.google.cloud.hooks.gcs import GCSHook
+from airflow.providers.google.cloud.operators.dataflow import (
+    CheckJobRunning,
+    DataflowJavaConfiguration,
+    DataflowPythonConfiguration,
+)
+from airflow.utils.decorators import apply_defaults
+from airflow.version import version
+
+
+class BeamRunPythonPipelineOperator(BaseOperator):
+    """
+    Launching 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.
+
+    .. code-block:: python
+
+        default_args = {
+            'default_pipeline_options':
+                {
+                    'labels': 'example-label'
+                }
+        }
+
+        with models.DAG(
+            "example_beam_native_python",
+            default_args=default_args,
+            start_date=days_ago(1),
+            schedule_interval=None,
+            tags=['example'],
+        ) as dag_native_python:
+
+            start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+                task_id="start_python_job_local_direct_runner",
+                runner="DirectRunner",
+                py_file='apache_beam.examples.wordcount',
+                py_options=['-m'],
+                py_requirements=['apache-beam[gcp]==2.21.0'],
+                py_interpreter='python3',
+                py_system_site_packages=False,
+            )
+
+    .. 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)
+    :type py_file: str
+    :param runner: Runner on which pipeline will be run. By default "DirectRunner" is being used.
+        Other possible options: DataflowRunner, SparkRunner, FlinkRunner.
+        See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
+        See: https://beam.apache.org/documentation/runners/capability-matrix/
+
+        If you use Dataflow runner check dedicated operator:
+        :class:`~providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+    :type runner: str
+    :param py_options: Additional python options, e.g., ["-m", "-v"].
+    :type py_options: list[str]
+    :param default_pipeline_options: Map of default pipeline options.
+    :type default_pipeline_options: dict
+    :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.
+    :type pipeline_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: Optional.
+        The connection ID to use connecting to Google Cloud Storage if pyfile is on GCS.
+    :type gcp_conn_id: str
+    :param delegate_to:  Optional.
+        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 dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner
+    :type dataflow_config: Union[dict, providers.google.cloud.operators.dataflow.DataflowPythonConfiguration]
+    """
+
+    template_fields = ["py_file", "runner", "pipeline_options", "default_pipeline_options", "dataflow_config"]
+    template_fields_renderers = {'dataflow_config': 'json'}
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        py_file: str,
+        runner: str = "DirectRunner",
+        default_pipeline_options: Optional[dict] = None,
+        pipeline_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,
+        gcp_conn_id: str = "google_cloud_default",
+        delegate_to: Optional[str] = None,
+        dataflow_config: Optional[Union[DataflowPythonConfiguration, dict]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+
+        self.py_file = py_file
+        self.runner = runner
+        self.py_options = py_options or []
+        self.default_pipeline_options = default_pipeline_options or {}
+        self.pipeline_options = pipeline_options or {}
+        self.pipeline_options.setdefault("labels", {}).update(
+            {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")}

Review comment:
       I think we may try to get version of providers package installed, but I'm not sure how much value this will give




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-744421456


   [The Workflow run](https://github.com/apache/airflow/actions/runs/420755272) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#issuecomment-740179106


   [The Workflow run](https://github.com/apache/airflow/actions/runs/406757473) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565236845



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,295 @@
+#
+# 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 a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code {self._proc.returncode}")

Review comment:
       Wouldn't `SystemExit` be better?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] TobKed commented on a change in pull request #12814: Add Apache Beam operators

Posted by GitBox <gi...@apache.org>.
TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r537697679



##########
File path: airflow/providers/apache/beam/example_dags/example_beam.py
##########
@@ -0,0 +1,262 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Apache Beam operators
+"""
+import os
+from urllib.parse import urlparse
+
+from airflow import models
+from airflow.providers.apache.beam.operators.beam import (
+    BeamRunJavaPipelineOperator,
+    BeamRunPythonPipelineOperator,
+)
+from airflow.providers.google.cloud.transfers.gcs_to_local import GCSToLocalFilesystemOperator
+from airflow.utils.dates import days_ago
+
+GCS_INPUT = os.environ.get('APACHE_BEAM_PYTHON', 'gs://apache-beam-samples/shakespeare/kinglear.txt')
+GCS_TMP = os.environ.get('APACHE_BEAM_GCS_TMP', 'gs://test-dataflow-example/temp/')
+GCS_STAGING = os.environ.get('APACHE_BEAM_GCS_STAGING', 'gs://test-dataflow-example/staging/')
+GCS_OUTPUT = os.environ.get('APACHE_BEAM_GCS_OUTPUT', 'gs://test-dataflow-example/output')
+GCS_PYTHON = os.environ.get('APACHE_BEAM_PYTHON', 'gs://test-dataflow-example/wordcount_debugging.py')
+
+GCS_JAR_DIRECT_RUNNER = os.environ.get(
+    'APACHE_BEAM_DIRECT_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-DirectRunner.jar',
+)
+GCS_JAR_DATAFLOW_RUNNER = os.environ.get(
+    'APACHE_BEAM_DATAFLOW_RUNNER_JAR', 'gs://test-dataflow-example/word-count-beam-bundled-0.1.jar'
+)
+GCS_JAR_SPARK_RUNNER = os.environ.get(
+    'APACHE_BEAM_SPARK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-SparkRunner.jar',
+)
+GCS_JAR_FLINK_RUNNER = os.environ.get(
+    'APACHE_BEAM_FLINK_RUNNER_JAR',
+    'gs://test-dataflow-example/tests/dataflow-templates-bundled-java=11-beam-v2.25.0-FlinkRunner.jar',
+)
+
+GCS_JAR_DIRECT_RUNNER_PARTS = urlparse(GCS_JAR_DIRECT_RUNNER)
+GCS_JAR_DIRECT_RUNNER_BUCKET_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.netloc
+GCS_JAR_DIRECT_RUNNER_OBJECT_NAME = GCS_JAR_DIRECT_RUNNER_PARTS.path[1:]
+GCS_JAR_DATAFLOW_RUNNER_PARTS = urlparse(GCS_JAR_DATAFLOW_RUNNER)
+GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.netloc
+GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME = GCS_JAR_DATAFLOW_RUNNER_PARTS.path[1:]
+GCS_JAR_SPARK_RUNNER_PARTS = urlparse(GCS_JAR_SPARK_RUNNER)
+GCS_JAR_SPARK_RUNNER_BUCKET_NAME = GCS_JAR_SPARK_RUNNER_PARTS.netloc
+GCS_JAR_SPARK_RUNNER_OBJECT_NAME = GCS_JAR_SPARK_RUNNER_PARTS.path[1:]
+GCS_JAR_FLINK_RUNNER_PARTS = urlparse(GCS_JAR_FLINK_RUNNER)
+GCS_JAR_FLINK_RUNNER_BUCKET_NAME = GCS_JAR_FLINK_RUNNER_PARTS.netloc
+GCS_JAR_FLINK_RUNNER_OBJECT_NAME = GCS_JAR_FLINK_RUNNER_PARTS.path[1:]
+
+
+default_args = {
+    'default_pipeline_options': {
+        'output': '/tmp/example_beam',
+    },
+    "trigger_rule": "all_done",
+}
+
+
+with models.DAG(
+    "example_beam_native_java_direct_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_direct_runner:
+
+    jar_to_local_direct_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_direct_runner",
+        bucket=GCS_JAR_DIRECT_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DIRECT_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_direct_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_direct_runner",
+        runner="DirectRunner",
+        jar="/tmp/beam_wordcount_direct_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_direct_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_direct_runner >> start_java_job_direct_runner
+
+with models.DAG(
+    "example_beam_native_java_dataflow_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_dataflow_runner:
+
+    jar_to_local_dataflow_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_dataflow_runner",
+        bucket=GCS_JAR_DATAFLOW_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_DATAFLOW_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_dataflow = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_dataflow",
+        runner="DataflowRunner",
+        jar="/tmp/beam_wordcount_dataflow_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_dataflow_runner >> start_java_job_dataflow
+
+with models.DAG(
+    "example_beam_native_java_spark_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_spark_runner:
+
+    jar_to_local_spark_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_spark_runner",
+        bucket=GCS_JAR_SPARK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_SPARK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_spark_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_spark_runner",
+        runner="SparkRunner",
+        jar="/tmp/beam_wordcount_spark_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_spark_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_spark_runner >> start_java_job_spark_runner
+
+with models.DAG(
+    "example_beam_native_java_flink_runner",
+    schedule_interval=None,  # Override to match your needs
+    start_date=days_ago(1),
+    tags=['example'],
+) as dag_native_java_flink_runner:
+
+    jar_to_local_flink_runner = GCSToLocalFilesystemOperator(
+        task_id="jar_to_local_flink_runner",
+        bucket=GCS_JAR_FLINK_RUNNER_BUCKET_NAME,
+        object_name=GCS_JAR_FLINK_RUNNER_OBJECT_NAME,
+        filename="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+    )
+
+    start_java_job_flink_runner = BeamRunJavaPipelineOperator(
+        task_id="start_java_job_flink_runner",
+        runner="FlinkRunner",
+        jar="/tmp/beam_wordcount_flink_runner_{{ ds_nodash }}.jar",
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_java_job_flink_runner',
+            'inputFile': GCS_INPUT,
+        },
+        job_class='org.apache.beam.examples.WordCount',
+    )
+
+    jar_to_local_flink_runner >> start_java_job_flink_runner
+
+
+with models.DAG(
+    "example_beam_native_python",
+    default_args=default_args,
+    start_date=days_ago(1),
+    schedule_interval=None,  # Override to match your needs
+    tags=['example'],
+) as dag_native_python:
+
+    start_python_job_local_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_direct_runner",
+        py_file='apache_beam.examples.wordcount',
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_direct_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_direct_runner",
+        py_file=GCS_PYTHON,
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_dataflow_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_dataflow_runner",
+        runner="DataflowRunner",
+        py_file=GCS_PYTHON,
+        pipeline_options={
+            'tempLocation': GCS_TMP,
+            'stagingLocation': GCS_STAGING,
+            'output': GCS_OUTPUT,
+        },
+        py_options=[],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_local_spark_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_spark_runner",
+        py_file='apache_beam.examples.wordcount',
+        runner="SparkRunner",
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_local_flink_runner = BeamRunPythonPipelineOperator(
+        task_id="start_python_job_local_flink_runner",
+        py_file='apache_beam.examples.wordcount',
+        runner="FlinkRunner",
+        py_options=['-m'],
+        job_name='{{task.task_id}}',
+        pipeline_options={
+            'output': '/tmp/start_python_job_local_flink_runner',
+        },
+        py_requirements=['apache-beam[gcp]==2.21.0'],
+        py_interpreter='python3',
+        py_system_site_packages=False,
+    )
+
+    start_python_job_local_direct_runner >> start_python_job_local_flink_runner

Review comment:
       You are right. It will be easier to follow the DAG, i made small refactor of this part.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org