You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@airflow.apache.org by abhishek sharma <ab...@gmail.com> on 2019/01/31 16:28:46 UTC

Custom scheduler support in Airflow

Hi All,

Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775) for
supporting custom scheduler in Airflow.

The idea is to have a scheduler base class which can be extended for
writing a custom scheduler. The logic of custom scheduling is user
specific, and at the DAGs task level we can mention the scheduler type, and
that scheduler will be used for starting a task. [Naive Idea]

Can we please discuss whether we need such functionality in Airflow or not
and If yes then we will proceed with the design and implementation.

Thanks
Abhishek Sharma

Re: Custom scheduler support in Airflow

Posted by Tao Feng <fe...@gmail.com>.
If I understand your request correctly, I don't think you need a custom
scheduler but have a custom way to create dag run which is non-time based?
Have you looked into TriggerDagRunOperator (
https://github.com/apache/airflow/blob/master/airflow/operators/dagrun_operator.py#L36
)?


On Thu, Jan 31, 2019 at 1:04 PM abhishek sharma <ab...@gmail.com>
wrote:

> Thanks, Brian & Ben.
>
> So, you guys also have such workflows and through Sensors or running DAGs
> frequently things are working out for your guys. In my case, I am running
> an application which works as a 'custom scheduler' and triggers DAGs based
> on event occurrence.
>
> Question to you guys, wouldn't it be easy if we have some custom scheduler
> support in Airflow? Also, I think that would open more possibilities for
> scheduling  DAGs.
>
> Thanks, {{Abhishek}}
>
> On Thu, Jan 31, 2019 at 3:25 PM Ben Tallman <bt...@gmail.com> wrote:
>
> > To solve that exact problem, we ran a DAG on a frequent schedule, that
> > basically acted as a scheduler. It used a shell script to kick off other
> > DAGS. Possibly a custom scheduler would be a more elegant solution.
> >
> > Thanks,
> > Ben
> >
> > --
> > Ben Tallman - 503.680.5709
> >
> >
> > On Thu, Jan 31, 2019 at 11:03 AM abhishek sharma <
> > abhioncbr.apache@gmail.com>
> > wrote:
> >
> > > Hi Ben,
> > >
> > > Just copying my comment form ticket.
> > >
> > > I think current airflow scheduler schedule DAGs only on time-basis
> (based
> > > on cron schedule string). ***Is it correct understanding?*
> > >
> > > How to approach a scenario where I want to trigger a DAG based on some
> > > event which is not so predictable/regular on time basis?
> > >
> > >    - One is to use sensor and DAG flow will start by first running a
> > sensor
> > >    task for checking an event and when an event happened actual
> > processing
> > > get
> > >    a start.
> > >    - Second, is to have a DAG with None schedule and it gets triggered
> by
> > >    some other application or utility which checks for an event
> > occurrence .
> > >
> > > If your most of the flow(DAGs) are suppose to run on this logic, then
> > > sensor doesn't make sense and hence left with the second approach only,
> > > which is nothing but a custom scheduling of DAGs.  ****Is it agreeable
> > use
> > > case for the custom scheduler?*
> > >
> > > Thanks, {{Abhishek}}
> > >
> > >
> > >
> > > On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com>
> wrote:
> > >
> > > > Can you explain a bit more what you are thinking for a custom
> > scheduler?
> > > > It's been awhile, but we added support for cron schedules without
> > > backfill
> > > > awhile back, so I'm wondering what you are thinking of adding with
> > this?
> > > >
> > > > Thanks,
> > > > Ben
> > > >
> > > > --
> > > > Ben Tallman - 503.680.5709
> > > >
> > > >
> > > > On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
> > > > abhioncbr.apache@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi All,
> > > > >
> > > > > Created a ticket(
> https://issues.apache.org/jira/browse/AIRFLOW-3775)
> > > for
> > > > > supporting custom scheduler in Airflow.
> > > > >
> > > > > The idea is to have a scheduler base class which can be extended
> for
> > > > > writing a custom scheduler. The logic of custom scheduling is user
> > > > > specific, and at the DAGs task level we can mention the scheduler
> > type,
> > > > and
> > > > > that scheduler will be used for starting a task. [Naive Idea]
> > > > >
> > > > > Can we please discuss whether we need such functionality in Airflow
> > or
> > > > not
> > > > > and If yes then we will proceed with the design and implementation.
> > > > >
> > > > > Thanks
> > > > > Abhishek Sharma
> > > > >
> > > >
> > >
> >
>

Re: Custom scheduler support in Airflow

Posted by Brian Greene <br...@heisenbergwoodworking.com>.
I’d agree with Ash as well - and the externally triggered Dag model works well and still allows you to use airflow for “normal” scheduled tasks.

Admittedly we struggled with this for a while, working really hard to use schedules and xcom etc..  this is really “state management” in my opinion, and the “state” is the trigger time.  Time is the “event” that starts the dag.

Given that in our solution we’re storing state authoritatively elsewhere, it makes MORE sense to ignore airflow’s state management in favor of externally controlling it.

Af still does all its wonderful magic for those dags (retries, callbacks, great integration across a truly diverse platform space...), but the triggering events just don’t come from the AF scheduler.

This has a side benefit of being able to use external logic to “reprocess” for a given event.

I actually see “our” use case often on the list - “my data is uploaded into directories, which should be my partitions, but it doesn’t always come in on time/in order.  We eventually switched to this external model, and the events that trigger a dag are based on aws lambda functions generating file upload events when the data lands.  This then gives us the place to determine when we should run the dag.

Ymmv 

Brian



Sent from a device with less than stellar autocorrect

On Jan 31, 2019, at 2:27 PM, abhishek sharma <ab...@gmail.com> wrote:

>> Have you looked into TriggerDagRunOperator [@Tao], I am aware of it, and
> in my opinion, it's same if you use an operator or invoking trigger from
> the command line.
> 
> @Ash, I agree that this is not an easy task and it requires a lot of
> effort. As I said previously; *first we have to decide whether we need this
> feature in Airflow or not?*
> 
> Thanks
> 
> On Thu, Jan 31, 2019 at 4:44 PM Ash Berlin-Taylor <as...@apache.org> wrote:
> 
>>> wouldn't it be easy if we have some custom scheduler support in Airflow
>> 
>> Don't underestimate JUST how much work this would actually involve.
>> 
>> Right now given the solutions presented, and the ability to trigger DAGs
>> in Airflow via the existing API I am not convinced that Airflow needs the
>> added complexity this change would involve.
>> 
>> -ash
>> 
>> 
>>> On 31 Jan 2019, at 21:04, abhishek sharma <ab...@gmail.com>
>> wrote:
>>> 
>>> Thanks, Brian & Ben.
>>> 
>>> So, you guys also have such workflows and through Sensors or running DAGs
>>> frequently things are working out for your guys. In my case, I am running
>>> an application which works as a 'custom scheduler' and triggers DAGs
>> based
>>> on event occurrence.
>>> 
>>> Question to you guys, wouldn't it be easy if we have some custom
>> scheduler
>>> support in Airflow? Also, I think that would open more possibilities for
>>> scheduling  DAGs.
>>> 
>>> Thanks, {{Abhishek}}
>>> 
>>>> On Thu, Jan 31, 2019 at 3:25 PM Ben Tallman <bt...@gmail.com> wrote:
>>>> 
>>>> To solve that exact problem, we ran a DAG on a frequent schedule, that
>>>> basically acted as a scheduler. It used a shell script to kick off other
>>>> DAGS. Possibly a custom scheduler would be a more elegant solution.
>>>> 
>>>> Thanks,
>>>> Ben
>>>> 
>>>> --
>>>> Ben Tallman - 503.680.5709
>>>> 
>>>> 
>>>> On Thu, Jan 31, 2019 at 11:03 AM abhishek sharma <
>>>> abhioncbr.apache@gmail.com>
>>>> wrote:
>>>> 
>>>>> Hi Ben,
>>>>> 
>>>>> Just copying my comment form ticket.
>>>>> 
>>>>> I think current airflow scheduler schedule DAGs only on time-basis
>> (based
>>>>> on cron schedule string). ***Is it correct understanding?*
>>>>> 
>>>>> How to approach a scenario where I want to trigger a DAG based on some
>>>>> event which is not so predictable/regular on time basis?
>>>>> 
>>>>>  - One is to use sensor and DAG flow will start by first running a
>>>> sensor
>>>>>  task for checking an event and when an event happened actual
>>>> processing
>>>>> get
>>>>>  a start.
>>>>>  - Second, is to have a DAG with None schedule and it gets triggered
>> by
>>>>>  some other application or utility which checks for an event
>>>> occurrence .
>>>>> 
>>>>> If your most of the flow(DAGs) are suppose to run on this logic, then
>>>>> sensor doesn't make sense and hence left with the second approach only,
>>>>> which is nothing but a custom scheduling of DAGs.  ****Is it agreeable
>>>> use
>>>>> case for the custom scheduler?*
>>>>> 
>>>>> Thanks, {{Abhishek}}
>>>>> 
>>>>> 
>>>>> 
>>>>> On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com>
>> wrote:
>>>>> 
>>>>>> Can you explain a bit more what you are thinking for a custom
>>>> scheduler?
>>>>>> It's been awhile, but we added support for cron schedules without
>>>>> backfill
>>>>>> awhile back, so I'm wondering what you are thinking of adding with
>>>> this?
>>>>>> 
>>>>>> Thanks,
>>>>>> Ben
>>>>>> 
>>>>>> --
>>>>>> Ben Tallman - 503.680.5709
>>>>>> 
>>>>>> 
>>>>>> On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
>>>>>> abhioncbr.apache@gmail.com>
>>>>>> wrote:
>>>>>> 
>>>>>>> Hi All,
>>>>>>> 
>>>>>>> Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775)
>>>>> for
>>>>>>> supporting custom scheduler in Airflow.
>>>>>>> 
>>>>>>> The idea is to have a scheduler base class which can be extended for
>>>>>>> writing a custom scheduler. The logic of custom scheduling is user
>>>>>>> specific, and at the DAGs task level we can mention the scheduler
>>>> type,
>>>>>> and
>>>>>>> that scheduler will be used for starting a task. [Naive Idea]
>>>>>>> 
>>>>>>> Can we please discuss whether we need such functionality in Airflow
>>>> or
>>>>>> not
>>>>>>> and If yes then we will proceed with the design and implementation.
>>>>>>> 
>>>>>>> Thanks
>>>>>>> Abhishek Sharma
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> 
>> 

