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/01 09:25:37 UTC

[GitHub] [airflow] dstandish opened a new pull request #13423: WIP - add bash hook

dstandish opened a new pull request #13423:
URL: https://github.com/apache/airflow/pull/13423


   The bash operator has dialed in how to run bash commands from python and airflow.
   
    There are other operators that sometimes need to do this.  But for various reasons, they might not want to inherit from BashOperator.  And in this case, it would be nice if there were a bash hook that would just let you run a bash command in your operator.
   
   We can also imagine circumstances where perhaps within a single "BashOperator", maybe you want to run a _series_ of bash commands.  Well, if you have a hook that encapsulates this functionality, then this is very clean to implement.  
   
   WIP cus still need to do some testing, think about some choices, and add docs.


----------------------------------------------------------------
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 #13423: Add SubprocessHook for running commands from operators

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/537707977) 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] ashb commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: airflow/hooks/subprocess.py
##########
@@ -0,0 +1,90 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, List, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class SubprocessHook(BaseHook):
+    """Hook for running processes with the ``subprocess`` module"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(
+        self, command: List[str], env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'
+    ):
+        """
+        Execute the command in a temporary directory which will be cleaned afterwards
+
+        If ``env`` is not supplied, ``os.environ`` is passed
+
+        :param command: the command to run
+        :param env: Optional dict containing environment variables to be made available to the shell
+            environment in which ``command`` will be executed.  If omitted, ``os.environ`` will be used.
+        :param output_encoding: encoding to use for decoding stdout
+        :return: last line of stdout

Review comment:
       stderr is redirect to stdout I think so:
   
   ```suggestion
           :return: last line of output (on stderr or stdout)
   ```




----------------------------------------------------------------
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] dstandish commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   hey @turbaszek @ashb @kaxil 
   
   rebased after merge of [BashOperator to raise AirflowSkipException on exit code 127](https://github.com/apache/airflow/pull/13421)
   
   that change has implications for this one
   
   what i have done in this rebase is keep the skip logic in the `run_command` function
   
   this means that the skip logic would stay with the hook
   
   i feel a little iffy about this.  i think best if the skip logic is handled in the operator, that the hook should not have an opinion about this.
   
   so what i'm thinking is add `namedtuple('SubprocessResult', ['exit_code', 'output')` and then let the caller decide what to do.
   
   so in `run_command` it would not raise _any_ exception based on exit code -- it would just provide code along with output
   
   please let me know your thoughts.
   
   small note: i think will be nice to parameterize returning all output vs last line but this can be added in later 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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       I don't know where this should live, but I think the class name should be something like `CommandRunner`.
   
   I guess we could call this a Hook even if we don't use any connections -- the only other option I can think of is to have this under airflow.utils.




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       Oh almost nothing to do with bash :)




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       3.0 will be the next time we can remove anything (which will probably by 6-12 months timeframe -- likely 3.0 will be a "we've just removed deprecated things" rather than the huge release that 2.0 was)




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       @ashb @turbaszek are you sure these imports really need to be preserved?
   
   you could make the same argument about [these lines](https://github.com/apache/airflow/pull/13423/files#diff-912cc49726a411fd88f68b771c1de66996acd9b68c7e0e6b2e0caa7d8b23a188L21-L23).
   
   or any time you remove an import from a module
   
   users probably shouldn't be importing Popen from `bash_operator` anyway... WDYT?
   
   if you think these left in, with deprecation warning, please also comment with guidance on the same issue for `airflow.operators.bash`
   




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13423: Add bash hook

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



##########
File path: tests/hooks/test_bash.py
##########
@@ -0,0 +1,100 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.bash import BashHook
+
+OS_ENV_KEY = 'BASH_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestBashOperator(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the bash environment.
+        When ``env`` is not provided, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = BashHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=command, env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    def test_return_value(self):
+        hook = BashHook()
+        return_value = hook.run_command(command='echo "stdout"')
+        self.assertEqual(return_value, 'stdout')
+
+    def test_raise_exception_on_non_zero_exit_code(self):
+        hook = BashHook()
+        with self.assertRaisesRegex(
+            AirflowException, "Bash command failed\\. The command returned a non-zero exit code\\."
+        ):

Review comment:
       ```suggestion
           with pytest.raises(
               AirflowException, match="Bash command failed\\. The command returned a non-zero exit code\\."
           ):
   ```
   As we are going to be more pytest oriented




----------------------------------------------------------------
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] dstandish commented on pull request #13423: WIP - add bash hook

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


   > For few minutes I was considering making this hook stateless and keeping the state in operators but this adds unnecessary complexity.
   
   Yes @turbaszek, this is one of the things I was thinking about also.
   
   I could not think of a clean way, apart from having BashHook return a "runner" object that has methods `run` and `send_sigterm`.... but in essense this would just be another stateful hook, so might as well just leave it on BashHook, I figure.
   
   I also considered making `command` `env` and `output_encoding` hook init params.  But then you would not be able to use cached property in BashOperator, and this would add complexity to BashOperator as a result.
   
   Another thing I considered was, does it make sense to ensure that `sub_process` is unset at successful completion?  But I think there's no benefit to aggressive unsetting.  If `run_command` is used again, then `sub_process` will be overwritten with new process.  We could add a check, to see if sub_process is still running (if is already set when Popen is about to be called), or at least a warning.  However, these I think should probably be considered separately from this refactor.
   
   In sum, I too think current approach is good enough and now will work on updating docs and possibly tests as necessary.
   
   Thanks


