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/05/10 23:44:58 UTC

[GitHub] [airflow] tanjinP opened a new pull request #8809: Airflow 6294 create guide for dataflow operators

tanjinP opened a new pull request #8809:
URL: https://github.com/apache/airflow/pull/8809


   Closes [AIRFLOW-6294](https://issues.apache.org/jira/browse/AIRFLOW-6294) / issue #8202 .
   
   Contains a guide that is appropriately linked to on how to use the various operators in Dataflow.
   
   Leveraged #8276 heavily - thank you @alexandraabbas for the contribution! 🙂 
   
   ## Qs for committer/issue creator (@mik-laj)
   - In the issue the [asynchronous execution component was mentioned](https://github.com/apache/airflow/issues/8202#issuecomment-626277797) and to specify it in the docs; can you elaborate? 
     - This will most likely be a follow-up commit(s) because I did not mention this at all in this initial version if the PR
   - I tried to be thorough but I may have missed some better kinds of example use cases, please call them out if you can think of any.
   - General arrangement is: Java, Python, Templated. Did this categorization based on the operators available - please let me know if that is intuitive or if you have other ideas.
   
   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [x] Target Github ISSUE in description if exists
   - [x] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [x] Relevant documentation is updated including usage instructions.
   - [x] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   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).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more information.
   


----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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



##########
File path: airflow/providers/google/cloud/example_dags/example_dataflow.py
##########
@@ -108,9 +114,17 @@
     )
     # [END howto_operator_start_python_job]
 
+    # [START howto_operator_start_local_python_job]
+    py_to_local = GCSToLocalOperator(

Review comment:
       This should not be found in the guide. This operator is part of DAG for testing only. We want to test if a local file and a GCS file can be run.




----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   In my opinion, we need to add some information.
   * new section which will describe the ways to run a pipeline. Currently, the pipeline can be started using a local running executable (DataflowCreateJavaJobOperator, DataflowCreatePythonJobOperator) or Dataflow Template (DataflowTemplatedJobStartOperator) or via KubernetesPodOperator). A description that describes the differences between these methods would be useful.
   * A new section that will describe the differences between asynchronic and blocking execution modes. You should not use blocking pipelines, because it causes a background process to run, which supervises the execution of the job, This is not positive for two reasons - increase consumption of resources and prevents Airflow supervision over the job
   * information that Python operators allow you to specify the Apache Beam version, but if no version is specified, the local version from the environment will be used
   
   


----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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



##########
File path: docs/howto/operator/gcp/dataflow.rst
##########
@@ -0,0 +1,120 @@
+ .. 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.
+
+Airflow provides operators to start/create Java and Python Dataflow jobs.

Review comment:
       ```suggestion
   ```
   It is not necessary. In the guides we write, we try to create lists that we then need to update. If a new operator is added, it is very unlikely that someone will read the entire file and update this sentence. It also doesn't have much value for the end user.




----------------------------------------------------------------
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] tanjinP commented on pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   > Oh no. I hope we can finish it.
   
   Yikes - forgot about this. Will come back and update relevant components of this PR this evening - including the system tests.


