Yeah. I kinda like (and see it emerging from the discussion) that we can (which I love) have cake and eat it too :). Means - I think we can have both 1) and 2) ...
1) We should raise an error if someone uses AnOperator in task context (the way TP described it would work nicely) - making calling the `execute` pattern directly wrong 2) MAYBE we can figure out a way to actually allow the users to use the logic that Bolke described in a nice, and "supported" way. I would actually love it, if we find an easy way to make the 3500+ operators we have - immediately available to our taskflow users. I don't particularly like the idea of having a run_bash, run_xxxx etc. for all the 3500+ operators we have. But how about just: result = run_operator(BashOperator(.,...)) If we pair it with the error from point 1) (and tell the user "Hey, if you want to do that you need to do it this way: run_operator(...") and implement appropriate pre/post processing in run_operator logic, I see no reason why it would not work.. Yes - it is slightly ugly, but only slightly - and it's entirely NOT magical and explicit. You would not even have to run `execute()` directly. Heck we could likely implement the run_operator() in the way to nicely work for sensors and deferrable operators and properly implement handling of all the cases I **think**. And properly design the rules on what happens with xcom and task ids (because the question is whether such xcom should belong to the parent task id or maybe we should have it for "internal" task_id etc. etc. I think that would be super-powerful, as TP wrote - operator would mostly fade away - but not disappear entirely J. On Tue, Feb 27, 2024 at 10:06 AM Andrey Anshin <andrey.ans...@taragol.is> wrote: > I think manually using *execute*, *poke*, *choose_branch* and etc. should > be considered as an antipattern, these methods only should be invoked by > Airflow Worker. > You never know what should be done before, especially if it about > deferrable operator or sensors in reschedule mode > > > > > On Tue, 27 Feb 2024 at 12:30, Bolke de Bruin <bdbr...@gmail.com> wrote: > > > Interesting! Jarek and I had a bit of a discussion on slack a couple of > > months ago in the context of Operators Need to Die (if I recall > correctly). > > > > The idea behind Operators is that you have a vetted way of executing a > > certain logic apart from the boilerplate (i.e. execute, post_execute) > that > > makes things work as a Task. A TaskFlow task only provides the > boilerplate. > > So if we can separate out the business logic from the boilerplate and > > generalize it as such so that it becomes useful in both TaskFlow and > > classic Operators it would allow for a more flexible pattern in TaskFlow > > and if required a more regulated / strict pattern with Operators and thus > > being backwards compatible. > > > > Jarek argued for snake case calling of those library functions then like > so > > (paraphrasing): > > > > @task(task_id='some_id', provide_context=True) > > def some_dummy_task(**context) -> int: > > ti = context['ti'] > > cmd2 = 'echo "pushing 2"' > > return run_bash(cmd=cmd2) > > > > There is little logic in the BashOperator that would prevent us from > > rewiring it so that it also calls out run_bash_command while keeping the > > same signature. > > > > Naming could be argued upon. It could be a standard pattern like: > > > > BashOperator -> run_bash() or just bash() > > EmailOperator -> run_email or email() > > > > I *think* I prefer a library pattern for this rather than hooks. Hooks > just > > don't feel entirely right. They are primarily there to manage > connections. > > In that way I can imagine that they register themselves upon usage for > > `on_kill` or something like that to ensure proper resource management. > > Maybe a HookMixin that gets autowired into the library function so that > > certain things get auto registered (Bash has the subprocess thing, this > can > > be generalized). > > > > Anyways with the above described pattern I like option 1 more because it > > distinguishes between opinionated / strict / classic and flexible. > > > > my 2 cents, > > Bolke > > > > > > On Tue, 27 Feb 2024 at 08:33, Tzu-ping Chung <t...@astronomer.io.invalid> > > wrote: > > > > > This kind of ties back to Bolke’s Operator Must Die manifesto—you > > > shouldn’t use the operator class here at all in this situation, but the > > > corresponding hook instead. > > > > > > Regarding preventing this anti-pattern, I can think of two ways: > > > > > > Add a check in BaseOperator to detect whether __init__ is called in a > > task > > > work context, and error out guiding users to use hooks instead. > > > Accept this as a valid pattern, and actually push to XCom. > > > > > > Personally I like options 1 more. Option 2 is admittedly less > disruptive, > > > but I’m pretty confident people will find other ways to use operators > > that > > > do not work. Operators will never die, but they should fade away. < > > > https://en.wikipedia.org/wiki/Old_soldiers_never_die> > > > > > > TP > > > > > > > > > > On 27 Feb 2024, at 15:09, Jarek Potiuk <ja...@potiuk.com> wrote: > > > > > > > > Hello here, > > > > > > > > I have seen recently at least a few times that our users started to > > use a > > > > strange pattern > > > > > > > > @task(task_id='some_id', provide_context=True) > > > > def some_dummy_task(**context): > > > > ti = context['ti'] > > > > cmd2 = 'echo "pushing 2"' > > > > dbt_task = BashOperator(task_id='some_dummy_task_id', > > > > bash_command=cmd2, > > > > do_xcom_push=True) > > > > dbt_task.execute(context)` > > > > > > > > This **works** for them but then - for example - they complain that > > xcom > > > is > > > > not pushed (well of course it's not if you use operator and manually > > run > > > > execute). > > > > > > > > Now - while this is a bad pattern - obviously - it seems that our > users > > > > **think** they can do itl. And maybe we can do something to prevent > > them > > > > shooting themselves in their foot? > > > > > > > > I do not know that well the @task decoration magic, but maybe we > could > > > > somehow detect the case where someone instantiates the operator (and > > > runs > > > > execute) inside a decorated task and give a helpful error in this > > case? I > > > > am afraid people will start using it more and more and the sooner we > > add > > > > protection against it, the better chance we have to contain it. > > > > > > > > J. > > > > > > > > > > -- > > > > -- > > Bolke de Bruin > > bdbr...@gmail.com > > >