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 2021/01/04 14:50:46 UTC

[GitHub] [airflow] TobKed opened a new pull request #13461: Add How To Guide for Dataflow

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


   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


----------------------------------------------------------------
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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r553372275



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline

Review comment:
       I would like to develop it, because it is very problematic for users.
   
   I would suggest such a text, but it can probably be improved and expanded.
   ```
   There are several ways to run a Dataflow pipeline depending on your environment, source files:
   - **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker if you have a '*.jar' file for Java or a '* .py` file for Python. This also means that the necessary system dependencies must be installed on the worker.  For Java, worker must have the JRE Runtime installed. For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions. This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies, it often causes problems. 
   - **Templated pipeline**: The programmer can make the pipeline independent of the environment by preparing a template that will then be run on a machine managed by Google. This way, changes to the environment won't affect your pipeline. There are two types of the templates:
        - **Classic templates**. Developers run the pipeline and create a template. The Apache Beam SDK stages files in Cloud Storage, creates a template file (similar to job request), and saves the template file in Cloud Storage.
       - **Flex Templates**. Developers package the pipeline into a Docker image and then use the `gcloud` command-line tool to build and save the Flex Template spec file in Cloud Storage. 
   - **SQL pipeline**: Developer can write pipeline as SQL statement and then execute it in Dataflow.
   
   It is a good idea to test your pipeline using the non-templated pipeline, and then run the pipeline in production using the templates.
   
   For details on the differences between the pipeline types, see `Dataflow templates <https://cloud.google.com/dataflow/docs/concepts/dataflow-templates>__` in the Google Cloud documentation.
   ```

##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline

Review comment:
       I would like to develop it, because it is very problematic for users.
   
   I would suggest such a text, but it can probably be improved and expanded.
   ```
   There are several ways to run a Dataflow pipeline depending on your environment, source files:
   - **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker if you have a '*.jar' file for Java or a '* .py` file for Python. This also means that the necessary system dependencies must be installed on the worker.  For Java, worker must have the JRE Runtime installed. For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions. This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies, it often causes problems. 
   - **Templated pipeline**: The programmer can make the pipeline independent of the environment by preparing a template that will then be run on a machine managed by Google. This way, changes to the environment won't affect your pipeline. There are two types of the templates:
        - **Classic templates**. Developers run the pipeline and create a template. The Apache Beam SDK stages files in Cloud Storage, creates a template file (similar to job request), and saves the template file in Cloud Storage.
         - **Flex Templates**. Developers package the pipeline into a Docker image and then use the `gcloud` command-line tool to build and save the Flex Template spec file in Cloud Storage. 
   - **SQL pipeline**: Developer can write pipeline as SQL statement and then execute it in Dataflow.
   
   It is a good idea to test your pipeline using the non-templated pipeline, and then run the pipeline in production using the templates.
   
   For details on the differences between the pipeline types, see `Dataflow templates <https://cloud.google.com/dataflow/docs/concepts/dataflow-templates>__` in the Google Cloud documentation.
   ```




----------------------------------------------------------------
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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r553668600



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline

Review comment:
       Very often these problems are not easy to solve because one common Docker image is used for many environments. For example, in Cloud Composer, you cannot install any system dependencies. The only thing you can do is install the new libraries via pip. I agree that this is a fairly strong-worded sentence and we can think about improving 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 #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.

Review comment:
       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] TobKed commented on a change in pull request #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline
+code from a source file (Java or Python) it would be best to use the language specific create operators.
+If a process exists to stage the pipeline code in an abstracted manner - a Templated job would be best as
+it allows development of the application without minimal intrusion to the DAG containing operators for it.
+It is also possible to run jobs defined in SQL language.
+
+Starting a new job
+^^^^^^^^^^^^^^^^^^
+
+To create a new pipeline using the source file (JAR in Java or Python file) use
+the create job operators. The source file can be located on GCS or on the local filesystem.
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+or
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+
+Please see the notes below on Java and Python specific SDKs as they each have their own set
+of execution options when running pipelines.
+
+Language specific pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Based on which language (SDK) is used for the Dataflow operators, there are specific options to be wary of.
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Java SDK pipelines
+""""""""""""""""""
+
+The ``jar`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+as it contains the pipeline to be executed on Dataflow. The JAR can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_jar_on_gcs]
+    :end-before: [END howto_operator_start_java_job_jar_on_gcs]
+
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_local_jar]
+    :end-before: [END howto_operator_start_java_job_local_jar]
+
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+Python SDK pipelines
+""""""""""""""""""""
+
+The ``py_file`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+as it contains the pipeline to be executed on Dataflow. The Python file can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+The ``py_interpreter`` argument specifies the Python version to be used when executing the pipeline, the default
+is ``python3`. If your Airflow instance is running on Python 2 - specify ``python2`` and ensure your ``py_file`` is
+in Python 2. For best results, use Python 3.
+
+If ``py_requirements`` argument is specified a temporary Python virtual environment with specified requirements will be created
+and within it pipeline will run.
+
+The ``py_system_site_packages`` argument specifies whether or not all the Python packages from your Airflow instance,
+will be accessible within virtual environment (if ``py_requirements`` argument is specified),
+recommend avoiding unless the Dataflow job requires it.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_python_job]
+    :end-before: [END howto_operator_start_python_job]
+
+
+Execution options for pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Dataflow has multiple options of executing pipelines. It can be done in the following modes:
+batch asynchronously (fire and forget), batch blocking (wait until completion), or streaming (run indefinitely).