----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   Sensors and non blocking jobs execution were added (https://github.com/apache/airflow/pull/11726, https://github.com/apache/airflow/pull/12249). It may be worth to mention 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] mik-laj commented on pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   Here is the system test. https://github.com/apache/airflow/blob/master/tests/providers/google/cloud/operators/test_dataflow_system.py
   I spent the last week fixing it because we deleted Jave in Breeze and Dataflow requires Java. I also wanted to adapt these integrations to the new version of Apache Beam SDK. For now, we only support Apache Beam SDKK <1.19. 
   <img width="332" alt="Screenshot 2020-06-12 at 06 55 31" src="https://user-images.githubusercontent.com/12058428/84466469-bec7b200-ac79-11ea-93f7-f2f013082f81.png">
   This required automated JAR file building.
   
   


----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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



##########
File path: docs/howto/operator/gcp/dataflow.rst
##########
@@ -0,0 +1,180 @@
+ .. 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:: _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.
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+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.
+
+Here is an example of creating and running a pipeline in Java:
+
+.. exampleinclude:: ../../../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job]
+    :end-before: [END howto_operator_start_java_job]
+
+.. _howto/operator:DataflowTemplatedJobStartOperator:
+
+Templated jobs
+""""""""""""""
+
+Templates give the ability to stage a pipeline on Cloud Storage and run it from there. This
+provides flexibility in the development workflow as it separates the development of a pipeline
+from the staging and execution steps. To start a templated job use the
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowTemplatedJobStartOperator`
+
+.. exampleinclude:: ../../../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_template_job]
+    :end-before: [END howto_operator_start_template_job]
+
+See the `list of Google-provided templates that can be used with this operator
+<https://cloud.google.com/dataflow/docs/guides/templates/provided-templates>`_.
+
+Execution options for pipelines
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Dataflow has multiple options of executing pipelines. It can be done in the following modes:
+asynchronously (fire and forget), blocking (wait until completion), or streaming (run indefinitely).
+In Airflow it is best to use asynchronous pipelines as blocking ones tax the Airflow resources by listening
+to the job until it completes.
+
+Asynchronous execution
+""""""""""""""""""""""
+
+Dataflow jobs are by default asynchronous - however this is dependent on the application code (contained in the JAR
+or Python file) and how it is written. In order for the Dataflow job to execute asynchronously, ensure the
+pipeline objects are not being waited upon (not calling ``waitUntilFinish`` or ``wait_until_finish`` on the
+``PipelineResult`` in your application code).
+
+This is the recommended way to execute your pipelines when using Airflow.
+
+Use the Dataflow monitoring or command-line interface to view the details of your pipeline's results after Airflow
+runs the operator.
+
+Blocking execution

Review comment:
       Asynchronous execution was added in https://github.com/apache/airflow/pull/11726




----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   > I added a new example, would this break anything? I know for sure it will not run as some of the arguments were made up. I know the CI would catch this if anything is wrong, but those checks are still running at the time of this message.
   
   Our system tests are not automatically launched, but the reviewer should run the system tests manually before merging this change to fully verify that it is valid. We plan to run these CI tests in the near future, but this is not happening yet. In this case, it will be a problem because this dataflow will never end.  In this Q I would like to prepare operators that will stop such a jobs using Airflow. For now, you could add new step in tearDown method. To stop jobs, you can use `gcloud` command.


----------------------------------------------------------------
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] tanjinP commented on pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   @TobKed closing as you will take over this on your terms. As we discussed offline - thank you once again!


----------------------------------------------------------------
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] stale[bot] closed pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

Posted by GitBox <gi...@apache.org>.
stale[bot] closed pull request #8809:
URL: https://github.com/apache/airflow/pull/8809


   


----------------------------------------------------------------
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] tanjinP closed pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

Posted by GitBox <gi...@apache.org>.
tanjinP closed pull request #8809:
URL: https://github.com/apache/airflow/pull/8809


   


----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   Oh no. I hope we can finish 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] mik-laj commented on a change in pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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



##########
File path: docs/howto/operator/gcp/dataflow.rst
##########
@@ -0,0 +1,120 @@
+ .. 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.
+
+Airflow provides operators to start/create Java and Python Dataflow jobs.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include:: _partials/prerequisite_tasks.rst
+
+Create a Java Job
+^^^^^^^^^^^^^^^^^
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Using a JAR on GCS
+""""""""""""""""""
+
+To start a Java Cloud Dataflow job with the JAR on GCS use
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`.
+
+.. exampleinclude:: ../../../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_java_job]
+    :end-before: [END howto_operator_start_java_job]
+
+Using a JAR on local
+""""""""""""""""""""
+
+To start a Java Cloud Dataflow job with a JAR on local use
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator`.
+If the JAR is on GCS, you can download it to your local with
+:class:`~airflow.providers.google.cloud.operators.gcs.GCSToLocalOperator`.
+
+.. exampleinclude:: ../../../../airflow/providers/google/cloud/example_dags/example_dataflow.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_start_local_java_job]
+    :end-before: [END howto_operator_start_local_java_job]
+
+Create a Python Job
+^^^^^^^^^^^^^^^^^^^
+
+.. _howto/operator:DataflowCreatePythonJobOperator:
+
+Using a Python file on GCS
+""""""""""""""""""""""""""
+
+To start a Python Cloud Dataflow job with a Python file on GCS use
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`.
+
+.. 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]
+
+Using a Python file on Local
+""""""""""""""""""""""""""""
+
+To start a Python Cloud Dataflow job with a Python file on local use
+:class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator`.
+If the Python file is on GCS, you can download it to your local with

Review comment:
       You do not have to use this operator. If the file is on GCS then it can be specified using the parameter directly. We have this task at DAG, but only for testing purposes.




----------------------------------------------------------------
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] tanjinP commented on pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   > > I added a new example, would this break anything? I know for sure it will not run as some of the arguments were made up. I know the CI would catch this if anything is wrong, but those checks are still running at the time of this message.
   > 
   > Our system tests are not automatically launched, but the reviewer should run the system tests manually before merging this change to fully verify that it is valid. We plan to run these CI tests in the near future, but this is not happening yet. In this case, it will be a problem because this dataflow will never end. In this Q I would like to prepare operators that will stop such a jobs using Airflow. For now, you could add new step in tearDown method. To stop jobs, you can use `gcloud` command.
   
   @mik-laj 
   
   I’ve checked the [system tests guide](https://github.com/apache/airflow/blob/master/TESTING.rst#airflow-system-tests) and it looks like there aren’t any Dataflow ones based on the `git grep @pytest.mark.system("google.cloud”)` check I’ve done so I think this should be fine. Let me know if not.
   Anything else to look at for 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] mik-laj commented on a change in pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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