Re: Custom scheduler support in Airflow

Posted by abhishek sharma <ab...@gmail.com>.
> Have you looked into TriggerDagRunOperator [@Tao], I am aware of it, and
in my opinion, it's same if you use an operator or invoking trigger from
the command line.

@Ash, I agree that this is not an easy task and it requires a lot of
effort. As I said previously; *first we have to decide whether we need this
feature in Airflow or not?*

Thanks

On Thu, Jan 31, 2019 at 4:44 PM Ash Berlin-Taylor <as...@apache.org> wrote:

> >  wouldn't it be easy if we have some custom scheduler support in Airflow
>
> Don't underestimate JUST how much work this would actually involve.
>
> Right now given the solutions presented, and the ability to trigger DAGs
> in Airflow via the existing API I am not convinced that Airflow needs the
> added complexity this change would involve.
>
> -ash
>
>
> > On 31 Jan 2019, at 21:04, abhishek sharma <ab...@gmail.com>
> wrote:
> >
> > Thanks, Brian & Ben.
> >
> > So, you guys also have such workflows and through Sensors or running DAGs
> > frequently things are working out for your guys. In my case, I am running
> > an application which works as a 'custom scheduler' and triggers DAGs
> based
> > on event occurrence.
> >
> > Question to you guys, wouldn't it be easy if we have some custom
> scheduler
> > support in Airflow? Also, I think that would open more possibilities for
> > scheduling  DAGs.
> >
> > Thanks, {{Abhishek}}
> >
> > On Thu, Jan 31, 2019 at 3:25 PM Ben Tallman <bt...@gmail.com> wrote:
> >
> >> To solve that exact problem, we ran a DAG on a frequent schedule, that
> >> basically acted as a scheduler. It used a shell script to kick off other
> >> DAGS. Possibly a custom scheduler would be a more elegant solution.
> >>
> >> Thanks,
> >> Ben
> >>
> >> --
> >> Ben Tallman - 503.680.5709
> >>
> >>
> >> On Thu, Jan 31, 2019 at 11:03 AM abhishek sharma <
> >> abhioncbr.apache@gmail.com>
> >> wrote:
> >>
> >>> Hi Ben,
> >>>
> >>> Just copying my comment form ticket.
> >>>
> >>> I think current airflow scheduler schedule DAGs only on time-basis
> (based
> >>> on cron schedule string). ***Is it correct understanding?*
> >>>
> >>> How to approach a scenario where I want to trigger a DAG based on some
> >>> event which is not so predictable/regular on time basis?
> >>>
> >>>   - One is to use sensor and DAG flow will start by first running a
> >> sensor
> >>>   task for checking an event and when an event happened actual
> >> processing
> >>> get
> >>>   a start.
> >>>   - Second, is to have a DAG with None schedule and it gets triggered
> by
> >>>   some other application or utility which checks for an event
> >> occurrence .
> >>>
> >>> If your most of the flow(DAGs) are suppose to run on this logic, then
> >>> sensor doesn't make sense and hence left with the second approach only,
> >>> which is nothing but a custom scheduling of DAGs.  ****Is it agreeable
> >> use
> >>> case for the custom scheduler?*
> >>>
> >>> Thanks, {{Abhishek}}
> >>>
> >>>
> >>>
> >>> On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com>
> wrote:
> >>>
> >>>> Can you explain a bit more what you are thinking for a custom
> >> scheduler?
> >>>> It's been awhile, but we added support for cron schedules without
> >>> backfill
> >>>> awhile back, so I'm wondering what you are thinking of adding with
> >> this?
> >>>>
> >>>> Thanks,
> >>>> Ben
> >>>>
> >>>> --
> >>>> Ben Tallman - 503.680.5709
> >>>>
> >>>>
> >>>> On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
> >>>> abhioncbr.apache@gmail.com>
> >>>> wrote:
> >>>>
> >>>>> Hi All,
> >>>>>
> >>>>> Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775)
> >>> for
> >>>>> supporting custom scheduler in Airflow.
> >>>>>
> >>>>> The idea is to have a scheduler base class which can be extended for
> >>>>> writing a custom scheduler. The logic of custom scheduling is user
> >>>>> specific, and at the DAGs task level we can mention the scheduler
> >> type,
> >>>> and
> >>>>> that scheduler will be used for starting a task. [Naive Idea]
> >>>>>
> >>>>> Can we please discuss whether we need such functionality in Airflow
> >> or
> >>>> not
> >>>>> and If yes then we will proceed with the design and implementation.
> >>>>>
> >>>>> Thanks
> >>>>> Abhishek Sharma
> >>>>>
> >>>>
> >>>
> >>
>
>

