Seems we are only now at the stage that we need to choose the best name for the operator
I started a name poll on Twitter :) https://twitter.com/jarekpotiuk/status/1563602012100767746 PR here: https://github.com/apache/airflow/pull/25780 J. On Thu, Aug 18, 2022 at 1:53 AM Jarek Potiuk <ja...@potiuk.com> wrote: > Draft PR - needs some more tests and review with typing changes - in > https://github.com/apache/airflow/pull/25780 > Eventually PythonExternalOperator seems like a good name. > > J. > > > On Wed, Aug 17, 2022 at 10:37 PM Jeambrun Pierre <pierrejb...@gmail.com> > wrote: > >> I also like the ability to use a specific interpreter. >> >> Maybe we could leave everything that is env related to the PVO (even >> using an existing one) and let another one handle the interpreter. >> >> As Ash mentioned I also feel like an additional parameter >> (python/interpreter etc.) to the PO would make sense and is quite intuitive >> rather than a complete new operator, but it might be harder to implement. >> >> Best >> Pierre Jeambrun >> >> Le mer. 17 août 2022 à 20:46, Collin McNulty <col...@astronomer.io.invalid> >> a écrit : >> >>> I concur that this would be very useful. I can see a common pattern >>> being to have a task to create an environment if it does not already exist >>> and then subsequent tasks use that environment. >>> >>> On Wed, Aug 17, 2022 at 12:30 PM Jarek Potiuk <ja...@potiuk.com> wrote: >>> >>>> Sounds like this is really in the middle between PVO and PO :). >>>> >>>> BTW. I spoke with a customer of mine today and they said they would >>>> ABSOLUTELY love it. They were actually blocked from migrating to 2.3.3 >>>> because one of their teams needed a DBT environment while the other >>>> team needed some other dependency and they are conflicting with each >>>> other. They are using Nomad + Docker already and while extending the >>>> image with another venv is super-easy for them, they were considering >>>> building several Docker images to serve their users but it is an order >>>> of magnitude more complex problem for them because they would have to >>>> make a whole new pipeline to build a distribute multiple images and >>>> implements queue-base split between the teams or switch to using >>>> DockerOperator. >>>> >>>> This one will allow them to do limited version of multi-tenancy for >>>> their teams - without the actual separation but with even more >>>> fine-grained separation of envs - because they would be able to use >>>> different deps even for different tasks in the same DAG. >>>> >>>> >>>> J, >>>> >>>> On Wed, Aug 17, 2022 at 6:21 PM Ash Berlin-Taylor <a...@apache.org> >>>> wrote: >>>> > >>>> > Another option would be to change the PythonOperator/@task to take a >>>> `python` argument (which also does change the behaviour of _that_ operator >>>> a lot with or without that argument if we did that.) >>>> > >>>> > On 17 August 2022 15:46:52 BST, Jarek Potiuk <ja...@potiuk.com> >>>> wrote: >>>> >> >>>> >> Yeah. TP - I like that explicit separation. It's much cleaner. I >>>> still >>>> >> have to think about the name though. While I see where >>>> >> ExternalPythonOperator comes from, It sounds a bit less than >>>> obvious. >>>> >> I think the name should somehow contain "Environment" because very >>>> few >>>> >> people realise that running Python from a virtualenv actually >>>> >> implicitly "activates" the venv. >>>> >> I think maybe deprecating the old PythonVirtualenvOperator and >>>> >> introducing two new operators: PythonInCreatedVirtualEnvOperator, >>>> >> PythonInExistingVirtualEnvOperator ? Not exactly those names - they >>>> >> are too long - but something like that. Maybe we should get rid of >>>> >> Python in the name at all ? >>>> >> >>>> >> BTW. I think we should generally do more of the discussions here and >>>> >> express our thoughts about Airflow here. Even if there are no answers >>>> >> or interest immediately, I think that it makes sense to do a bit of a >>>> >> melting pot that sometimes might produce some cool (or rather hot) >>>> >> stuff as a result. >>>> >> >>>> >> On Wed, Aug 17, 2022 at 8:45 AM Tzu-ping Chung >>>> <t...@astronomer.io.invalid> wrote: >>>> >>> >>>> >>> >>>> >>> One thing I thought of (but never bothered to write about) is to >>>> introduce a separate operator instead, say ExternalPythonOperator (bike >>>> shedding on name is welcomed), that explicitly takes a path to the >>>> interpreter (say in a virtual environment) and just use that to run the >>>> code. This also enables users to create a virtual environment upfront, but >>>> avoids needing to overload PythonVirtualenvOperator for the purpose. This >>>> also opens an extra use case that you can use any Python installation to >>>> run the code (say a custom-compiled interpreter), although nobody asked >>>> about that. >>>> >>> >>>> >>> TP >>>> >>> >>>> >>> >>>> >>> On 13 Aug 2022, at 02:52, Jeambrun Pierre <pierrejb...@gmail.com> >>>> wrote: >>>> >>> >>>> >>> I feel like this is a great alternative at the price of a very >>>> moderate effort. (I'd be glad to help with it). >>>> >>> >>>> >>> Mutually exclusive sounds good to me as well. >>>> >>> >>>> >>> Best, >>>> >>> Pierre >>>> >>> >>>> >>> Le ven. 12 août 2022 à 15:23, Jarek Potiuk <ja...@potiuk.com> a >>>> écrit : >>>> >>>> >>>> >>>> >>>> >>>> Mutually exclusive. I think that has the nice property of forcing >>>> people to prepare immutable venvs upfront. >>>> >>>> >>>> >>>> On Fri, Aug 12, 2022 at 3:15 PM Ash Berlin-Taylor <a...@apache.org> >>>> wrote: >>>> >>>>> >>>> >>>>> >>>> >>>>> Yes, this has been on my background idea list for an age -- I'd >>>> love to see it happen! >>>> >>>>> >>>> >>>>> Have you thought about how it would behave when you specify an >>>> existing virtualenv and include requirements in the operator that are not >>>> already installed there? Or would they be mutually exclusive? (I don't mind >>>> either way, just wondering which way you are heading) >>>> >>>>> >>>> >>>>> -ash >>>> >>>>> >>>> >>>>> On Fri, Aug 12 2022 at 14:58:44 +02:00:00, Jarek Potiuk < >>>> ja...@potiuk.com> wrote: >>>> >>>>> >>>> >>>>> Hello everyone, >>>> >>>>> >>>> >>>>> TL;DR; I propose to extend our PythonVirtualenvOperator with >>>> "use existing venv" feature and make it a viable way of handling some >>>> multi-dependency sets using multiple pre-installed venvs. >>>> >>>>> >>>> >>>>> More context: >>>> >>>>> >>>> >>>>> I had this idea coming after a discussion in our Slack: >>>> https://apache-airflow.slack.com/archives/CCV3FV9KL/p1660233834355179 >>>> >>>>> >>>> >>>>> My thoughts were - why don't we add support for "use existing >>>> venv" in PythonVirtualenvOperator as first-class-citizen ? >>>> >>>>> >>>> >>>>> Currently (unless there are some tricks I am not aware of) or >>>> extend PVO, the PVO will always attempt to create a virtualenv based on >>>> extra requirements. And while it gives the users a possibility of having >>>> some tasks use different dependencies, the drawback is that the venv is >>>> created dynamically when tasks starts - potentially a lot of overhead for >>>> startup time and some unpleasant failure scenarios - like networking >>>> problems, PyPI or local repoi not available, automated (and unnoticed) >>>> upgrade of dependencies. >>>> >>>>> >>>> >>>>> Those are basically the same problems that caused us to strongly >>>> discourage our users in our Helm Chart to use _PIP_ADDITIONAL_DEPENDENCIES >>>> in production and criticize the Community Helm Chart for dynamic >>>> dependency installation they promote as a "valid" approach. Yet our PVO >>>> currently does exactly this. >>>> >>>>> >>>> >>>>> We had some past discussions how this can be improved - with >>>> caching, or using different images for different dependencies and similar - >>>> and even we have >>>> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-46+Runtime+isolation+for+airflow+tasks+and+dag+parsing >>>> proposal to use different images for different sets of requirements. >>>> >>>>> >>>> >>>>> Proposal: >>>> >>>>> >>>> >>>>> During the discussion yesterday I started to think a simpler >>>> solution is possible and rather simple to implement by us and for users to >>>> use. >>>> >>>>> >>>> >>>>> Why not have different venvs preinstalled and let the PVO choose >>>> the one that should be used? >>>> >>>>> >>>> >>>>> It does not invalidate AIP-46. AIP-46 serves a bit different >>>> purpose and some cases cannot be handled this way - when you need different >>>> "system level" dependencies for example) but it might be much simpler from >>>> deployment point of view and allow it to handle "multi-dependency sets" for >>>> Python libraries only with minimal deployment overhead (which AIP-46 >>>> necessarily has). And I think it will be enough for a vast number of the >>>> "multi-dependency-sets" cases. >>>> >>>>> >>>> >>>>> Why don't we allow the users to prepare those venvs upfront and >>>> simply enable PVE to use them rather than create them dynamically ? >>>> >>>>> >>>> >>>>> Advantages: >>>> >>>>> >>>> >>>>> * it nicely handles cases where some of your tasks need a >>>> different set of dependencies than others (for execution, not necessarily >>>> parsing at least initially). >>>> >>>>> >>>> >>>>> * no startup time overhead needed as with current PVO >>>> >>>>> >>>> >>>>> * possible to run in both cases - "venv installation" and >>>> "docker image" installation >>>> >>>>> >>>> >>>>> * it has finer granularity level than AIP-46 - unlike in AIP-46 >>>> you could use different sets of dependencies >>>> >>>>> >>>> >>>>> * very easy to pull off for the users without modifying their >>>> deployments,For local venv, you just create the venvs, For Docker image >>>> case, your custom image needs to add several lines similar to: >>>> >>>>> >>>> >>>>> RUN python -m venv --system-site-packages PACKAGE1==NN >>>> PACKAGE2==NN /opt/venv1 >>>> >>>>> RUN python -m venv --system-site-packages PACKAGE1==NN >>>> PACKAGE2==NN /opt/venv2 >>>> >>>>> >>>> >>>>> and PythonVenvOperator should have extra >>>> "use_existing_venv=/opt/venv2") parameter >>>> >>>>> >>>> >>>>> * we only need to manage ONE image (!) even if you have multiple >>>> sets of dependencies (this has the advantage that it is actually LOWER >>>> overhead than having separate images for each env -when it comes to various >>>> resources overhead (same workers could handle multiple dependency sets for >>>> examples, same image is reused by multiple PODs in K8S etc. ). >>>> >>>>> >>>> >>>>> * later (when AIP-43 (separate dag processor with ability to use >>>> different processors for different subdirectories) is completed and AIP-46 >>>> is approved/implemented, we could also extend DAG Parsing to be able to use >>>> those predefined venvs for parsing. That would eliminate the need for local >>>> imports and add support to even use different sets of libraries in >>>> top-level code (per DAG, not per task). It would not solve different >>>> "system" level dependencies - and for that AiP-46 is still a very valid >>>> case. >>>> >>>>> >>>> >>>>> Disadvantages: >>>> >>>>> >>>> >>>>> I thought very hard about this one and I actually could not find >>>> any disadvantages :) >>>> >>>>> >>>> >>>>> It's simple to implement, use and explain, it can be implemented >>>> very quickly (like - in a few hours with tests and documentation I think) >>>> and performance-wise it is better for any other solution (including AIP-46) >>>> providing that the case is limited to different Python dependencies. >>>> >>>>> >>>> >>>>> But possibly there are things that I missed. It all looks too >>>> good to be true, and I wonder why we do not have it already today - once I >>>> thought about it, it seems very obvious. So I probably missed something. >>>> >>>>> >>>> >>>>> WDYT? >>>> >>>>> >>>> >>>>> J. >>>> >>>>> >>>> >>>>> >>>> >>>>> >>>> >>>>> >>>> >>>>> >>>> >>>>> >>>> >>>>> >>>> >>> >>>> >>> -- >>> >>> Collin McNulty >>> Lead Airflow Engineer >>> >>> Email: col...@astronomer.io <john....@astronomer.io> >>> Time zone: US Central (CST UTC-6 / CDT UTC-5) >>> >>> >>> <https://www.astronomer.io/> >>> >>