Review comment:
       It is based on the Dataflow documentation:
   
   https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#configuring-pipelineoptions-for-execution-on-the-cloud-dataflow-service




----------------------------------------------------------------
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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r553372275



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline

Review comment:
       I would like to develop it, because it is very problematic for users.
   
   I would suggest such a text, but it can probably be improved and expanded.
   ```
   There are several ways to run a Dataflow pipeline depending on your environment, source files:
   - **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker if you have a '*.jar' file for Java or a '* .py` file for Python. This also means that the necessary system dependencies must be installed on the worker.  For Java, worker must have the JRE Runtime installed. For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions. This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies, it often causes problems. 
   - **Templated pipeline**: The programmer can make the pipeline independent of the environment by preparing a template that will then be run on a machine managed by Google. This way, changes to the environment won't affect your pipeline. There are two types of the templates:
        - **Classic templates**. Developers run the pipeline and create a template. The Apache Beam SDK stages files in Cloud Storage, creates a template file (similar to job request), and saves the template file in Cloud Storage.
        - **Flex Templates**. Developers package the pipeline into a Docker image and then use the `gcloud` command-line tool to build and save the Flex Template spec file in Cloud Storage. 
   - **SQL pipeline**: Developer can write pipeline as SQL statement and then execute it in Dataflow.
   
   It is a good idea to test your pipeline using the non-templated pipeline, and then run the pipeline in production using the templates.
   
   For details on the differences between the pipeline types, see `Dataflow templates <https://cloud.google.com/dataflow/docs/concepts/dataflow-templates>__` in the Google Cloud documentation.
   ```




----------------------------------------------------------------
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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r553579612



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline
+code from a source file (Java or Python) it would be best to use the language specific create operators.
+If a process exists to stage the pipeline code in an abstracted manner - a Templated job would be best as
+it allows development of the application without minimal intrusion to the DAG containing operators for it.
+It is also possible to run jobs defined in SQL language.
+
+Starting a new job
+^^^^^^^^^^^^^^^^^^
+
+To create a new pipeline using the source file (JAR in Java or Python file) use
+the create job operators. The source file can be located on GCS or on the local filesystem.
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+or
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+
+Please see the notes below on Java and Python specific SDKs as they each have their own set
+of execution options when running pipelines.
+
+Language specific pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Based on which language (SDK) is used for the Dataflow operators, there are specific options to be wary of.
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Java SDK pipelines
+""""""""""""""""""
+
+The ``jar`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+as it contains the pipeline to be executed on Dataflow. The JAR can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_jar_on_gcs]
+    :end-before: [END howto_operator_start_java_job_jar_on_gcs]
+
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_local_jar]
+    :end-before: [END howto_operator_start_java_job_local_jar]
+
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+Python SDK pipelines
+""""""""""""""""""""
+
+The ``py_file`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+as it contains the pipeline to be executed on Dataflow. The Python file can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+The ``py_interpreter`` argument specifies the Python version to be used when executing the pipeline, the default
+is ``python3`. If your Airflow instance is running on Python 2 - specify ``python2`` and ensure your ``py_file`` is
+in Python 2. For best results, use Python 3.
+
+If ``py_requirements`` argument is specified a temporary Python virtual environment with specified requirements will be created
+and within it pipeline will run.
+
+The ``py_system_site_packages`` argument specifies whether or not all the Python packages from your Airflow instance,
+will be accessible within virtual environment (if ``py_requirements`` argument is specified),
+recommend avoiding unless the Dataflow job requires it.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_python_job]
+    :end-before: [END howto_operator_start_python_job]
+
+
+Execution options for pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^