Re: Custom scheduler support in Airflow

Posted by Ash Berlin-Taylor <as...@apache.org>.
>  wouldn't it be easy if we have some custom scheduler support in Airflow

Don't underestimate JUST how much work this would actually involve.

Right now given the solutions presented, and the ability to trigger DAGs in Airflow via the existing API I am not convinced that Airflow needs the added complexity this change would involve.

-ash


> On 31 Jan 2019, at 21:04, abhishek sharma <ab...@gmail.com> wrote:
> 
> Thanks, Brian & Ben.
> 
> So, you guys also have such workflows and through Sensors or running DAGs
> frequently things are working out for your guys. In my case, I am running
> an application which works as a 'custom scheduler' and triggers DAGs based
> on event occurrence.
> 
> Question to you guys, wouldn't it be easy if we have some custom scheduler
> support in Airflow? Also, I think that would open more possibilities for
> scheduling  DAGs.
> 
> Thanks, {{Abhishek}}
> 
> On Thu, Jan 31, 2019 at 3:25 PM Ben Tallman <bt...@gmail.com> wrote:
> 
>> To solve that exact problem, we ran a DAG on a frequent schedule, that
>> basically acted as a scheduler. It used a shell script to kick off other
>> DAGS. Possibly a custom scheduler would be a more elegant solution.
>> 
>> Thanks,
>> Ben
>> 
>> --
>> Ben Tallman - 503.680.5709
>> 
>> 
>> On Thu, Jan 31, 2019 at 11:03 AM abhishek sharma <
>> abhioncbr.apache@gmail.com>
>> wrote:
>> 
>>> Hi Ben,
>>> 
>>> Just copying my comment form ticket.
>>> 
>>> I think current airflow scheduler schedule DAGs only on time-basis (based
>>> on cron schedule string). ***Is it correct understanding?*
>>> 
>>> How to approach a scenario where I want to trigger a DAG based on some
>>> event which is not so predictable/regular on time basis?
>>> 
>>>   - One is to use sensor and DAG flow will start by first running a
>> sensor
>>>   task for checking an event and when an event happened actual
>> processing
>>> get
>>>   a start.
>>>   - Second, is to have a DAG with None schedule and it gets triggered by
>>>   some other application or utility which checks for an event
>> occurrence .
>>> 
>>> If your most of the flow(DAGs) are suppose to run on this logic, then
>>> sensor doesn't make sense and hence left with the second approach only,
>>> which is nothing but a custom scheduling of DAGs.  ****Is it agreeable
>> use
>>> case for the custom scheduler?*
>>> 
>>> Thanks, {{Abhishek}}
>>> 
>>> 
>>> 
>>> On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com> wrote:
>>> 
>>>> Can you explain a bit more what you are thinking for a custom
>> scheduler?
>>>> It's been awhile, but we added support for cron schedules without
>>> backfill
>>>> awhile back, so I'm wondering what you are thinking of adding with
>> this?
>>>> 
>>>> Thanks,
>>>> Ben
>>>> 
>>>> --
>>>> Ben Tallman - 503.680.5709
>>>> 
>>>> 
>>>> On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
>>>> abhioncbr.apache@gmail.com>
>>>> wrote:
>>>> 
>>>>> Hi All,
>>>>> 
>>>>> Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775)
>>> for
>>>>> supporting custom scheduler in Airflow.
>>>>> 
>>>>> The idea is to have a scheduler base class which can be extended for
>>>>> writing a custom scheduler. The logic of custom scheduling is user
>>>>> specific, and at the DAGs task level we can mention the scheduler
>> type,
>>>> and
>>>>> that scheduler will be used for starting a task. [Naive Idea]
>>>>> 
>>>>> Can we please discuss whether we need such functionality in Airflow
>> or
>>>> not
>>>>> and If yes then we will proceed with the design and implementation.
>>>>> 
>>>>> Thanks
>>>>> Abhishek Sharma
>>>>> 
>>>> 
>>> 
>> 