##########
File path: airflow/providers/google/cloud/example_dags/example_dataflow.py
##########
@@ -108,9 +114,16 @@
     )
     # [END howto_operator_start_python_job]
 
+    py_to_local = GCSToLocalOperator(
+        task_id="py-to-local",
+        bucket=GCS_PYTHON_BUCKET_NAME,
+        object_name=GCS_PYTHON_OBJECT_NAME,
+        filename="/tmp/dataflow-{{ ds_nodash }}.py",
+    )
+
     start_python_job_local = DataflowCreatePythonJobOperator(
         task_id="start-python-job-local",
-        py_file='apache_beam.examples.wordcount',
+        py_file='/tmp/dataflow-{{ ds_nodash }}.py',
         py_options=['-m'],

Review comment:
       If you use the -m option, you should specify the module, not the file name. 




----------------------------------------------------------------
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] boring-cyborg[bot] commented on pull request #8809: Airflow 6294 create guide for dataflow operators

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #8809:
URL: https://github.com/apache/airflow/pull/8809#issuecomment-626408155


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, pylint and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/master/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/master/docs/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/master/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better 🚀.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://apache-airflow-slack.herokuapp.com/
   


----------------------------------------------------------------
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 #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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



##########
File path: docs/howto/operator/gcp/dataflow.rst
##########
@@ -0,0 +1,120 @@
+ .. 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.
+
+Airflow provides operators to start/create Java and Python Dataflow jobs.
+
+.. contents::
+  :depth: 1
+  :local:
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include:: _partials/prerequisite_tasks.rst
+
+Create a Java Job
+^^^^^^^^^^^^^^^^^
+
+.. _howto/operator:DataflowCreateJavaJobOperator:
+
+Using a JAR on GCS

Review comment:
       Can you include it in one section? And then describe that the file should be indicated by the parameter. This can be the local file path or the GCS file path.  Then you can create two paragraphs with two examples.




----------------------------------------------------------------
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] stale[bot] commented on pull request #8809: [AIRFLOW-6294] Create guide for Dataflow operators

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


   This issue has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs. Thank you for your contributions.
   


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