----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       operators == orchestrator of task
   hooks == vocabulary of operators
   
   i had this thought too as i typed out a hook with no init params.  but i don't think that connections are really what defines a hook.  it's the collection of methods used for interacting with a resource.  there could be many hooks that don't need to worry about auth.
   
   operators do one thing.  hooks can potentially do very many things.  so the things that have general use should get pushed down to a hook that can be reused.
   




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):

Review comment:
       ```suggestion
       def run_command(self, command; List[str], env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
   ```
   
   We should _enforce_ command to be a list here -- it's more secure (otherwise shell escaping issues are a big risk) and should be the only option this supports




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       Possibly a breaking change, but I'm leaning towards believing that no one used those objects...




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: docs/spelling_wordlist.txt
##########
@@ -310,6 +310,7 @@ Reinitialising
 Riccomini
 Roadmap
 Robinhood
+sigterm

Review comment:
       omg hah .... are you kidding or you actually want `def send_SIGTERM`?




----------------------------------------------------------------
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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   > > i feel a little iffy about this. i am tempted to have the hook return `namedtuple('SubprocessResult', ['exit_code', 'output'])` and let the operator decide what to do (skip fail or any other action)
   > 
   > @dstandish Yes, that sounds better.
   
   Ok cool @ashb .  I have pushed this change. PTAL


----------------------------------------------------------------
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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   thanks and updated 🤞


----------------------------------------------------------------
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] kaxil commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")

Review comment:
       Intentional? Or just remnant from debugging ?




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash.py
##########
@@ -145,43 +141,15 @@ def execute(self, context):
             '\n'.join([f"{k}={v}" for k, v in airflow_context_vars.items()]),
         )
         env.update(airflow_context_vars)
+        return env
+
+    def execute(self, context=None):

Review comment:
       i like making context optional where it's not used (or in some cases even when it is used, make it optional to use it)
   
   shouldn't be in this PR i think i left it in subconsiously to start discussion.
   
   will remove
   
   and i have created a discussion in the proper place: https://github.com/apache/airflow/discussions/13492
   




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       We usually create an issue with a milestone like `Airflow 2.1` and once releasing we are about to release we try to solve all of those. Am I correct @potiuk @kaxil or do we do something else?




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")

Review comment:
       Whoops miss that. Yes this was likely debugging that should be removed.




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
+        """
+        Execute the bash command in a temporary directory which will be cleaned afterwards
+
+        If ``env`` is not supplied, ``os.environ`` is passed
+
+        :param command: the bash command to run
+        :param env: Optional dict containing environment variables to be made available to the shell
+            environment in which ``command`` will be executed.  If omitted, ``os.environ`` will be used.
+        :param output_encoding: encoding to use for decoding stdout
+        :return: last line of stdout
+        """
+        self.log.info('Tmp dir root location: \n %s', gettempdir())
+
+        with TemporaryDirectory(prefix='airflowtmp') as tmp_dir:
+
+            def pre_exec():
+                # Restore default signal disposition and invoke setsid
+                for sig in ('SIGPIPE', 'SIGXFZ', 'SIGXFSZ'):
+                    if hasattr(signal, sig):
+                        signal.signal(getattr(signal, sig), signal.SIG_DFL)
+                os.setsid()
+
+            self.log.info('Running command: %s', command)
+
+            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                ['bash', "-c", command],

