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

Reply via email to