I have kind of the done the same hack now, but I am starting this
discussion thread to see if we can make a feature out of this. As Ash said,
making scheduler run user code is against the design and I also had
realized it will make the scheduler thread slower. I will try to come up
with a different solution to implement this.

Does AIP need to have solution explained ? Can't it have just the problem
statement, so that it helps to track and it can be used for discussion

On Fri, Feb 14, 2020, 8:39 PM Shaw, Damian P. <
[email protected]> wrote:

> A possible (hacky?) workaround would be:
>
> 1. Create a DAG that is scheduled to run every 1 minute with
> max_active_runs=2 and catchup=False
> 2. As the first task in the DAG set as the sensor which needs wait on when
> the DAG in Airflow should be triggered and set task_concurrency to 1
> 3. As the second task (downstream from the 1st task) have the
> TriggerDagRunOperator
>
> This should work in the use case that you're not needing the triggered DAG
> to run more than once a minute. Also you will need to handle with care the
> sensor logic so the next sensor doesn't trigger the same Trigger DAG.
>
> Damian
>
> -----Original Message-----
> From: bharath palaksha <[email protected]>
> Sent: Friday, February 14, 2020 05:33
> To: [email protected]
> Subject: Re: Trigger based dag run
>
> 1. External system which i am thinking of doesn't have the ability to run
> code 2. If we use first task as sensor, how is the dag run getting created
> for the first task which is a sensor to run?
>
> Running user code on scheduler does seems problematic, can think of some
> other way for that. I will think about it
>
> Thanks,
> Bharath
>
>
> On Fri, Feb 14, 2020 at 3:57 PM Ash Berlin-Taylor <[email protected]> wrote:
>
> > If you have the ability to run code from the external system you might
> > want to consider using the ("experimental") API to trigger the dag run
> > from the external system?
> >
> >
> > http://airflow.apache.org/docs/stable/api.html#post--api-experimental-
> > dags--DAG_ID--dag_runs When using the API doesn't work for you the
> > common approach I have seen is as you hint at -- having a "trigger"
> > dag that runs (frequently depending on your needs), checks the
> > external condition and uses TriggerDagRunOperator.
> > The other way I have seen this done is to just have the first task of
> > your dag be a sensor that checks/waits on the external resource. With
> > the recently added "reschedule" mode of sensors this also doesn't tie
> > up a worker slot when the sensor isn't running. This is the approach I
> > have used in the past when processingly weekly datasets that would
> > appear anywhere in a 72 hour window after the expected delivery time.
> >
> > Given these options exist I'm not quite sure I see the need for a new
> > parameter to the DAG (especially one which runs user code in the
> > scheduler, that gets quite a strong no from me) Could you perhaps
> > explain your idea in more detail, specifically how it fits in to your
> > workflow, and why you don't want to use the two methods I talked about
> here?
> > Thanks,
> > Ash
> > On Feb 14 2020, at 10:10 am, bharath palaksha <[email protected]>
> > wrote:
> > > Hi,
> > >
> > > I have been using airflow extensively in my current work at Walmart
> Labs.
> > > while working on our requirements, came across a functionality which
> > > is missing in airflow and if implemented will be very useful.
> > > Currently, Airflow is a schedule based workflow management, a cron
> > > expression defines the creation of dag runs. If there is a
> > > dependency on
> > a
> > > different dag - TriggerDagRunOperator helps in creating dag runs.
> > > Suppose, there is a dependency which is outside of Airflow cluster eg:
> > > different database, filesystem or an event from an API which is an
> > upstream
> > > dependency. There is no way in Airflow to achieve this unless we
> > schedule a
> > > DAG for a a very short interval and allow it to poll.
> > >
> > > To solve above issue, what if airflow takes 2 different args -
> > > schedule_interval and trigger_sensor.
> > >
> > > - schedule_interval - works the same way as it is already working
> > > now
> > > - trigger_sensor - accepts a sensor which returns true when an event
> > > is sensed and this in turn creates a dag run
> > >
> > > If you specify both the argument, schedule_interval takes precedence.
> > > Scheduler parses all DAGs in a loop for every heartbeat and checks
> > > for
> > DAGs
> > > which has reached scheduled time and creates DAG run, same loop can
> > > also check for trigger_sensor and if argument is set - check if it
> > > returns
> > true
> > > to create dag run. This might slow down scheduler as it has to
> > > execute sensors now, we can find some other way to avoid slowness.
> > > Can we create AIP for this? Any thoughts?
> > >
> > > Thanks,
> > > Bharath
> > >
> >
> >
>
>
>
> ===============================================================================
>
> Please access the attached hyperlink for an important electronic
> communications disclaimer:
> http://www.credit-suisse.com/legal/en/disclaimer_email_ib.html
> ===============================================================================
>
>

Reply via email to