Re: Custom scheduler support in Airflow

Posted by abhishek sharma <ab...@gmail.com>.
Thanks, Brian & Ben.

So, you guys also have such workflows and through Sensors or running DAGs
frequently things are working out for your guys. In my case, I am running
an application which works as a 'custom scheduler' and triggers DAGs based
on event occurrence.

Question to you guys, wouldn't it be easy if we have some custom scheduler
support in Airflow? Also, I think that would open more possibilities for
scheduling  DAGs.

Thanks, {{Abhishek}}

On Thu, Jan 31, 2019 at 3:25 PM Ben Tallman <bt...@gmail.com> wrote:

> To solve that exact problem, we ran a DAG on a frequent schedule, that
> basically acted as a scheduler. It used a shell script to kick off other
> DAGS. Possibly a custom scheduler would be a more elegant solution.
>
> Thanks,
> Ben
>
> --
> Ben Tallman - 503.680.5709
>
>
> On Thu, Jan 31, 2019 at 11:03 AM abhishek sharma <
> abhioncbr.apache@gmail.com>
> wrote:
>
> > Hi Ben,
> >
> > Just copying my comment form ticket.
> >
> > I think current airflow scheduler schedule DAGs only on time-basis (based
> > on cron schedule string). ***Is it correct understanding?*
> >
> > How to approach a scenario where I want to trigger a DAG based on some
> > event which is not so predictable/regular on time basis?
> >
> >    - One is to use sensor and DAG flow will start by first running a
> sensor
> >    task for checking an event and when an event happened actual
> processing
> > get
> >    a start.
> >    - Second, is to have a DAG with None schedule and it gets triggered by
> >    some other application or utility which checks for an event
> occurrence .
> >
> > If your most of the flow(DAGs) are suppose to run on this logic, then
> > sensor doesn't make sense and hence left with the second approach only,
> > which is nothing but a custom scheduling of DAGs.  ****Is it agreeable
> use
> > case for the custom scheduler?*
> >
> > Thanks, {{Abhishek}}
> >
> >
> >
> > On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com> wrote:
> >
> > > Can you explain a bit more what you are thinking for a custom
> scheduler?
> > > It's been awhile, but we added support for cron schedules without
> > backfill
> > > awhile back, so I'm wondering what you are thinking of adding with
> this?
> > >
> > > Thanks,
> > > Ben
> > >
> > > --
> > > Ben Tallman - 503.680.5709
> > >
> > >
> > > On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
> > > abhioncbr.apache@gmail.com>
> > > wrote:
> > >
> > > > Hi All,
> > > >
> > > > Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775)
> > for
> > > > supporting custom scheduler in Airflow.
> > > >
> > > > The idea is to have a scheduler base class which can be extended for
> > > > writing a custom scheduler. The logic of custom scheduling is user
> > > > specific, and at the DAGs task level we can mention the scheduler
> type,
> > > and
> > > > that scheduler will be used for starting a task. [Naive Idea]
> > > >
> > > > Can we please discuss whether we need such functionality in Airflow
> or
> > > not
> > > > and If yes then we will proceed with the design and implementation.
> > > >
> > > > Thanks
> > > > Abhishek Sharma
> > > >
> > >
> >
>