Review comment:
       ```suggestion
   Execution models
   ^^^^^^^^^^^^^^^^
   ```




----------------------------------------------------------------
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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r553580575



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline
+code from a source file (Java or Python) it would be best to use the language specific create operators.
+If a process exists to stage the pipeline code in an abstracted manner - a Templated job would be best as
+it allows development of the application without minimal intrusion to the DAG containing operators for it.
+It is also possible to run jobs defined in SQL language.
+
+Starting a new job
+^^^^^^^^^^^^^^^^^^
+
+To create a new pipeline using the source file (JAR in Java or Python file) use
+the create job operators. The source file can be located on GCS or on the local filesystem.
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+or
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+
+Please see the notes below on Java and Python specific SDKs as they each have their own set
+of execution options when running pipelines.
+
+Language specific pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Based on which language (SDK) is used for the Dataflow operators, there are specific options to be wary of.
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Java SDK pipelines
+""""""""""""""""""
+
+The ``jar`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+as it contains the pipeline to be executed on Dataflow. The JAR can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_jar_on_gcs]
+    :end-before: [END howto_operator_start_java_job_jar_on_gcs]
+
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_local_jar]
+    :end-before: [END howto_operator_start_java_job_local_jar]
+
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+Python SDK pipelines
+""""""""""""""""""""
+
+The ``py_file`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+as it contains the pipeline to be executed on Dataflow. The Python file can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+The ``py_interpreter`` argument specifies the Python version to be used when executing the pipeline, the default
+is ``python3`. If your Airflow instance is running on Python 2 - specify ``python2`` and ensure your ``py_file`` is
+in Python 2. For best results, use Python 3.
+
+If ``py_requirements`` argument is specified a temporary Python virtual environment with specified requirements will be created
+and within it pipeline will run.
+
+The ``py_system_site_packages`` argument specifies whether or not all the Python packages from your Airflow instance,
+will be accessible within virtual environment (if ``py_requirements`` argument is specified),
+recommend avoiding unless the Dataflow job requires it.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_python_job]
+    :end-before: [END howto_operator_start_python_job]
+
+
+Execution options for pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Dataflow has multiple options of executing pipelines. It can be done in the following modes:
+batch asynchronously (fire and forget), batch blocking (wait until completion), or streaming (run indefinitely).

Review comment:
       Personally, I wouldn't consider streaming as another execution model.




----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming processing.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are several ways to run a Dataflow pipeline depending on your environment, source files:
+
+- **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker
+  if you have a '*.jar' file for Java or a '* .py` file for Python. This also means that the necessary system
+  dependencies must be installed on the worker.  For Java, worker must have the JRE Runtime installed.
+  For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions.
+  This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies,
+  it often causes problems. See:
+  :ref:`howto/operator:DataflowCreateJavaJobOperator`, :ref:`howto/operator:DataflowCreatePythonJobOperator`
+- **Templated pipeline**: The programmer can make the pipeline independent of the environment by preparing
+  a template that will then be run on a machine managed by Google. This way, changes to the environment
+  won't affect your pipeline. There are two types of the templates:
+
+  - **Classic templates**. Developers run the pipeline and create a template. The Apache Beam SDK stages
+    files in Cloud Storage, creates a template file (similar to job request),
+    and saves the template file in Cloud Storage. See: :ref:`howto/operator:DataflowTemplatedJobStartOperator`
+  - **Flex Templates**. Developers package the pipeline into a Docker image and then use the ``gcloud``

Review comment:
       @mik-laj  is right. Only `DataflowStartSqlJobOperator` requires `gcloud`.
   
   I added warning in `DataflowStartSqlJobOperator` section and operator itself about required `gcloud` SDK




----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming processing.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are several ways to run a Dataflow pipeline depending on your environment, source files:
+
+- **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker

Review comment:
       You are right. I changed to "airflow 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] mik-laj commented on pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#issuecomment-756467767


   > Does airflow community has a reviewer to help with consistent documentation?
   
   The community doesn't have any technical writer. We rely only on the contributions of other people and they are mostly developers.


----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/477627230) 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 #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline

Review comment:
       I slightly rephrased it from : "it often causes problems." to "  it may cause problems."




