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/04/08 17:44:47 UTC

[GitHub] [airflow] json2d opened a new issue #15286: use PythonVirtualenvOperator with a prebuilt env

json2d opened a new issue #15286:
URL: https://github.com/apache/airflow/issues/15286


   **Description**
   
   
   **Use case / motivation**
   
   ```py
   ```
   **Are you willing to submit a PR?**
   
   Perhaps
   
   **Related Issues**
   


-- 
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] gaoyibin0001 commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   > @ManikandanUV We are doing it the following way for now:
   > 
   > ```python
   > env = vars_dict.get("conda_env", None)
   > path_to_python = f"/home/username/.conda{'/envs/'+env if env is not None else ''}/bin/python"
   > 
   > parse_files = BashOperator(
   >             task_id='parse-files',
   >             bash_command=f"{path_to_python} {abs_path_code}/my_repo/parse.py {files_to_parse}",
   >             env={"PATH": os.environ["PATH"],
   >                  "DB_CONN": db_conn}
   >         )
   > ```
   > 
   > We have an environment variable containing the conda-env name which is used to get the full path to the Python executable. Then, using a BashOperator, we can use the same environment again for different Tasks.
   > 
   > Additionally, we run an update to the environment if requirements changed (note that we are using poetry as package manager):
   > 
   > ```python
   > update_repo = BashOperator(
   >     task_id=f"update-repo-{folder}",
   >     bash_command=f"cd {abs_path_code}/{folder}; "
   >            "git checkout master; git stash; git stash drop; git pull"
   >     )
   > install_dependencies = BashOperator(
   >     task_id=f"install-dependencies-{folder}",
   >           bash_command=f"cd {abs_path_code}/{folder}; conda activate {env_name}; poetry install "
   >     )
   > update_repo >> install_dependencies
   > `
   > ``
   > ```
   
   may use "conda run -n env_name python xxx.py


-- 
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] Jakobhenningjensen commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   If I may pitch an idea; instead of creating Venvs for each task/operator, what about creating a virtualdag e.g
   
   ```python
   
   with VenvDAG(
       default_args=default_args,
       schedule_interval=timedelta(days=1),
       start_date=days_ago(1),
       tags=['venv'],
      requirements = "requirements.txt"
   ) as dag:
   
       t1 = PythonOperator()
       t2 = PythonOperator()
   
   t1 >>t2
   ```
   such that `VenvDAG` spins up a venv where `t1` and `t2` lives in.
   
   
   


-- 
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] uranusjr edited a comment on issue #15286: use PythonVirtualenvOperator with a prebuilt env

Posted by GitBox <gi...@apache.org>.
uranusjr edited a comment on issue #15286:
URL: https://github.com/apache/airflow/issues/15286#issuecomment-816019786


   Edit: I misread the issue. Thanks for the added example.
   
   ---
   
   <del>Hi, could provide more context to the issue? Why do you want this feature? Is the current virtualenv approach not working? Are there technical advantages to the built-in `venv` over `virtualenv`?</del>
   
   <del>From my understanding, virtual environments created by `virtualenv` should be technically identical to those created by `venv`, but `virtualenv` provides much better configurability and performance characteristics in the implementation. The only real advantage to `venv` is it does not require installing a third-party dependency, but that is a non-issue for Airflow, which has a ton of them anyway.</del>


-- 
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] gaoyibin0001 edited a comment on issue #15286: use PythonVirtualenvOperator with a prebuilt env

Posted by GitBox <gi...@apache.org>.
gaoyibin0001 edited a comment on issue #15286:
URL: https://github.com/apache/airflow/issues/15286#issuecomment-1046503180


   one walk around may be use DockerOperator or KubernetesPodOperator  to isolate env depends on your deploment.  
   PythonVirtualenvOperator in my understanding is suitable for task requiring few dependencies.
   for usecase that demands too many packages or very big pakage like torch, the installation overhead is too much for per task. using prebuild images is a good choice, which can be maintained offline without influence the airflow online worker


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

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 commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   Just one comment - this fine, if you can make sure all your - distributed - venvs are present on all the workers (which might be tricky if you want to update those) - and you have to somehow link the "task" definition (expecting certain venv with certain requirement versions) with the "deployment" (i.e. worker definition). Any kind of "upgrade" to such an env might be tricky. The "local" installation pattern had the advantage, that you always got the requirements in the version you described in the task definition (via requirements specification).  
   
   I think a better solution would be to use caching mechanism to the task and modify the PythonVirtualenv to use it. However this might be tricky to get right when you have multiple tasks of the same type running in the same runner in Celery deployment. 


-- 
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] uranusjr commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   I thought about this a bit and feel there are two things here to consider. The first is the overhead for `PythonVirtualenvOperator` to populate the virtuale environment, which (as mentioned above) should be solved by introducing some caching mechanism, something similar to how CI caches stuff between runs. This is very much worth doing.
   
   There is another use case surrounding `PythonVirtualenvOperator`, however—people wanting more control over the environment used to run Python code. Maybe there are some dependencies that can’t be covered by Python packaging, or require special configuration of the environment. Or maybe the user is simply migrating from an existing cron setup and want to reuse the environments first to avoid re-writing everything all at once. Currently people would need to “drop down” to `BashOperator` to achieve this, and while that definitely works, kind of “wastes” the knowledge the operator is running Python, and prevents nice things we can do with that knowledge.
   
   I think two solutions are needed for the two problems. The first is probably more intuitive to design; we can add caching options to `PythonVirtualenvOperator` to make Airflow cache and reuse the environment (or a subset of it); we can steal some ideas from CI designs for this. The other is less straightforward; my current idea is to introduce a `ExternalPythonOperator` (please recommend better names) that, instead of taking a requirement to create a virtual environment from, simply takes a path to a Python executable to run the Python callable with. The behaviour would otherwise be very similar to `PythonVirtualenvOperator`, including all the code generation and pickling caveats. This would be much easier to implement than the caching one (which, also mentioned above, requires tricky considerations with parallelism). So I’ll probably start with it and see what I can do.
   
   Any advices are very welcomed!


-- 
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] HansBambel commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   @ManikandanUV We are doing it the following way for now:
   ```py
   env = vars_dict.get("conda_env", None)
   path_to_python = f"/home/username/.conda{'/envs/'+env if env is not None else ''}/bin/python"
   
   parse_files = BashOperator(
               task_id='parse-files',
               bash_command=f"{path_to_python} {abs_path_code}/my_repo/parse.py {files_to_parse}",
               env={"PATH": os.environ["PATH"],
                    "DB_CONN": db_conn}
           )
   ```
   We have an environment variable containing the conda-env name which is used to get the full path to the Python executable. Then, using a BashOperator, we can use the same environment again for different Tasks.
   
   Additionally, we run an update to the environment if requirements changed (note that we are using poetry as package manager):
   ```py
   update_repo = BashOperator(
       task_id=f"update-repo-{folder}",
       bash_command=f"cd {abs_path_code}/{folder}; "
              "git checkout master; git stash; git stash drop; git pull"
       )
   install_dependencies = BashOperator(
       task_id=f"install-dependencies-{folder}",
             bash_command=f"cd {abs_path_code}/{folder}; conda activate {env_name}; poetry install "
       )
   update_repo >> install_dependencies
   `
   ``


-- 
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 commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   I came back to the discussion after the Summit, and It gave me some idea @uranusjr. The problem with this approach is that you cannot have separate operator to prepare env and another to run the env. The problem is that they might run on different workers. 
   
   However with the custom XCom backends, I think we are very closet to use those backends as a generic Caching mechanism (That we could also use to store virtualenv caches). I think it is not that far to be able to add similar mechanism as we see in many CI environments, where we could specify ID of the cache (with some variations) and pull it from shared location (if exists) or push it there after task succeeds. Then we could make python virtualenv  to build venv if it is missing (using requirments) and push it after complete. We would have to add some basic mechanism of invalidation of the hash (for example when hash of requirements.txt changes). 
   
   WDYT @uranusjr ?


-- 
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] uranusjr commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   Hi, could provide more context to the issue? Why do you want this feature? Is the current virtualenv approach not working? Are there technical advantages to the built-in `venv` over `virtualenv`?
   
   From my understanding, virtual environments created by `virtualenv` should be technically identical to those created by `venv`, but `virtualenv` provides much better configurability and performance characteristics in the implementation. The only real advantage to `venv` is it does not require installing a third-party dependency, but that is a virtual non-issue for Airflow, which has a ton of them anyway.


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

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #15286:
URL: https://github.com/apache/airflow/issues/15286#issuecomment-816015987


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


-- 
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] json2d commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   > it is pretty wasteful to install the requirements every time the task is run, if the environment is persisted
   
   i agree
   
   > populate the environment yourself, and use `BashOperator` instead.
   
   this could work, though in my experience one disadvantage with `BashOperator` vs `PythonOperator`/`PythonVirtualenvOperator` is that it only provides limited logging in Airflow for errors 


-- 
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] ManikandanUV commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   Bump. Selecting my own existing venv, or at the very least reuse existing venvs instead of creating one every time, is a great feature to have.


-- 
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] uranusjr commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   That’s an interesting idea, but would require much more change since we don’t currently have a hook point for DAG to do pre-processing before operators are run.
   
   Maybe something like this would be easier:
   
   ```python
   with DAG(...) as dag:
       t0 = CreateVirtualEnvironmentOperator(task_id="init_venv", ...)
   
       python_prefix_template = "{{ ti.xcom_pull(task_ids='init_venv')['prefix'] }}"
       t1 = ExternalPythonOperator(..., python_prefix=python_prefix_template)
       t2 = ExternalPythonOperator(..., python_prefix=python_prefix_template)
   
       t0 >> t1 >> t2
   ```
   
   There are probably abstractions available to make this easier, but that’s the basic idea.


-- 
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] uranusjr commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   I wonder if this should be made more generic as a new operator that can take *any* Python installation prefix (e.g. `/home/uranusjr/.local/my-custom-compiled-python`)


-- 
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] json2d commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   hey @uranusjr yeah sorry i hit submit by accident there


-- 
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] uranusjr edited a comment on issue #15286: use PythonVirtualenvOperator with a prebuilt env

Posted by GitBox <gi...@apache.org>.
uranusjr edited a comment on issue #15286:
URL: https://github.com/apache/airflow/issues/15286#issuecomment-816019786


   Hi, could provide more context to the issue? Why do you want this feature? Is the current virtualenv approach not working? Are there technical advantages to the built-in `venv` over `virtualenv`?
   
   From my understanding, virtual environments created by `virtualenv` should be technically identical to those created by `venv`, but `virtualenv` provides much better configurability and performance characteristics in the implementation. The only real advantage to `venv` is it does not require installing a third-party dependency, but that is a non-issue for Airflow, which has a ton of them anyway.


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

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



[GitHub] [airflow] uranusjr edited a comment on issue #15286: use PythonVirtualenvOperator with a prebuilt env

Posted by GitBox <gi...@apache.org>.
uranusjr edited a comment on issue #15286:
URL: https://github.com/apache/airflow/issues/15286#issuecomment-816026927


   I wonder if this should be made more generic as a new operator that can take *any* Python installation prefix (e.g. `/home/uranusjr/.local/my-custom-compiled-python`). It might not be too useful in general though, since it is pretty wasteful to install the requirements every time the task is run, if the environment is persisted. It is probably easier to populate the environment yourself, and use `BashOperator` instead.


-- 
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] gaoyibin0001 commented on issue #15286: use PythonVirtualenvOperator with a prebuilt env

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


   one walk around may be use DockerOperator or KubernetesPodOperator  to isolate env depends on your deploment.  
   PythonVirtualenvOperator in my understanding is suitable for requiring few dependencies.
   for usecase that demands too many packages or very big pakage like torch, the installation overhead is too much for per task. using prebuild images is a good choice, which can be maintained offline without influence the airflow online worker


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

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

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