Re: Custom scheduler support in Airflow

Posted by Ben Tallman <bt...@gmail.com>.
To solve that exact problem, we ran a DAG on a frequent schedule, that
basically acted as a scheduler. It used a shell script to kick off other
DAGS. Possibly a custom scheduler would be a more elegant solution.

Thanks,
Ben

--
Ben Tallman - 503.680.5709


On Thu, Jan 31, 2019 at 11:03 AM abhishek sharma <ab...@gmail.com>
wrote:

> Hi Ben,
>
> Just copying my comment form ticket.
>
> I think current airflow scheduler schedule DAGs only on time-basis (based
> on cron schedule string). ***Is it correct understanding?*
>
> How to approach a scenario where I want to trigger a DAG based on some
> event which is not so predictable/regular on time basis?
>
>    - One is to use sensor and DAG flow will start by first running a sensor
>    task for checking an event and when an event happened actual processing
> get
>    a start.
>    - Second, is to have a DAG with None schedule and it gets triggered by
>    some other application or utility which checks for an event occurrence .
>
> If your most of the flow(DAGs) are suppose to run on this logic, then
> sensor doesn't make sense and hence left with the second approach only,
> which is nothing but a custom scheduling of DAGs.  ****Is it agreeable use
> case for the custom scheduler?*
>
> Thanks, {{Abhishek}}
>
>
>
> On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com> wrote:
>
> > Can you explain a bit more what you are thinking for a custom scheduler?
> > It's been awhile, but we added support for cron schedules without
> backfill
> > awhile back, so I'm wondering what you are thinking of adding with this?
> >
> > Thanks,
> > Ben
> >
> > --
> > Ben Tallman - 503.680.5709
> >
> >
> > On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
> > abhioncbr.apache@gmail.com>
> > wrote:
> >
> > > Hi All,
> > >
> > > Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775)
> for
> > > supporting custom scheduler in Airflow.
> > >
> > > The idea is to have a scheduler base class which can be extended for
> > > writing a custom scheduler. The logic of custom scheduling is user
> > > specific, and at the DAGs task level we can mention the scheduler type,
> > and
> > > that scheduler will be used for starting a task. [Naive Idea]
> > >
> > > Can we please discuss whether we need such functionality in Airflow or
> > not
> > > and If yes then we will proceed with the design and implementation.
> > >
> > > Thanks
> > > Abhishek Sharma
> > >
> >
>