----------------------------------------------------------------
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] mik-laj merged pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj merged pull request #13461:
URL: https://github.com/apache/airflow/pull/13461


   


----------------------------------------------------------------
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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r553579311



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline
+code from a source file (Java or Python) it would be best to use the language specific create operators.
+If a process exists to stage the pipeline code in an abstracted manner - a Templated job would be best as
+it allows development of the application without minimal intrusion to the DAG containing operators for it.
+It is also possible to run jobs defined in SQL language.
+
+Starting a new job
+^^^^^^^^^^^^^^^^^^
+
+To create a new pipeline using the source file (JAR in Java or Python file) use
+the create job operators. The source file can be located on GCS or on the local filesystem.
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+or
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+
+Please see the notes below on Java and Python specific SDKs as they each have their own set
+of execution options when running pipelines.
+
+Language specific pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Based on which language (SDK) is used for the Dataflow operators, there are specific options to be wary of.
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Java SDK pipelines
+""""""""""""""""""
+
+The ``jar`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+as it contains the pipeline to be executed on Dataflow. The JAR can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_jar_on_gcs]
+    :end-before: [END howto_operator_start_java_job_jar_on_gcs]
+
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_local_jar]
+    :end-before: [END howto_operator_start_java_job_local_jar]
+
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+Python SDK pipelines
+""""""""""""""""""""
+
+The ``py_file`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+as it contains the pipeline to be executed on Dataflow. The Python file can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+The ``py_interpreter`` argument specifies the Python version to be used when executing the pipeline, the default
+is ``python3`. If your Airflow instance is running on Python 2 - specify ``python2`` and ensure your ``py_file`` is
+in Python 2. For best results, use Python 3.
+
+If ``py_requirements`` argument is specified a temporary Python virtual environment with specified requirements will be created
+and within it pipeline will run.
+
+The ``py_system_site_packages`` argument specifies whether or not all the Python packages from your Airflow instance,
+will be accessible within virtual environment (if ``py_requirements`` argument is specified),
+recommend avoiding unless the Dataflow job requires it.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_python_job]
+    :end-before: [END howto_operator_start_python_job]
+
+
+Execution options for pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Dataflow has multiple options of executing pipelines. It can be done in the following modes:
+batch asynchronously (fire and forget), batch blocking (wait until completion), or streaming (run indefinitely).
+In Airflow it is best practice to use asynchronous batch pipelines or streams and use sensors to listen for expected job state.
+
+By default :class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`

Review comment:
       And what is the behavior of the DataflowStartFlexTemplateOperator? It seems to me that these sections need to be generalized a little in order to describe the general assumptions and only then describe the specific cases. I suspect that a well-written section will allow us to delete subsections.




----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline
+code from a source file (Java or Python) it would be best to use the language specific create operators.
+If a process exists to stage the pipeline code in an abstracted manner - a Templated job would be best as
+it allows development of the application without minimal intrusion to the DAG containing operators for it.
+It is also possible to run jobs defined in SQL language.
+
+Starting a new job
+^^^^^^^^^^^^^^^^^^
+
+To create a new pipeline using the source file (JAR in Java or Python file) use
+the create job operators. The source file can be located on GCS or on the local filesystem.
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+or
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+
+Please see the notes below on Java and Python specific SDKs as they each have their own set
+of execution options when running pipelines.
+
+Language specific pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Based on which language (SDK) is used for the Dataflow operators, there are specific options to be wary of.
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Java SDK pipelines
+""""""""""""""""""
+
+The ``jar`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+as it contains the pipeline to be executed on Dataflow. The JAR can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_jar_on_gcs]
+    :end-before: [END howto_operator_start_java_job_jar_on_gcs]
+
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_local_jar]
+    :end-before: [END howto_operator_start_java_job_local_jar]
+
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+Python SDK pipelines
+""""""""""""""""""""
+
+The ``py_file`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+as it contains the pipeline to be executed on Dataflow. The Python file can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+The ``py_interpreter`` argument specifies the Python version to be used when executing the pipeline, the default
+is ``python3`. If your Airflow instance is running on Python 2 - specify ``python2`` and ensure your ``py_file`` is
+in Python 2. For best results, use Python 3.
+
+If ``py_requirements`` argument is specified a temporary Python virtual environment with specified requirements will be created
+and within it pipeline will run.
+
+The ``py_system_site_packages`` argument specifies whether or not all the Python packages from your Airflow instance,
+will be accessible within virtual environment (if ``py_requirements`` argument is specified),
+recommend avoiding unless the Dataflow job requires it.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_python_job]
+    :end-before: [END howto_operator_start_python_job]
+
+
+Execution options for pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Dataflow has multiple options of executing pipelines. It can be done in the following modes:
+batch asynchronously (fire and forget), batch blocking (wait until completion), or streaming (run indefinitely).
+In Airflow it is best practice to use asynchronous batch pipelines or streams and use sensors to listen for expected job state.
+
+By default :class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`

