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/08/18 11:56:49 UTC

[GitHub] [airflow] Craig-Chatfield opened a new pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Craig-Chatfield opened a new pull request #10373:
URL: https://github.com/apache/airflow/pull/10373


   ### Bug Fix
   
   Currently `py_requirements` defaults to an empty list in the Dataflow operator if no value is passed into the function call. As such the check in the hook would always evaluate to True - running the Dataflow commands inside a virtualenv.
   
   This PR corrects the evaluation behaviour of the `if` statement so that an empty list will evaluate as `false`.
   
   ### UnitTests
   
   As part of this PR two unit tests have been introduced, for testing the calls with an empty/non-empty `py_requirements`.
   
   closes: #ISSUE
   
   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 edited a comment on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #10373:
URL: https://github.com/apache/airflow/pull/10373#issuecomment-676394674


   @Craig-Chatfield I see that you work for King, and I know that your company uses Cloud Composer and GCP services very intensively. Do you have any gaps in integration between Airflow and GCP? I am open to feedback from you and your team as I would like to develop this.
   
   PS We may have seen each other at [Airflow Meetup](https://www.meetup.com/pl-PL/London-Apache-Airflow-Meetup/events/264521028/) in London ;-) 


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

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



[GitHub] [airflow] potiuk commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   @mik-laj - has it been released in Backport packages? If so, they need a note in Updating.


----------------------------------------------------------------
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 #10373: Dataflow operators don't not always create a virtualenv

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


   


----------------------------------------------------------------
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 edited a comment on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #10373:
URL: https://github.com/apache/airflow/pull/10373#issuecomment-675481048


   > @mik-laj - has it been released already in Backports?
   
   Yes. This was released in backport packages, but we don't have UPDATING.md for backport packages  yet. In the UPDATING.md file we have breaking changes between Airflow versions only.


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

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



[GitHub] [airflow] potiuk commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   > > @mik-laj - has it been released already in Backports?
   
   OK. Then I will make sure to add support for that when preparing backport releases 2nd wave. Added appropriate issues linked to #10014


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   @potiuk Support for virtual environments has not been released in any release.


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

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



[GitHub] [airflow] potiuk commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   > @potiuk I don't think this needs a note in UPDATING.md because it is a regression to the previous version. Now, when we fix this bug, we will restore the original behavior of this operator. Take a look at this ticket: [#10374 (comment)](https://github.com/apache/airflow/issues/10374#issuecomment-675473990)
   
   Ah - so that's a bugfix. Then good :)


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   ```python
       start_python_job = DataflowCreatePythonJobOperator(
           task_id="start-python-job",
           py_file=GCS_PYTHON,
           py_options=[],
           job_name='{{task.task_id}}',
           options={
               'output': GCS_OUTPUT,
           },
           py_requirements=None,
           py_interpreter='python3',
           py_system_site_packages=False,
           location='europe-west3'
       )
   ```
   This should not create a virtual environment since py_requirements is set to None.
   
   ```python
       start_python_job = DataflowCreatePythonJobOperator(
           task_id="start-python-job",
           py_file=GCS_PYTHON,
           py_options=[],
           job_name='{{task.task_id}}',
           options={
               'output': GCS_OUTPUT,
           },
           py_requirements=[], # Enable virtuall environment
           py_interpreter='python3',
           py_system_site_packages=True, # set to True
           location='europe-west3'
       )
   ```
   This should not create a virtual environment since py_requirements is set to ``[]`` and ``py_system_site_packages`` is set to True, so we can use ``apache-beam`` from system site-packages?
   
   Can you confirm it is behaving this way now?


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

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



[GitHub] [airflow] mik-laj commented on pull request #10373: Dataflow operators don't not always create a virtualenv

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


   @Craig-Chatfield Thanks for this contribution. What's next? 


----------------------------------------------------------------
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] Craig-Chatfield commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Posted by GitBox <gi...@apache.org>.
Craig-Chatfield commented on pull request #10373:
URL: https://github.com/apache/airflow/pull/10373#issuecomment-675455994


   > I think it requires a short note in UPDATING.md but otherwise it is LGTM
   
   @potiuk Can you advise on what the change required should be?
   
   I also see two locations for the hook in `UPDATING.md`:
   https://github.com/apache/airflow/blob/master/UPDATING.md#airflowprovidersgooglecloudhooksdataflowdataflowhook
   https://github.com/apache/airflow/blob/master/UPDATING.md#airflowprovidersgooglecloudhooksdataflowdataflowhook-1
   
   > @potiuk Support for virtual environments has not been released in any release.
   
   @mik-laj Does that hinder this PR? I'm not sure what the implications are.