Review comment:
       Oh we do have bash in here. Still, I think this should become
   
   ```suggestion
                  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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
+        """
+        Execute the bash command in a temporary directory which will be cleaned afterwards
+
+        If ``env`` is not supplied, ``os.environ`` is passed
+
+        :param command: the bash command to run
+        :param env: Optional dict containing environment variables to be made available to the shell
+            environment in which ``command`` will be executed.  If omitted, ``os.environ`` will be used.
+        :param output_encoding: encoding to use for decoding stdout
+        :return: last line of stdout
+        """
+        self.log.info('Tmp dir root location: \n %s', gettempdir())
+
+        with TemporaryDirectory(prefix='airflowtmp') as tmp_dir:
+
+            def pre_exec():
+                # Restore default signal disposition and invoke setsid
+                for sig in ('SIGPIPE', 'SIGXFZ', 'SIGXFSZ'):
+                    if hasattr(signal, sig):
+                        signal.signal(getattr(signal, sig), signal.SIG_DFL)
+                os.setsid()
+
+            self.log.info('Running command: %s', command)
+
+            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                ['bash', "-c", command],
+                stdout=PIPE,
+                stderr=STDOUT,
+                cwd=tmp_dir,
+                env=env or os.environ,

Review comment:
       This makes it impossible to run in an empty enviroment, as even passing in `dict()` would then mean it uses os.environ.




----------------------------------------------------------------
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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   hey @turbaszek @ashb @kaxil 
   
   rebased after merge of [BashOperator to raise AirflowSkipException on exit code 127](https://github.com/apache/airflow/pull/13421)
   
   that change has implications for this one
   
   what i have done for now is keep the skip logic in the `run_command` function, along with the fail logic.
   
   this means that the skip logic would stay with the hook
   
   i feel a little iffy about this.  i think best if the skip logic is handled in the operator, that the hook should not have an opinion about this.
   
   so what i'm thinking is add `namedtuple('SubprocessResult', ['exit_code', 'output'])` and then let the caller decide what to do.
   
   so in `run_command` it would not raise _any_ exception based on exit code -- it would just provide code along with output
   
   please let me know your thoughts.
   
   small note: i think will be nice to parameterize returning all output vs last line but this can be added in later 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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash.py
##########
@@ -145,43 +141,15 @@ def execute(self, context):
             '\n'.join([f"{k}={v}" for k, v in airflow_context_vars.items()]),
         )
         env.update(airflow_context_vars)
+        return env
+
+    def execute(self, context=None):

Review comment:
       Why do we have None default here?




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

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



[GitHub] [airflow] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   @ashb rebased and looks like one flakey build issue


----------------------------------------------------------------
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 #13423: WIP - add bash hook

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


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


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

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



[GitHub] [airflow] turbaszek commented on pull request #13423: Add bash hook

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


   > We could add a check, to see if sub_process is still running (if is already set when Popen is about to be called), or at least a warning. However, these I think should probably be considered separately from this refactor.
   
   That was also my concern, but I figured that the chance that we will run two processes in parallel in one operator (using one instance of a hook) is low.


----------------------------------------------------------------
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] dstandish edited a comment on pull request #13423: WIP - add bash hook

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


   > For few minutes I was considering making this hook stateless and keeping the state in operators but this adds unnecessary complexity.
   
   Yes @turbaszek, this is one of the things I was thinking about also.
   
   I could not think of a clean way, apart from having BashHook return a "runner" object that has methods `run` and `send_sigterm`.... but in essense this would just be another stateful hook, so might as well just leave it on BashHook, I figure.
   
   I also considered making `command` `env` and `output_encoding` hook init params.  But then you would not be able to use cached property in BashOperator, and this would add complexity to BashOperator as a result.  And we should be able to reuse the hook multiple times in different ways.
   
   Another thing I considered was, does it make sense to ensure that `sub_process` is unset at successful completion?  But I think there's no benefit to aggressive unsetting.  If `run_command` is used again, then `sub_process` will be overwritten with new process.  We could add a check, to see if sub_process is still running (if is already set when Popen is about to be called), or at least a warning.  However, these I think should probably be considered separately from this refactor.
   
   In sum, I too think current approach is good enough and now will work on updating docs and possibly tests as necessary.
   
   Thanks


----------------------------------------------------------------
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] dstandish commented on pull request #13423: WIP - add bash hook

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


   *one material change*
   
   in latest commit i made I made it so BashHook's run_command will use `os.environ` if `env` not supplied (and will _not_ use `os.environ` otherwise)
   
   *added / updated tests*
   
   i left all but one BashOperator test untouched.  only this one -- checking the args applied to Popen -- i moved to BashHook.
   
   where appropriate, i translated the existing tests for BashHook and copied them there.
   
   i will remove WIP designation
   


----------------------------------------------------------------
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 #13423: Add bash hook

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/458253668) is cancelling this PR. Building image for the PR has been cancelled


----------------------------------------------------------------
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] ashb commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   @dstandish Looks good now, though looks like you've got some test failures - one import ordering one and this one:
   
   ```
     File "/opt/airflow/airflow/example_dags/example_bash_operator.py", line 25, in <module>
       from airflow.hooks.bash import EXIT_CODE_SKIP
   ModuleNotFoundError: No module named 'airflow.hooks.bash'
   ```


----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       @ashb @turbaszek are you sure these imports really need to be preserved?
   
   you could make the same argument about [these lines](https://github.com/apache/airflow/pull/13423/files#diff-912cc49726a411fd88f68b771c1de66996acd9b68c7e0e6b2e0caa7d8b23a188L21-L23).
   
   or any time you remove an import from a module
   
   users probably shouldn't be importing Popen from `bash_operator` anyway... WDYT?
   
   if you think these left in, with deprecation warning, please also comment with guidance on the same issue for `airflow.operators.bash`
   
   I've left it as is for now




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

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



[GitHub] [airflow] dstandish commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   ok at long last @ashb i think this should be good
   
   there seems to be just _one_ test failure in one run which, while i don't see how it _could_ be flakey, does appear to be flakey.  it didn't happen in other configurations and doesn't repro locally.
   
   perhaps someone could rerun that test?  i am hesitant to force push again given how fussy builds have been lately


----------------------------------------------------------------
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 #13423: Add SubprocessHook for running commands from operators

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


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


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13423: Add bash hook

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



##########
File path: tests/hooks/test_bash.py
##########
@@ -0,0 +1,100 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.bash import BashHook
+
+OS_ENV_KEY = 'BASH_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestBashOperator(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the bash environment.
+        When ``env`` is not provided, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = BashHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=command, env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    def test_return_value(self):
+        hook = BashHook()
+        return_value = hook.run_command(command='echo "stdout"')
+        self.assertEqual(return_value, 'stdout')
+
+    def test_raise_exception_on_non_zero_exit_code(self):
+        hook = BashHook()
+        with self.assertRaisesRegex(
+            AirflowException, "Bash command failed\\. The command returned a non-zero exit code\\."
+        ):

Review comment:
       When squashing the commits before merging (or when doing it manually) I usually try to keep all "co-authored by". But I think we don't have any particular rules for that. 




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")

Review comment:
       resolved




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
+        """
+        Execute the bash command in a temporary directory which will be cleaned afterwards
+
+        If ``env`` is not supplied, ``os.environ`` is passed
+
+        :param command: the bash command to run
+        :param env: Optional dict containing environment variables to be made available to the shell
+            environment in which ``command`` will be executed.  If omitted, ``os.environ`` will be used.
+        :param output_encoding: encoding to use for decoding stdout
+        :return: last line of stdout
+        """
+        self.log.info('Tmp dir root location: \n %s', gettempdir())
+
+        with TemporaryDirectory(prefix='airflowtmp') as tmp_dir:
+
+            def pre_exec():
+                # Restore default signal disposition and invoke setsid
+                for sig in ('SIGPIPE', 'SIGXFZ', 'SIGXFSZ'):
+                    if hasattr(signal, sig):
+                        signal.signal(getattr(signal, sig), signal.SIG_DFL)
+                os.setsid()
+
+            self.log.info('Running command: %s', command)
+
+            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                ['bash', "-c", command],

Review comment:
       Ok and rename it `ShellHook`?




----------------------------------------------------------------
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] dstandish edited a comment on pull request #13423: Add bash hook

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


   **one material change**
   
   in latest commit i made I made it so BashHook's run_command will use `os.environ` if `env` not supplied (and will _not_ use `os.environ` otherwise)
   
   **added / updated tests**
   
   i left all but one BashOperator test untouched.  only this one -- checking the args applied to Popen -- i moved to BashHook.
   
   where appropriate, i translated the existing tests for BashHook and copied them there.
   
   i will remove WIP designation
   


----------------------------------------------------------------
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 #13423: Add SubprocessHook for running commands from operators

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/545898681) 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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   hey @turbaszek @ashb @kaxil 
   
   rebased after merge of [BashOperator to raise AirflowSkipException on exit code 127](https://github.com/apache/airflow/pull/13421)
   
   that change has implications for this one
   
   what i have done for now is leave the exit code parsing in the hook's `run_command` method.
   
   i feel a little iffy about this.  tempted to have the hook return `namedtuple('SubprocessResult', ['exit_code', 'output'])` and let the operator decide what to do (skip fail or any other action)
   
   please let me know your thoughts on which way is best 🙏
   
   small sid note: i think will be nice to parameterize returning all output vs last line but this can be added in later 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] github-actions[bot] commented on pull request #13423: Add SubprocessHook for running commands from operators

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


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


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

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



[GitHub] [airflow] dstandish commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   I will find time to do at some point today (pst) just not positive exactly when
   


----------------------------------------------------------------
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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   hey @turbaszek @ashb @kaxil 
   
   rebased after merge of [BashOperator to raise AirflowSkipException on exit code 127](https://github.com/apache/airflow/pull/13421)
   
   that change has implications for this one
   
   what i have done for now is leave the exit code parsing in the hook's `run_command` method.
   
   i feel a little iffy about this.  i am tempted to have the hook return `namedtuple('SubprocessResult', ['exit_code', 'output'])` and let the operator decide what to do (skip fail or any other action)
   
   please let me know your thoughts on which way is best 🙏
   
   small sid note: i think will be nice to parameterize returning all output vs last line but this can be added in later 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] dstandish commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = SubprocessHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=['bash', '-c', command], env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    @parameterized.expand(
+        [
+            ('test-val', 'test-val'),
+            ('test-val\ntest-val\n', ''),

Review comment:
       Existing behaviors to return the last line in standard out, not the last non-empty line.
   
   I thought about maybe tweaking this behavior in the hook for example returning all output and letting operators decide what to do with it, or somehow returning a generator or storing the full output on a instance attribute. But I figure these things could be done later. Lmk if you think something like that is worth pursuing 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] dstandish commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = SubprocessHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=['bash', '-c', command], env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    @parameterized.expand(
+        [
+            ('test-val', 'test-val'),
+            ('test-val\ntest-val\n', ''),

Review comment:
       Existing behaviors to return the last line in standard out, not the last non-empty line.
   
   I thought about maybe tweaking this behavior in the hook for example returning all output and letting operators decide what to do with it, or somehow returning a generator or storing the full output on a instance attribute. But I figure these things could be done later. Lmk if you think something like that is worth pursuing 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] ashb commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   @dstandish Hey -- do you have cycles to pick this up? I think just a rebase then wait for tests to pass is where we are at.


----------------------------------------------------------------
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] ashb commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   > i feel a little iffy about this. i am tempted to have the hook return `namedtuple('SubprocessResult', ['exit_code', 'output'])` and let the operator decide what to do (skip fail or any other action)
   
   @dstandish Yes, that sounds better.


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

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



[GitHub] [airflow] dstandish commented on pull request #13423: Add bash hook

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


   Alright folks... updated to make it shell-agnostic, and to take List[str]
   
   I have called it _SubprocessHook_
   
   This is obvs negotiable
   
   Options:
   * CommandHook (what kind of command?)
   * ProcessHook (what kind of process? isn't everything a process)
   * ShellHook (i kindof like it)
   * SubprocessHook (it is a subprocess, it uses subprocess module... why get creative?)
   
   Please take a look 🙏
   


----------------------------------------------------------------
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 #13423: WIP - add bash hook

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/455940917) is cancelling this PR. Building image for the PR has been cancelled


----------------------------------------------------------------
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 #13423: WIP - add bash hook

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


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


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

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



[GitHub] [airflow] ashb commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = SubprocessHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=['bash', '-c', command], env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    @parameterized.expand(
+        [
+            ('test-val', 'test-val'),
+            ('test-val\ntest-val\n', ''),

Review comment:
       Oh right, if this is existing behaviour 👍🏻 




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       that said, i am not completely opposed to making this a CommandRunner -- that's is all it's doing after all.
   
   just saying, i don't think that absence of connections is decisive
   
   but i think though, making this a hook is consistent with the airflow design pattern.
   
   and if you make it a one-off thing like CommandRunner, that will make it tougher to find, when someone needs to use bash in an operator.




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       yes, totally possible.
   
   we could let them remain and add a deprecation warning.
   
   i am not sure --  is there a special syntax we have for "this must be removed at this release", like to automate deprecated removals?
   
   whatever you suggest 




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: docs/spelling_wordlist.txt
##########
@@ -310,6 +310,7 @@ Reinitialising
 Riccomini
 Roadmap
 Robinhood
+sigterm

Review comment:
       Make it `SIGTERM` then we don't need to add a new word to the spelling list? ;)




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       @dstandish I agree, I always look at hooks as repository of methods reused by hooks. The layer of authentication is just a part of it but not crucial in my opinion. 




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       This is probably one of the cases where removing this is _technically_ breaking, but people are unlikely to have used it, and it was never documented anyway, so is probably fine.
   
   Probably.




----------------------------------------------------------------
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] ashb commented on pull request #13423: Add bash hook

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


   SubprocessHook is good.
   
   (ShellHook is _not_ because that has a different meaning -- see https://docs.python.org/3/library/subprocess.html#security-considerations)


----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
+        """
+        Execute the bash command in a temporary directory which will be cleaned afterwards
+
+        If ``env`` is not supplied, ``os.environ`` is passed
+
+        :param command: the bash command to run
+        :param env: Optional dict containing environment variables to be made available to the shell
+            environment in which ``command`` will be executed.  If omitted, ``os.environ`` will be used.
+        :param output_encoding: encoding to use for decoding stdout
+        :return: last line of stdout
+        """
+        self.log.info('Tmp dir root location: \n %s', gettempdir())
+
+        with TemporaryDirectory(prefix='airflowtmp') as tmp_dir:
+
+            def pre_exec():
+                # Restore default signal disposition and invoke setsid
+                for sig in ('SIGPIPE', 'SIGXFZ', 'SIGXFSZ'):
+                    if hasattr(signal, sig):
+                        signal.signal(getattr(signal, sig), signal.SIG_DFL)
+                os.setsid()
+
+            self.log.info('Running command: %s', command)
+
+            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                ['bash', "-c", command],

Review comment:
       Rename it to something CommandHook or ProcessExecutorHook please




----------------------------------------------------------------
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 #13423: Add bash hook

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


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


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

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



[GitHub] [airflow] ashb commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   @dstandish Hey -- do you have cycles to pick this up? I think just a rebase then wait for tests to pass is where we are at.


----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = SubprocessHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=['bash', '-c', command], env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    @parameterized.expand(
+        [
+            ('test-val', 'test-val'),
+            ('test-val\ntest-val\n', ''),

Review comment:
       This case looks odd - I would have expected "last line" here to be `test-val`, not empty.




----------------------------------------------------------------
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 #13423: Add SubprocessHook for running commands from operators

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/474368640) 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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   hey @turbaszek @ashb @kaxil 
   
   rebased after merge of [BashOperator to raise AirflowSkipException on exit code 127](https://github.com/apache/airflow/pull/13421)
   
   that change has implications for this one
   
   what i have done for now is keep the skip logic in the `run_command` function, along with the fail logic.
   
   this means that the skip logic would stay with the hook
   
   i feel a little iffy about this.  on one hand i think skip / fail logic is best handled in the operator -- that the hook should not have an opinion about this.
   
   on the other hand, hooks are only used in the running of an operator, so it is legitimate to have this kind of logic here.
   
   if we want operator to handle, i was thinking `run_command` could return `namedtuple('SubprocessResult', ['exit_code', 'output'])` and then let the caller decide what to do. `run_command` it would not raise _any_ exception based on exit code -- it would just provide code along with output
   
   please let me know your thoughts on which way is best 🙏
   
   small note: i think will be nice to parameterize returning all output vs last line but this can be added in later 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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):

Review comment:
       ```suggestion
       def run_command(self, command: List[str], env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
   ```
   
   We should _enforce_ command to be a list here -- it's more secure (otherwise shell escaping issues are a big risk) and should be the only option this supports




----------------------------------------------------------------
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] ashb commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   Yeah, that's a timing based flakey test. Merging.


----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       @ashb @turbaszek are you sure these imports really need to be preserved?
   
   you could make the same argument about [these lines](https://github.com/apache/airflow/pull/13423/files#diff-912cc49726a411fd88f68b771c1de66996acd9b68c7e0e6b2e0caa7d8b23a188L21-L23).
   
   or any time you remove an import from a module
   
   users probably shouldn't be importing Popen from `bash_operator` anyway... WDYT?
   
   if you think these left in, with deprecation warning, please also comment on the same issue for `airflow.operators.bash`
   




----------------------------------------------------------------
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] dstandish commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   @ashb revised and looks like one flakey build issue


----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       that said, i am not at all opposed to making this a CommandRunner -- that's is all it's doing after all.
   
   just saying, i don't think that absence of connections is decisive




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
+        """
+        Execute the bash command in a temporary directory which will be cleaned afterwards
+
+        If ``env`` is not supplied, ``os.environ`` is passed
+
+        :param command: the bash command to run
+        :param env: Optional dict containing environment variables to be made available to the shell
+            environment in which ``command`` will be executed.  If omitted, ``os.environ`` will be used.
+        :param output_encoding: encoding to use for decoding stdout
+        :return: last line of stdout
+        """
+        self.log.info('Tmp dir root location: \n %s', gettempdir())
+
+        with TemporaryDirectory(prefix='airflowtmp') as tmp_dir:
+
+            def pre_exec():
+                # Restore default signal disposition and invoke setsid
+                for sig in ('SIGPIPE', 'SIGXFZ', 'SIGXFSZ'):
+                    if hasattr(signal, sig):
+                        signal.signal(getattr(signal, sig), signal.SIG_DFL)
+                os.setsid()
+
+            self.log.info('Running command: %s', command)
+
+            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                ['bash', "-c", command],
+                stdout=PIPE,
+                stderr=STDOUT,
+                cwd=tmp_dir,
+                env=env or os.environ,

Review comment:
       good point.
   
   what do you think is better:
   * not touch env at all in the hook 
   * pass `os.environ` only if `env is None`




----------------------------------------------------------------
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] ashb merged pull request #13423: Add SubprocessHook for running commands from operators

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


   


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       hook > utils in my opinion. Maybe in future we will add some connections for auth (for example for gcp or aws)




----------------------------------------------------------------
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 #13423: Add SubprocessHook for running commands from operators

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






----------------------------------------------------------------
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] dstandish commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   > @dstandish Looks good now, though looks like you've got some test failures - one import ordering one and this one:
   > 
   > ```
   >   File "/opt/airflow/airflow/example_dags/example_bash_operator.py", line 25, in <module>
   >     from airflow.hooks.bash import EXIT_CODE_SKIP
   > ModuleNotFoundError: No module named 'airflow.hooks.bash'
   > ```
   
   that's what sleep-deprived rebasing can do
   thanks and updated 🤞


----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add SubprocessHook for running commands from operators

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



##########
File path: tests/hooks/test_subprocess.py
##########
@@ -0,0 +1,110 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+import pytest
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.subprocess import SubprocessHook
+
+OS_ENV_KEY = 'SUBPROCESS_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestSubprocessHook(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('empty env', {}, {OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the command environment.
+        When ``env`` is ``None``, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = SubprocessHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=['bash', '-c', command], env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    @parameterized.expand(
+        [
+            ('test-val', 'test-val'),
+            ('test-val\ntest-val\n', ''),

Review comment:
       Existing behavior is to return the last line in standard out, not the last non-empty line.
   
   I thought about maybe tweaking this behavior in the hook for example returning all output and letting operators decide what to do with it, or somehow returning a generator or storing the full output on a instance attribute. But I figure these things could be done later. Lmk if you think something like that is worth pursuing 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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       ok so sounds like what I need to do is put these imports back and add deprecation warning for 3.0 for them
   




----------------------------------------------------------------
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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/operators/bash_operator.py
##########
@@ -20,7 +20,7 @@
 import warnings
 
 # pylint: disable=unused-import
-from airflow.operators.bash import STDOUT, BashOperator, Popen, gettempdir  # noqa
+from airflow.operators.bash import BashOperator  # noqa

Review comment:
       yes, totally possible.
   
   we could let them remain and add a deprecation warning.
   
   i am not sure --  is there a special syntax we have for "this must be removed at this release", like to an automated check for deprecated removals?
   
   whatever you suggest 




----------------------------------------------------------------
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 #13423: Add SubprocessHook for running commands from operators

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/474368640) 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] dstandish commented on pull request #13423: Add SubprocessHook for running commands from operators

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






----------------------------------------------------------------
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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   @ashb rebased and looks like one flakey build issue


----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       So two observationsj:
   
   This hook has nothing to do with bash -- it doesn't run bash, it just runs a command.
   
   Secondly the main purpose of a Hook is to make use of a connection -- this doesn't do that either.
   
   So I like the idea of separation, but I'm not sure of the exact separation/code layout.
   
   One monent.




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13423: Add bash hook

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



##########
File path: tests/hooks/test_bash.py
##########
@@ -0,0 +1,100 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.bash import BashHook
+
+OS_ENV_KEY = 'BASH_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestBashOperator(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the bash environment.
+        When ``env`` is not provided, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = BashHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=command, env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    def test_return_value(self):
+        hook = BashHook()
+        return_value = hook.run_command(command='echo "stdout"')
+        self.assertEqual(return_value, 'stdout')

Review comment:
       ```suggestion
           assert return_value == 'stdout'
   ```
   See #12951 




----------------------------------------------------------------
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 #13423: Add SubprocessHook for running commands from operators

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/478661779) 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] dstandish commented on a change in pull request #13423: Add bash hook

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



##########
File path: tests/hooks/test_bash.py
##########
@@ -0,0 +1,100 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from pathlib import Path
+from subprocess import PIPE, STDOUT
+from tempfile import TemporaryDirectory
+from unittest import mock
+
+from parameterized import parameterized
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.bash import BashHook
+
+OS_ENV_KEY = 'BASH_ENV_TEST'
+OS_ENV_VAL = 'this-is-from-os-environ'
+
+
+class TestBashOperator(unittest.TestCase):
+    @parameterized.expand(
+        [
+            ('with env', {'ABC': '123', 'AAA': '456'}, {'ABC': '123', 'AAA': '456', OS_ENV_KEY: ''}),
+            ('no env', None, {OS_ENV_KEY: OS_ENV_VAL}),
+        ]
+    )
+    def test_env(self, name, env, expected):
+        """
+        Test that env variables are exported correctly to the bash environment.
+        When ``env`` is not provided, ``os.environ`` should be passed to ``Popen``.
+        Otherwise, the variables in ``env`` should be available, and ``os.environ`` should not.
+        """
+        print(f"test_name: {name}")
+        hook = BashHook()
+
+        def build_cmd(keys, filename):
+            """
+            Produce bash command to echo env vars into filename.
+            Will always echo the special test var named ``OS_ENV_KEY`` into the file to test whether
+            ``os.environ`` is passed or not.
+            """
+            return '\n'.join([f"echo {k}=${k}>> {filename}" for k in [*keys, OS_ENV_KEY]])
+
+        with TemporaryDirectory() as tmp_dir, mock.patch.dict('os.environ', {OS_ENV_KEY: OS_ENV_VAL}):
+            tmp_file = Path(tmp_dir, 'test.txt')
+            command = build_cmd(env and env.keys() or [], tmp_file.as_posix())
+            hook.run_command(command=command, env=env)
+            actual = dict([x.split('=') for x in tmp_file.read_text().splitlines()])
+            assert actual == expected
+
+    def test_return_value(self):
+        hook = BashHook()
+        return_value = hook.run_command(command='echo "stdout"')
+        self.assertEqual(return_value, 'stdout')
+
+    def test_raise_exception_on_non_zero_exit_code(self):
+        hook = BashHook()
+        with self.assertRaisesRegex(
+            AirflowException, "Bash command failed\\. The command returned a non-zero exit code\\."
+        ):

Review comment:
       sounds good
   
   will do
   
   one question
   
   do we have a protocol for when reviewer suggests changes, what to do about authorship? i.e. do we take care to ensure that they have joint authorship? or, do we not care, and one may go ahead and fix locally, and / or squash away after a rebase etc?
   
   thanks 🙏




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: docs/spelling_wordlist.txt
##########
@@ -310,6 +310,7 @@ Reinitialising
 Riccomini
 Roadmap
 Robinhood
+sigterm

Review comment:
       No, just in the doc string where you say "send sigterm" change it to "send SIGTERM" -  spell checking doesn't look at function names




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):
+    """Hook for running bash commands"""
+
+    def __init__(self) -> None:
+        self.sub_process = None
+        super().__init__()
+
+    def run_command(self, command, env: Optional[Dict[str, str]] = None, output_encoding: str = 'utf-8'):
+        """
+        Execute the bash command in a temporary directory which will be cleaned afterwards
+
+        If ``env`` is not supplied, ``os.environ`` is passed
+
+        :param command: the bash command to run
+        :param env: Optional dict containing environment variables to be made available to the shell
+            environment in which ``command`` will be executed.  If omitted, ``os.environ`` will be used.
+        :param output_encoding: encoding to use for decoding stdout
+        :return: last line of stdout
+        """
+        self.log.info('Tmp dir root location: \n %s', gettempdir())
+
+        with TemporaryDirectory(prefix='airflowtmp') as tmp_dir:
+
+            def pre_exec():
+                # Restore default signal disposition and invoke setsid
+                for sig in ('SIGPIPE', 'SIGXFZ', 'SIGXFSZ'):
+                    if hasattr(signal, sig):
+                        signal.signal(getattr(signal, sig), signal.SIG_DFL)
+                os.setsid()
+
+            self.log.info('Running command: %s', command)
+
+            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                ['bash', "-c", command],
+                stdout=PIPE,
+                stderr=STDOUT,
+                cwd=tmp_dir,
+                env=env or os.environ,

Review comment:
       Let's go with the if none approach




----------------------------------------------------------------
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] ashb commented on a change in pull request #13423: Add bash hook

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



##########
File path: airflow/hooks/bash.py
##########
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import signal
+from subprocess import PIPE, STDOUT, Popen
+from tempfile import TemporaryDirectory, gettempdir
+from typing import Dict, Optional
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class BashHook(BaseHook):

Review comment:
       Let's keep this as a hook - runner is already a concept in airflow (check out StandardTaskRunner) so we don't want to overload that 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] dstandish edited a comment on pull request #13423: Add SubprocessHook for running commands from operators

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


   hey @turbaszek @ashb @kaxil 
   
   rebased after merge of [BashOperator to raise AirflowSkipException on exit code 127](https://github.com/apache/airflow/pull/13421)
   
   that change has implications for this one
   
   what i have done for now is keep the skip logic in the `run_command` function, along with the fail logic.
   
   this means that the skip logic would stay with the hook
   
   i feel a little iffy about this.  i think best if the skip logic is handled in the operator, that the hook should not have an opinion about this.
   
   so what i'm thinking is add `namedtuple('SubprocessResult', ['exit_code', 'output')` and then let the caller decide what to do.
   
   so in `run_command` it would not raise _any_ exception based on exit code -- it would just provide code along with output
   
   please let me know your thoughts.
   
   small note: i think will be nice to parameterize returning all output vs last line but this can be added in later 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] dstandish commented on pull request #13423: Add SubprocessHook for running commands from operators

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


   > > i feel a little iffy about this. i am tempted to have the hook return `namedtuple('SubprocessResult', ['exit_code', 'output'])` and let the operator decide what to do (skip fail or any other action)
   > 
   > @dstandish Yes, that sounds better.
   
   Ok cool.  I have pushed this change. PTAL


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