Re: Custom scheduler support in Airflow

Posted by Brian Greene <br...@heisenbergwoodworking.com>.
This is actually the majority of our airflow work.

We use this pattern:
Sensor pings api (fairly quickly, in a dag that’s constrained to only run one instance, every minute)

If the sensor gets a valid response, the next task is a custom operator that extends Trigger, builds up the DagRun context, and triggers the actual processing dag.

This has worked very well, and doesn’t require a custom scheduler.  Additionally, the “trigger dags” are fairly light on resources- 1 active sensor each.

Sent from a device with less than stellar autocorrect

> On Jan 31, 2019, at 10:55 AM, abhishek sharma <ab...@gmail.com> wrote:
> 
> Hi Ben,
> 
> Just copying my comment form ticket.
> 
> I think current airflow scheduler schedule DAGs only on time-basis (based
> on cron schedule string). ***Is it correct understanding?*
> 
> How to approach a scenario where I want to trigger a DAG based on some
> event which is not so predictable/regular on time basis?
> 
>   - One is to use sensor and DAG flow will start by first running a sensor
>   task for checking an event and when an event happened actual processing get
>   a start.
>   - Second, is to have a DAG with None schedule and it gets triggered by
>   some other application or utility which checks for an event occurrence .
> 
> If your most of the flow(DAGs) are suppose to run on this logic, then
> sensor doesn't make sense and hence left with the second approach only,
> which is nothing but a custom scheduling of DAGs.  ****Is it agreeable use
> case for the custom scheduler?*
> 
> Thanks, {{Abhishek}}
> 
> 
> 
>> On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com> wrote:
>> 
>> Can you explain a bit more what you are thinking for a custom scheduler?
>> It's been awhile, but we added support for cron schedules without backfill
>> awhile back, so I'm wondering what you are thinking of adding with this?
>> 
>> Thanks,
>> Ben
>> 
>> --
>> Ben Tallman - 503.680.5709
>> 
>> 
>> On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
>> abhioncbr.apache@gmail.com>
>> wrote:
>> 
>>> Hi All,
>>> 
>>> Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775) for
>>> supporting custom scheduler in Airflow.
>>> 
>>> The idea is to have a scheduler base class which can be extended for
>>> writing a custom scheduler. The logic of custom scheduling is user
>>> specific, and at the DAGs task level we can mention the scheduler type,
>> and
>>> that scheduler will be used for starting a task. [Naive Idea]
>>> 
>>> Can we please discuss whether we need such functionality in Airflow or
>> not
>>> and If yes then we will proceed with the design and implementation.
>>> 
>>> Thanks
>>> Abhishek Sharma
>>> 
>> 