----------------------------------------------------------------
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 #10373: Dataflow operators don't not always create a virtualenv

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


   Awesome work, congrats on your first merged pull request!
   


----------------------------------------------------------------
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] Craig-Chatfield commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Posted by GitBox <gi...@apache.org>.
Craig-Chatfield commented on pull request #10373:
URL: https://github.com/apache/airflow/pull/10373#issuecomment-677514288


   @mik-laj Hey, yeah I'm just taking a look now


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

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



[GitHub] [airflow] mik-laj commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   @Craig-Chatfield  I prepared a minor change to improve the support for virtual environments with system packages. 
   ```diff
   From 7ac7b7a0d68abd7f5b5e06d28b4850252f81c679 Mon Sep 17 00:00:00 2001
   From: =?UTF-8?q?Kamil=20Bregu=C5=82a?= <ka...@polidea.com>
   Date: Wed, 19 Aug 2020 14:30:29 +0200
   Subject: Improve the support of virtual environments with system packages
   
   ---
    .../providers/google/cloud/hooks/dataflow.py  | 18 +++++++-
    .../google/cloud/operators/dataflow.py        |  4 +-
    .../google/cloud/hooks/test_dataflow.py       | 41 +++++++++++--------
    3 files changed, 41 insertions(+), 22 deletions(-)
   
   diff --git a/airflow/providers/google/cloud/hooks/dataflow.py b/airflow/providers/google/cloud/hooks/dataflow.py
   index 01996bc12..44dc87727 100644
   --- a/airflow/providers/google/cloud/hooks/dataflow.py
   +++ b/airflow/providers/google/cloud/hooks/dataflow.py
   @@ -24,6 +24,7 @@ import re
    import select
    import shlex
    import subprocess
   +import textwrap
    import time
    import uuid
    import warnings
   @@ -633,7 +634,7 @@ class DataflowHook(GoogleBaseHook):
            :param py_system_site_packages: Whether to include system_site_packages in your virtualenv.
                See virtualenv documentation for more information.
    
   -            This option is only relevant if the ``py_requirements`` parameter is passed.
   +            This option is only relevant if the ``py_requirements`` parameter is not None.
            :type py_interpreter: str
            :param append_job_name: True if unique suffix has to be appended to job name.
            :type append_job_name: bool
   @@ -652,7 +653,20 @@ class DataflowHook(GoogleBaseHook):
                return ['--labels={}={}'.format(key, value)
                        for key, value in labels_dict.items()]
    
   -        if py_requirements:
   +        if py_requirements is not None:
   +            if not py_requirements and not py_system_site_packages:
   +                warning_invalid_environment = textwrap.dedent(
   +                    """\
   +                    Invalid method invocation. You have disabled inclusion of system packages and empty list
   +                    required for installation, so it is not possible to create a valid virtual environment.
   +                    In the virtual environment, apache-beam package must be installed for your job to be \
   +                    executed. To fix this problem:
   +                    * install apache-beam on the system, then set parameter py_system_site_packages to True,
   +                    * add apache-beam to the list of required packages in parameter py_requirements.
   +                    """
   +                )
   +                raise AirflowException(warning_invalid_environment)
   +
                with TemporaryDirectory(prefix='dataflow-venv') as tmp_dir:
                    py_interpreter = prepare_virtualenv(
                        venv_directory=tmp_dir,
   diff --git a/airflow/providers/google/cloud/operators/dataflow.py b/airflow/providers/google/cloud/operators/dataflow.py
   index 85471e3c7..33adbacf4 100644
   --- a/airflow/providers/google/cloud/operators/dataflow.py
   +++ b/airflow/providers/google/cloud/operators/dataflow.py
   @@ -470,7 +470,7 @@ class DataflowCreatePythonJobOperator(BaseOperator):
        :param py_system_site_packages: Whether to include system_site_packages in your virtualenv.
            See virtualenv documentation for more information.
    
   -        This option is only relevant if the ``py_requirements`` parameter is passed.
   +        This option is only relevant if the ``py_requirements`` parameter is not None.
        :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
        :type gcp_conn_id: str
        :param project_id: Optional, the GCP project ID in which to start a job.
   @@ -517,7 +517,7 @@ class DataflowCreatePythonJobOperator(BaseOperator):
            self.options.setdefault('labels', {}).update(
                {'airflow-version': 'v' + version.replace('.', '-').replace('+', '-')})
            self.py_interpreter = py_interpreter
   -        self.py_requirements = py_requirements or []
   +        self.py_requirements = py_requirements
            self.py_system_site_packages = py_system_site_packages
            self.project_id = project_id
            self.location = location
   diff --git a/tests/providers/google/cloud/hooks/test_dataflow.py b/tests/providers/google/cloud/hooks/test_dataflow.py
   index 48b2191c1..50b323ae4 100644
   --- a/tests/providers/google/cloud/hooks/test_dataflow.py
   +++ b/tests/providers/google/cloud/hooks/test_dataflow.py
   @@ -314,13 +314,25 @@ class TestDataflowHook(unittest.TestCase):
            self.assertListEqual(sorted(mock_dataflow.call_args[1]["cmd"]),
                                 sorted(expected_cmd))
    
   +    @parameterized.expand([
   +        (['foo-bar'], False),
   +        (['foo-bar'], True),
   +        ([], True),
   +    ])
        @mock.patch(DATAFLOW_STRING.format('prepare_virtualenv'))
        @mock.patch(DATAFLOW_STRING.format('uuid.uuid4'))
        @mock.patch(DATAFLOW_STRING.format('_DataflowJobsController'))
        @mock.patch(DATAFLOW_STRING.format('_DataflowRunner'))
        @mock.patch(DATAFLOW_STRING.format('DataflowHook.get_conn'))
   -    def test_start_python_dataflow_with_non_empty_py_requirements(
   -        self, mock_conn, mock_dataflow, mock_dataflowjob, mock_uuid, mock_virtualenv
   +    def test_start_python_dataflow_with_non_empty_py_requirements_and_without_system_packages(
   +        self,
   +        current_py_requirements,
   +        current_py_system_site_packages,
   +        mock_conn,
   +        mock_dataflow,
   +        mock_dataflowjob,
   +        mock_uuid,
   +        mock_virtualenv,
        ):
            mock_uuid.return_value = MOCK_UUID
            mock_conn.return_value = None
   @@ -332,7 +344,8 @@ class TestDataflowHook(unittest.TestCase):
            self.dataflow_hook.start_python_dataflow(  # pylint: disable=no-value-for-parameter
                job_name=JOB_NAME, variables=DATAFLOW_VARIABLES_PY,
                dataflow=PY_FILE, py_options=PY_OPTIONS,
   -            py_requirements=['foo-bar']
   +            py_requirements=current_py_requirements,
   +            py_system_site_packages=current_py_system_site_packages
            )
            expected_cmd = ['/dummy_dir/bin/python', '-m', PY_FILE,
                            '--region=us-central1',
   @@ -347,7 +360,7 @@ class TestDataflowHook(unittest.TestCase):
        @mock.patch(DATAFLOW_STRING.format('_DataflowJobsController'))
        @mock.patch(DATAFLOW_STRING.format('_DataflowRunner'))
        @mock.patch(DATAFLOW_STRING.format('DataflowHook.get_conn'))
   -    def test_start_python_dataflow_with_empty_py_requirements(
   +    def test_start_python_dataflow_with_empty_py_requirements_and_without_system_packages(
            self, mock_conn, mock_dataflow, mock_dataflowjob, mock_uuid
        ):
            mock_uuid.return_value = MOCK_UUID
   @@ -356,20 +369,12 @@ class TestDataflowHook(unittest.TestCase):
            dataflow_instance.wait_for_done.return_value = None
            dataflowjob_instance = mock_dataflowjob.return_value
            dataflowjob_instance.wait_for_done.return_value = None
   -        self.dataflow_hook.start_python_dataflow(  # pylint: disable=no-value-for-parameter
   -            job_name=JOB_NAME, variables=DATAFLOW_VARIABLES_PY,
   -            dataflow=PY_FILE, py_options=PY_OPTIONS,
   -            py_requirements=[]
   -        )
   -        expected_cmd = ['python3', '-m', PY_FILE,
   -                        '--region=us-central1',
   -                        '--runner=DataflowRunner',
   -                        '--project=test',
   -                        '--labels=foo=bar',
   -                        '--staging_location=gs://test/staging',
   -                        '--job_name={}-{}'.format(JOB_NAME, MOCK_UUID)]
   -        self.assertListEqual(sorted(mock_dataflow.call_args[1]["cmd"]),
   -                             sorted(expected_cmd))
   +        with self.assertRaisesRegex(AirflowException, "Invalid method invocation."):
   +            self.dataflow_hook.start_python_dataflow(  # pylint: disable=no-value-for-parameter
   +                job_name=JOB_NAME, variables=DATAFLOW_VARIABLES_PY,
   +                dataflow=PY_FILE, py_options=PY_OPTIONS,
   +                py_requirements=[]
   +            )
    
        @mock.patch(DATAFLOW_STRING.format('uuid.uuid4'))
        @mock.patch(DATAFLOW_STRING.format('_DataflowJobsController'))
   -- 
   2.28.0
   
   ```
   You can add this change with the command below.
   ```
   curl https://termbin.com/75yd | git am
   ```
   What do you think about it?
   
   PS. Today I got the same submission from one Google engineer so would like to resolve this ticket quickly.
   


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   @Craig-Chatfield I see that you work for King, and I know that your company uses Cloud Composer and GCP services very intensively. Do you have any gaps in integration between Airflow and GCP? I am open to feedback and I think that I can even help with its implementation.
   
   PS We may have seen each other at [Airflow Meetup](https://www.meetup.com/pl-PL/London-Apache-Airflow-Meetup/events/264521028/) in London ;-) 


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

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



[GitHub] [airflow] potiuk edited a comment on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   > @potiuk I don't think this needs a note in UPDATING.md because it is a regression to the previous version. Now, when we fix this bug, we will restore the original behavior of this operator. Take a look at this ticket: [#10374 (comment)](https://github.com/apache/airflow/issues/10374#issuecomment-675473990)
   
   Ah - so that's a bugfix. Then good :)
   
   Still I think such mechanism is useful - i will review all changes since the last backport packages and see if we have any of that. If we plan (we do) to split 2.0 into separate packages for release, we need to have such mechanism in place per-provider anyway.


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   @potiuk I don't think this needs a note in UPDATING.md because it is a regression to the previous version. Now, when we fix this bug, we will restore the original behavior of this operator. Take a look at this ticket: https://github.com/apache/airflow/issues/10374#issuecomment-675473990


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   If anyone asks, I'm responsible for maintaining the Dataflow integration and now I'm looking at this ticket.


----------------------------------------------------------------
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 edited a comment on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #10373:
URL: https://github.com/apache/airflow/pull/10373#issuecomment-676394674


   @Craig-Chatfield I see that you work for King, and I know that your company uses Cloud Composer and GCP services very intensively. Do you have any gaps in integration between Airflow and GCP? I am open to feedback from you and your team as I would like to develop this.
   Email: kamil.bregula@pollidea.com 
   Slack: @mik-laj  ([![Slack Status](https://img.shields.io/badge/slack-join_chat-white.svg?logo=slack&style=social)](https://s.apache.org/airflow-slack))
   
   PS We may have seen each other at [Airflow Meetup](https://www.meetup.com/pl-PL/London-Apache-Airflow-Meetup/events/264521028/) in London ;-) 


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   > @mik-laj - has it been released already in Backports?
   
   Yes. This was released in backport packages, but we don't yet have UPDATING.md for backport packages. In the UPDATING.md file we have breaking changes between Airflow versions only.


----------------------------------------------------------------
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] Craig-Chatfield commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Posted by GitBox <gi...@apache.org>.
Craig-Chatfield commented on pull request #10373:
URL: https://github.com/apache/airflow/pull/10373#issuecomment-676365977


   @mik-laj Looks good. I have updated my pull request accordingly. 


----------------------------------------------------------------
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 edited a comment on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #10373:
URL: https://github.com/apache/airflow/pull/10373#issuecomment-676394674


   @Craig-Chatfield I see that you work for King, and I know that your company uses Cloud Composer and GCP services very intensively. Do you have any gaps in integration between Airflow and GCP? I am open to feedback from you and your team as I would like to develop this.
   Email: kamil.bregula@pollidea.com ([![Slack Status](https://img.shields.io/badge/slack-join_chat-white.svg?logo=slack&style=social)](https://s.apache.org/airflow-slack))
   Slack: @mik-laj 
   
   PS We may have seen each other at [Airflow Meetup](https://www.meetup.com/pl-PL/London-Apache-Airflow-Meetup/events/264521028/) in London ;-) 


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   @Craig-Chatfield  Some tests failed. Can you fix 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] potiuk commented on pull request #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   @Craig-Chatfield - the reason for UPDATING.md is to give the users an information about potential backwards-incompatible changes in behaviours that were introduced vs. code that was released officially before in earlier releases.
   
   We have two mechanisms for releases now:
   
   1) Releasing Airflow 1.10.x releases (from the v1-10-test branch)
   2) Releasing Backport Packages (we plan to release the 2nd wave in #10014) 
   
   So if we released it before, we should make a note about it. However it's the second time we release Backport Packages (soon) and we have indeed no mechanism to capture any of those changes. I will introduce something soon so indeed if we have not released in 1.10 and we released it in the previous backport packages, we need to add some mechanism for that. 
   
   So - in short - do not worry for now @Craig-Chatfield. I think it can be merged as is and I will take a close look when releasing the backport packages.. 
   
   @mik-laj - has it been released already in Backports?
   


----------------------------------------------------------------
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 #10373: fix dataflow hook behaviour to not always launch a virtualenv

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


   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.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - 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