Review comment:
       I refactored it and added description how it works with templates operator

##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline
+code from a source file (Java or Python) it would be best to use the language specific create operators.
+If a process exists to stage the pipeline code in an abstracted manner - a Templated job would be best as
+it allows development of the application without minimal intrusion to the DAG containing operators for it.
+It is also possible to run jobs defined in SQL language.
+
+Starting a new job
+^^^^^^^^^^^^^^^^^^
+
+To create a new pipeline using the source file (JAR in Java or Python file) use
+the create job operators. The source file can be located on GCS or on the local filesystem.
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+or
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+
+Please see the notes below on Java and Python specific SDKs as they each have their own set
+of execution options when running pipelines.
+
+Language specific pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Based on which language (SDK) is used for the Dataflow operators, there are specific options to be wary of.
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Java SDK pipelines
+""""""""""""""""""
+
+The ``jar`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`
+as it contains the pipeline to be executed on Dataflow. The JAR can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_jar_on_gcs]
+    :end-before: [END howto_operator_start_java_job_jar_on_gcs]
+
+
+Here is an example of creating and running a pipeline in Java with jar stored on GCS:
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job_local_jar]
+    :end-before: [END howto_operator_start_java_job_local_jar]
+
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+Python SDK pipelines
+""""""""""""""""""""
+
+The ``py_file`` argument must be specified for
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`
+as it contains the pipeline to be executed on Dataflow. The Python file can be available on GCS that Airflow
+has the ability to download or available on the local filesystem (provide the absolute path to it).
+
+The ``py_interpreter`` argument specifies the Python version to be used when executing the pipeline, the default
+is ``python3`. If your Airflow instance is running on Python 2 - specify ``python2`` and ensure your ``py_file`` is
+in Python 2. For best results, use Python 3.
+
+If ``py_requirements`` argument is specified a temporary Python virtual environment with specified requirements will be created
+and within it pipeline will run.
+
+The ``py_system_site_packages`` argument specifies whether or not all the Python packages from your Airflow instance,
+will be accessible within virtual environment (if ``py_requirements`` argument is specified),
+recommend avoiding unless the Dataflow job requires it.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_python_job]
+    :end-before: [END howto_operator_start_python_job]
+
+
+Execution options for pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Dataflow has multiple options of executing pipelines. It can be done in the following modes:
+batch asynchronously (fire and forget), batch blocking (wait until completion), or streaming (run indefinitely).
+In Airflow it is best practice to use asynchronous batch pipelines or streams and use sensors to listen for expected job state.
+
+By default :class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`

Review comment:
       I refactored it and added description how it works with templates 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] aaltay commented on a change in pull request #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.

Review comment:
       Maybe "for both batch and streaming processing." instead of "for both batch and streaming" ?

##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline

Review comment:
       I like the expanded version.
   
   "This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies, it often causes problems. " is very strongly worded. We could maybe add more explanation on how to manage dependency problems instead. Otherwise this will prevent most users from trying this option. And I think the airflow operator has been developed significantly overtime and allows managing dependencies. So I am guessing this is more of a documentation problem than a problem with the operator itself.
   
   




----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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


   I made some changes. PTAL @mik-laj @aaltay 


----------------------------------------------------------------
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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r553372275



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -0,0 +1,274 @@
+ .. 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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are multiple options to execute a Dataflow pipeline on Airflow. If looking to execute the pipeline

Review comment:
       I would like to develop it, because it is very problematic for users.
   
   ```
   There are several ways to run a Dataflow pipeline depending on your environment, source files:
   - **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker if you have a '*.jar' file for Java or a '* .py` file for Python. This also means that the necessary system dependencies must be installed on the worker.  For Java, worker must have the JRE Runtime installed. For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions. This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies, it often causes problems. 
   - **Templated pipeline**: The programmer can make the pipeline independent of the environment by preparing a template that will then be run on a machine managed by Google. This way, changes to the environment won't affect your pipeline. There are two types of the templates:
        - **Classic templates**. Developers run the pipeline and create a template. The Apache Beam SDK stages files in Cloud Storage, creates a template file (similar to job request), and saves the template file in Cloud Storage.
       - **Flex Templates**. Developers package the pipeline into a Docker image and then use the `gcloud` command-line tool to build and save the Flex Template spec file in Cloud Storage. 
   - **SQL pipeline**: Developer can write pipeline as SQL statement and then execute it in Dataflow.
   
   It is a good idea to test your pipeline using the non-templated pipeline, and then run the pipeline in production using the templates.
   
   For details on the differences between the pipeline types, see `Dataflow templates <https://cloud.google.com/dataflow/docs/concepts/dataflow-templates>__` in the Google Cloud documentation.
   ```




----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming processing.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are several ways to run a Dataflow pipeline depending on your environment, source files:
+
+- **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker

Review comment:
       to clarify, worker is an "airflow worker" not a "dataflow worker" in this case right?

##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming processing.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are several ways to run a Dataflow pipeline depending on your environment, source files:
+
+- **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker
+  if you have a '*.jar' file for Java or a '* .py` file for Python. This also means that the necessary system
+  dependencies must be installed on the worker.  For Java, worker must have the JRE Runtime installed.
+  For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions.
+  This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies,
+  it often causes problems. See:
+  :ref:`howto/operator:DataflowCreateJavaJobOperator`, :ref:`howto/operator:DataflowCreatePythonJobOperator`
+- **Templated pipeline**: The programmer can make the pipeline independent of the environment by preparing
+  a template that will then be run on a machine managed by Google. This way, changes to the environment
+  won't affect your pipeline. There are two types of the templates:
+
+  - **Classic templates**. Developers run the pipeline and create a template. The Apache Beam SDK stages
+    files in Cloud Storage, creates a template file (similar to job request),
+    and saves the template file in Cloud Storage. See: :ref:`howto/operator:DataflowTemplatedJobStartOperator`
+  - **Flex Templates**. Developers package the pipeline into a Docker image and then use the ``gcloud``