Re: Custom scheduler support in Airflow

Posted by abhishek sharma <ab...@gmail.com>.
Hi Ben,

Just copying my comment form ticket.

I think current airflow scheduler schedule DAGs only on time-basis (based
on cron schedule string). ***Is it correct understanding?*

How to approach a scenario where I want to trigger a DAG based on some
event which is not so predictable/regular on time basis?

   - One is to use sensor and DAG flow will start by first running a sensor
   task for checking an event and when an event happened actual processing get
   a start.
   - Second, is to have a DAG with None schedule and it gets triggered by
   some other application or utility which checks for an event occurrence .

If your most of the flow(DAGs) are suppose to run on this logic, then
sensor doesn't make sense and hence left with the second approach only,
which is nothing but a custom scheduling of DAGs.  ****Is it agreeable use
case for the custom scheduler?*

Thanks, {{Abhishek}}



On Thu, Jan 31, 2019 at 11:53 AM Ben Tallman <bt...@gmail.com> wrote:

> Can you explain a bit more what you are thinking for a custom scheduler?
> It's been awhile, but we added support for cron schedules without backfill
> awhile back, so I'm wondering what you are thinking of adding with this?
>
> Thanks,
> Ben
>
> --
> Ben Tallman - 503.680.5709
>
>
> On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <
> abhioncbr.apache@gmail.com>
> wrote:
>
> > Hi All,
> >
> > Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775) for
> > supporting custom scheduler in Airflow.
> >
> > The idea is to have a scheduler base class which can be extended for
> > writing a custom scheduler. The logic of custom scheduling is user
> > specific, and at the DAGs task level we can mention the scheduler type,
> and
> > that scheduler will be used for starting a task. [Naive Idea]
> >
> > Can we please discuss whether we need such functionality in Airflow or
> not
> > and If yes then we will proceed with the design and implementation.
> >
> > Thanks
> > Abhishek Sharma
> >
>

Re: Custom scheduler support in Airflow

Posted by Ben Tallman <bt...@gmail.com>.
Can you explain a bit more what you are thinking for a custom scheduler?
It's been awhile, but we added support for cron schedules without backfill
awhile back, so I'm wondering what you are thinking of adding with this?

Thanks,
Ben

--
Ben Tallman - 503.680.5709


On Thu, Jan 31, 2019 at 8:29 AM abhishek sharma <ab...@gmail.com>
wrote:

> Hi All,
>
> Created a ticket(https://issues.apache.org/jira/browse/AIRFLOW-3775) for
> supporting custom scheduler in Airflow.
>
> The idea is to have a scheduler base class which can be extended for
> writing a custom scheduler. The logic of custom scheduling is user
> specific, and at the DAGs task level we can mention the scheduler type, and
> that scheduler will be used for starting a task. [Naive Idea]
>
> Can we please discuss whether we need such functionality in Airflow or not
> and If yes then we will proceed with the design and implementation.
>
> Thanks
> Abhishek Sharma
>