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 2022/07/27 04:25:47 UTC

[GitHub] [airflow] MatrixManAtYrService opened a new issue, #25330: User defined `env` clobbers PATH, BashOperator can't find bash

MatrixManAtYrService opened a new issue, #25330:
URL: https://github.com/apache/airflow/issues/25330

   ### Apache Airflow version
   
   main (development)
   
   ### What happened
   
   NixOS is unconventional in some ways. For instance `which bash` prints `/run/current-system/sw/bin/bash`, which isn't a place that most people expect to go looking for bash.
   
   I can't be sure if this is the reason--or if it's some other peculiarity--but on NixOS, cases where `BashOperator` defines an `env` cause the task to fail with this error:
   
   ```
   venv ❯ airflow dags test nopath "$(date +%Y-%m-%d)"
   [2022-07-26 21:54:09,704] {dagbag.py:508} INFO - Filling up the DagBag from /home/matt/today/dags
   [2022-07-26 21:54:10,129] {base_executor.py:91} INFO - Adding to queue: ['<TaskInstance: nopath.nopath backfill__2022-07-26T00:00:00+00:00 [queued]>']
   [2022-07-26 21:54:15,148] {subprocess.py:62} INFO - Tmp dir root location:
    /tmp
   [2022-07-26 21:54:15,149] {subprocess.py:74} INFO - Running command: ['bash', '-c', 'echo hello world']
   [2022-07-26 21:54:15,238] {debug_executor.py:84} ERROR - Failed to execute task: [Errno 2] No such file or directory: 'bash'.
   Traceback (most recent call last):
     File "/home/matt/src/airflow/airflow/executors/debug_executor.py", line 78, in _run_task
       ti.run(job_id=ti.job_id, **params)
     File "/home/matt/src/airflow/airflow/utils/session.py", line 71, in wrapper
       return func(*args, session=session, **kwargs)
     File "/home/matt/src/airflow/airflow/models/taskinstance.py", line 1782, in run
       self._run_raw_task(
     File "/home/matt/src/airflow/airflow/utils/session.py", line 68, in wrapper
       return func(*args, **kwargs)
     File "/home/matt/src/airflow/airflow/models/taskinstance.py", line 1445, in _run_raw_task
       self._execute_task_with_callbacks(context, test_mode)
     File "/home/matt/src/airflow/airflow/models/taskinstance.py", line 1623, in _execute_task_with_callbacks
       result = self._execute_task(context, task_orig)
     File "/home/matt/src/airflow/airflow/models/taskinstance.py", line 1694, in _execute_task
       result = execute_callable(context=context)
     File "/home/matt/src/airflow/airflow/operators/bash.py", line 183, in execute
       result = self.subprocess_hook.run_command(
     File "/home/matt/src/airflow/airflow/hooks/subprocess.py", line 76, in run_command
       self.sub_process = Popen(
     File "/nix/store/cgxc3jz7idrb1wnb2lard9rvcx6aw2si-python3-3.9.6/lib/python3.9/subprocess.py", line 951, in __init__
       self._execute_child(args, executable, preexec_fn, close_fds,
     File "/nix/store/cgxc3jz7idrb1wnb2lard9rvcx6aw2si-python3-3.9.6/lib/python3.9/subprocess.py", line 1821, in _execute_child
       raise child_exception_type(errno_num, err_msg, err_filename)
   FileNotFoundError: [Errno 2] No such file or directory: 'bash'
   ```
   
   On the other hand, tasks succeed if:
   - The author doesn't use the `env` kwarg
   - they use `env` to explicitly set `PATH` to the output of `which bash`
   - or they are run on a more conventional system (like my MacBook)
   
   Here is a DAG which demonstrates this:
   ```python3
   from airflow.models import DAG
   from airflow.operators.bash import BashOperator
   from datetime import datetime, timedelta
   
   with DAG(
       dag_id="withpath",
       start_date=datetime(1970, 1, 1),
       schedule_interval=None,
   ) as withpath:
   
       BashOperator(
           task_id="withpath",
           env={"PATH": "/run/current-system/sw/bin/", "WORLD": "world"},
           bash_command="echo hello $WORLD",
       )
   
   with DAG(
       dag_id="nopath",
       start_date=datetime(1970, 1, 1),
       schedule_interval=None,
   ) as nopath:
   
       BashOperator(
           task_id="nopath",
           env={"WORLD": "world"},
           bash_command="echo hello $WORLD",
       )
   ```
   
   `withpath` succeeds, but `nopath` fails, showing the above error.
   
   ### What you think should happen instead
   
   Unless the user explicitly sets PATH via the `env` kwarg, airflow should populate it with whatever it finds in the enclosing environment.
   
   ### How to reproduce
   
   I can reproduce it reliably, but only on this machine.  I'm willing to fix this myself--since I can test it right here--but I'm filing this issue because I need a hint.  Where should I start?
   
   ### Operating System
   
   NixOS 21.11 (Porcupine)
   
   ### Versions of Apache Airflow Providers
   
   n/a
   
   ### Deployment
   
   Virtualenv installation
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org.apache.org

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


[GitHub] [airflow] uranusjr commented on issue #25330: User defined `env` clobbers PATH, BashOperator can't find bash

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #25330:
URL: https://github.com/apache/airflow/issues/25330#issuecomment-1196254637

   The easiest fix is likely to find Bash before `PATH` gets overridden. In `execute`, we can first resolve `bash` with `shutil.which`, and then call `run_command` with that resolved bash path (which should be absolute).


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk closed issue #25330: User defined `env` clobbers PATH, BashOperator can't find bash

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #25330: User defined `env` clobbers PATH, BashOperator can't find bash
URL: https://github.com/apache/airflow/issues/25330


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] MatrixManAtYrService commented on issue #25330: User defined `env` clobbers PATH, BashOperator can't find bash

Posted by GitBox <gi...@apache.org>.
MatrixManAtYrService commented on issue #25330:
URL: https://github.com/apache/airflow/issues/25330#issuecomment-1196255157

   Thanks @uranusjr, I'll give that a try.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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