Review comment:
       Would this not require "gcloud" as a depedency pre-installed in the airflow worker nodes? (similar to the JRE or python requirements above)




----------------------------------------------------------------
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] mik-laj merged pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj merged pull request #13461:
URL: https://github.com/apache/airflow/pull/13461


   


----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/480120033) 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] mik-laj commented on a change in pull request #13461: Add How To Guide for Dataflow

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13461:
URL: https://github.com/apache/airflow/pull/13461#discussion_r555605916



##########
File path: docs/apache-airflow-providers-google/operators/cloud/dataflow.rst
##########
@@ -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.
+
+Google Cloud Dataflow Operators
+===============================
+
+`Dataflow <https://cloud.google.com/dataflow/>`__ is a managed service for
+executing a wide variety of data processing patterns. These pipelines are created
+using the Apache Beam programming model which allows for both batch and streaming processing.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+Ways to run a data pipeline
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+There are several ways to run a Dataflow pipeline depending on your environment, source files:
+
+- **Non-templated pipeline**: Developer can run the pipeline as a local process on the worker
+  if you have a '*.jar' file for Java or a '* .py` file for Python. This also means that the necessary system
+  dependencies must be installed on the worker.  For Java, worker must have the JRE Runtime installed.
+  For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions.
+  This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies,
+  it often causes problems. See:
+  :ref:`howto/operator:DataflowCreateJavaJobOperator`, :ref:`howto/operator:DataflowCreatePythonJobOperator`
+- **Templated pipeline**: The programmer can make the pipeline independent of the environment by preparing
+  a template that will then be run on a machine managed by Google. This way, changes to the environment
+  won't affect your pipeline. There are two types of the templates:
+
+  - **Classic templates**. Developers run the pipeline and create a template. The Apache Beam SDK stages
+    files in Cloud Storage, creates a template file (similar to job request),
+    and saves the template file in Cloud Storage. See: :ref:`howto/operator:DataflowTemplatedJobStartOperator`
+  - **Flex Templates**. Developers package the pipeline into a Docker image and then use the ``gcloud``

Review comment:
       It seems to me that only SQL Operator requires Google Cloud CLI to be installed.




----------------------------------------------------------------
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 #13461: Add How To Guide for Dataflow

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


   cc @tanjinP @mik-laj @aaltay 


----------------------------------------------------------------
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