You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ignite.apache.org by Anton Vinogradov <av...@apache.org> on 2018/05/04 13:03:57 UTC

Re: Replace Cron4J with Quartz for ignite-schedule module.

Folks,

How can it be at PATCH AVAILABLE since *none* of my latest comments (made
Feb 8) are resolved at Upsource?
Changed state to IP.

пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <dp...@gmail.com>:

> Hi Andrey,
>
> Could you please pick up review?
>
> Sincerely,
> Dmitriy Pavlov
>
> пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <ds...@apache.org>:
>
> > Dmitriy, who is a good candidate within the community to review this
> > ticket?
> >
> > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <dp...@gmail.com>
> > wrote:
> >
> > > Hi Igniters,
> > >
> > > it seems ticket https://issues.apache.org/jira/browse/IGNITE-5565 is
> > still
> > > in PA state. What are our next steps?
> > >
> > > Who did review of this patch?
> > >
> > > Sincerely,
> > > Dmitriy Pavlov
> > >
> > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <dm...@apache.org>:
> > >
> > > > Yakov,
> > > >
> > > > No, the mentioned discussion didn’t turn into a JIRA ticket.
> > > >
> > > > Alex K., please follow to some thoughts from there and wrap them up
> in
> > a
> > > > form of the ticket.
> > > >
> > > > —
> > > > Denis
> > > >
> > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <yz...@apache.org>
> > > wrote:
> > > > >
> > > > > Guys, I remember we discussed this some time ago.
> > > > >
> > > > >
> > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > >
> > > > > Denis, do you have any ticket or SoW?
> > > > >
> > > > > --Yakov
> > > >
> > > >
> > >
> >
>

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Ilya Kasnacheev <il...@gmail.com>.
Hello!

In any case I don't see why we would keep ignite-schedule module. We could
merge this change if you moved this functionality into ignite-spring (while
also fixing single thread problem).

The sole reason of existence of ignite-schedule module was the extra
dependency, if we don't have it anymore there's no reason to have this tiny
module.

Regards,
-- 
Ilya Kasnacheev


пт, 28 дек. 2018 г. в 17:18, Sergey <ma...@gmail.com>:

> Ilya,
> before deciding about drop just don't forget that our scheduleLocal method
> extends functionality with optional prefix before standard Cron expression
> {n1,
> n2} where n1 is delay of scheduling and n2 is number of executions.
>
> Also we have own protection against task that has not finished before new
> execution:
> Look at
> org.apache.ignite.internal.processors.schedule.ScheduleFutureImpl#run
> In this case  a new task won't be executed (it just log warning message)
> disregarding Cron4J or Spring is used.
>
> Best regards,
> Sergey Kosarev.
>
>
> пт, 28 дек. 2018 г. в 13:51, Ilya Kasnacheev <il...@gmail.com>:
>
> > Hello!
> >
> > I've spent considerable amount of time working on your patch yesterday,
> but
> > eventually I have come to conclusion that we should Deprecate
> > IgniteScheduler and ignite-schedule module, and Remove it in 3.0.
> >
> > The reason for this: IgnoteScheduler interface has 5 methods, of those
> > callLocal() and runLocal() should probably be moved to IgniteCompute,
> > runLocal() with timeout should be moved somewhere elsewhere, and
> > scheduleLocal() which we are discussing should be removed since it does
> not
> > offer anything that you can't already do with Spring
> > ScheduledThreadPoolExecutor. It has no non-Local methods and it is hard
> to
> > understand why this functionality should belong to Ignite at all. WDYT?
> I'd
> > drop it in 3.0 and call it a day. Since then, avoid touching it.
> >
> > If we decide to NOT drop it, we should remove ignite-schedule module
> anyway
> > and move this code into ignite-spring module. As soon as we have Spring
> we
> > can have fully functioning IgniteScheduler so a dedicated module is NOT
> > needed. Are you ready to do that?
> >
> > I'm not really ready to merge your patch for two additional reasons:
> > - 1 thread limitation looks severe and will make scheduling more than 1-2
> > tasks a dangerous affair - one task may block your scheduler forever and
> > nothing else will be ran.
> > - What should happen if you scheduled a task every 5 minutes, task runs
> for
> > 6 minutes already and it's time to start it again: will it be started
> > concurrently, will it wait for the old one to finish and then start, will
> > it be discarded and ran only in 10 minutes after the original one is
> > started? What is the behavior of cron4j? What is the behavior of proposed
> > impl? Unfortunately we don't have tests on this and we don't want to,
> since
> > they'll take a lot of wall clock time, which we try to avoid to decrease
> TC
> > run time.
> >
> > Regards,
> > --
> > Ilya Kasnacheev
> >
> >
> > чт, 27 дек. 2018 г. в 18:53, Sergey <ma...@gmail.com>:
> >
> > > Hi, Ilya!
> > > It looks like Spring *ThreadPoolTaskScheduler* is bounded to JDK's
> > > *ScheduledThreadPoolExecutor*, so to launch scheduled tasks on public
> > pool
> > > we need provide *ScheduledThreadPoolExecutor* for Public pool.
> Currently
> > I
> > > see we create Public pool executor as *IgniteThreadPoolExecutor* and it
> > > extends *ThreadPoolExecutor*.
> > > It seems we can easily introduce *IgniteScheduledThreadPoolExecutor*
> and
> > > extend it from *ScheduledThreadPoolExecutor* for public pool. How do
> you
> > > like it?
> > >
> > > Your idea about delegating tasks from Spring 1-threaded pool to Public
> > pool
> > > looks also workable.
> > >
> > > Best regards,
> > > Sergey Kosarev.
> > >
> > >
> > > вт, 25 дек. 2018 г. в 18:19, Ilya Kasnacheev <
> ilya.kasnacheev@gmail.com
> > >:
> > >
> > > > Hello!
> > > >
> > > > I have started reviewing your pull request.
> > > > I will expect that scheduled tasks are executed on Public pool. Is it
> > > > possible that tasks are launched on Public pool? If Spring Scheduler
> > > > insists on its own thread pool, we can have single-thread pool which
> > will
> > > > execute put of tasks to public pool and immediately return. Is it
> > > possible
> > > > to do that?
> > > >
> > > > Regards,
> > > > --
> > > > Ilya Kasnacheev
> > > >
> > > >
> > > > вт, 25 дек. 2018 г. в 17:46, Alexey Kuznetsov <akuznetsov@apache.org
> >:
> > > >
> > > > > Hi, Sergey!
> > > > >
> > > > > I think we should keep compatibility as much as possible for Ignite
> > > 2.x.
> > > > > And we can do breaking changes in Ignite 3.x
> > > > >
> > > > > What do you think?
> > > > >
> > > > >
> > > > > On Mon, Dec 24, 2018 at 11:58 PM Sergey <ma...@gmail.com>
> > wrote:
> > > > >
> > > > > > HI, Igniters!
> > > > > >
> > > > > > I've updated and rebased implementation to master branch and made
> > > some
> > > > > > fixes.
> > > > > > Also I have a question regarding current implementation.
> > > > > >
> > > > > > As I found Cron4J source code this implementation checks schedule
> > > every
> > > > > > minute (seconds not supported) but spawns a thread for every task
> > > which
> > > > > > scheduling pattern matches the current time. There no any limits
> to
> > > the
> > > > > > number of tasks launched silmultaneously.
> > > > > >
> > > > > >  New implementation is based on
> > > > > > org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler
> > > > > > with its default parameters currently, i.e thread pool size is 1.
> > > > > >
> > > > > > Could you advise me, do we need to add some system property or
> > > > introduce
> > > > > > some attribute to IgniteConfiguration to configure Scheduler
> thread
> > > > pool
> > > > > > size? And what should be default value?
> > > > > >
> > > > > >
> > > > > > Best regards,
> > > > > > Sergey Kosarev.
> > > > > >
> > > > > >
> > > > > > чт, 10 мая 2018 г. в 20:23, Dmitry Pavlov <dpavlov.spb@gmail.com
> >:
> > > > > >
> > > > > > > Hi Anton,
> > > > > > >
> > > > > > > Thank you for joining and review.
> > > > > > > I hope all proposals will be applied.
> > > > > > >
> > > > > > > Sincerely,
> > > > > > > Dmitriy Pavlov
> > > > > > >
> > > > > > > пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:
> > > > > > >
> > > > > > > > Folks,
> > > > > > > >
> > > > > > > > How can it be at PATCH AVAILABLE since *none* of my latest
> > > comments
> > > > > > (made
> > > > > > > > Feb 8) are resolved at Upsource?
> > > > > > > > Changed state to IP.
> > > > > > > >
> > > > > > > > пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <
> > > dpavlov.spb@gmail.com
> > > > >:
> > > > > > > >
> > > > > > > > > Hi Andrey,
> > > > > > > > >
> > > > > > > > > Could you please pick up review?
> > > > > > > > >
> > > > > > > > > Sincerely,
> > > > > > > > > Dmitriy Pavlov
> > > > > > > > >
> > > > > > > > > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <
> > > > > > dsetrakyan@apache.org
> > > > > > > >:
> > > > > > > > >
> > > > > > > > > > Dmitriy, who is a good candidate within the community to
> > > review
> > > > > > this
> > > > > > > > > > ticket?
> > > > > > > > > >
> > > > > > > > > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <
> > > > > > > dpavlov.spb@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Igniters,
> > > > > > > > > > >
> > > > > > > > > > > it seems ticket
> > > > > > https://issues.apache.org/jira/browse/IGNITE-5565
> > > > > > > is
> > > > > > > > > > still
> > > > > > > > > > > in PA state. What are our next steps?
> > > > > > > > > > >
> > > > > > > > > > > Who did review of this patch?
> > > > > > > > > > >
> > > > > > > > > > > Sincerely,
> > > > > > > > > > > Dmitriy Pavlov
> > > > > > > > > > >
> > > > > > > > > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <
> > dmagda@apache.org
> > > >:
> > > > > > > > > > >
> > > > > > > > > > > > Yakov,
> > > > > > > > > > > >
> > > > > > > > > > > > No, the mentioned discussion didn’t turn into a JIRA
> > > > ticket.
> > > > > > > > > > > >
> > > > > > > > > > > > Alex K., please follow to some thoughts from there
> and
> > > wrap
> > > > > > them
> > > > > > > up
> > > > > > > > > in
> > > > > > > > > > a
> > > > > > > > > > > > form of the ticket.
> > > > > > > > > > > >
> > > > > > > > > > > > —
> > > > > > > > > > > > Denis
> > > > > > > > > > > >
> > > > > > > > > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <
> > > > > > > yzhdanov@apache.org>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Guys, I remember we discussed this some time ago.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > > > > > > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > > > > > > > > >
> > > > > > > > > > > > > Denis, do you have any ticket or SoW?
> > > > > > > > > > > > >
> > > > > > > > > > > > > --Yakov
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Alexey Kuznetsov
> > > > >
> > > >
> > >
> >
>

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Sergey <ma...@gmail.com>.
Ilya,
before deciding about drop just don't forget that our scheduleLocal method
extends functionality with optional prefix before standard Cron expression {n1,
n2} where n1 is delay of scheduling and n2 is number of executions.

Also we have own protection against task that has not finished before new
execution:
Look at
org.apache.ignite.internal.processors.schedule.ScheduleFutureImpl#run
In this case  a new task won't be executed (it just log warning message)
disregarding Cron4J or Spring is used.

Best regards,
Sergey Kosarev.


пт, 28 дек. 2018 г. в 13:51, Ilya Kasnacheev <il...@gmail.com>:

> Hello!
>
> I've spent considerable amount of time working on your patch yesterday, but
> eventually I have come to conclusion that we should Deprecate
> IgniteScheduler and ignite-schedule module, and Remove it in 3.0.
>
> The reason for this: IgnoteScheduler interface has 5 methods, of those
> callLocal() and runLocal() should probably be moved to IgniteCompute,
> runLocal() with timeout should be moved somewhere elsewhere, and
> scheduleLocal() which we are discussing should be removed since it does not
> offer anything that you can't already do with Spring
> ScheduledThreadPoolExecutor. It has no non-Local methods and it is hard to
> understand why this functionality should belong to Ignite at all. WDYT? I'd
> drop it in 3.0 and call it a day. Since then, avoid touching it.
>
> If we decide to NOT drop it, we should remove ignite-schedule module anyway
> and move this code into ignite-spring module. As soon as we have Spring we
> can have fully functioning IgniteScheduler so a dedicated module is NOT
> needed. Are you ready to do that?
>
> I'm not really ready to merge your patch for two additional reasons:
> - 1 thread limitation looks severe and will make scheduling more than 1-2
> tasks a dangerous affair - one task may block your scheduler forever and
> nothing else will be ran.
> - What should happen if you scheduled a task every 5 minutes, task runs for
> 6 minutes already and it's time to start it again: will it be started
> concurrently, will it wait for the old one to finish and then start, will
> it be discarded and ran only in 10 minutes after the original one is
> started? What is the behavior of cron4j? What is the behavior of proposed
> impl? Unfortunately we don't have tests on this and we don't want to, since
> they'll take a lot of wall clock time, which we try to avoid to decrease TC
> run time.
>
> Regards,
> --
> Ilya Kasnacheev
>
>
> чт, 27 дек. 2018 г. в 18:53, Sergey <ma...@gmail.com>:
>
> > Hi, Ilya!
> > It looks like Spring *ThreadPoolTaskScheduler* is bounded to JDK's
> > *ScheduledThreadPoolExecutor*, so to launch scheduled tasks on public
> pool
> > we need provide *ScheduledThreadPoolExecutor* for Public pool. Currently
> I
> > see we create Public pool executor as *IgniteThreadPoolExecutor* and it
> > extends *ThreadPoolExecutor*.
> > It seems we can easily introduce *IgniteScheduledThreadPoolExecutor* and
> > extend it from *ScheduledThreadPoolExecutor* for public pool. How do you
> > like it?
> >
> > Your idea about delegating tasks from Spring 1-threaded pool to Public
> pool
> > looks also workable.
> >
> > Best regards,
> > Sergey Kosarev.
> >
> >
> > вт, 25 дек. 2018 г. в 18:19, Ilya Kasnacheev <ilya.kasnacheev@gmail.com
> >:
> >
> > > Hello!
> > >
> > > I have started reviewing your pull request.
> > > I will expect that scheduled tasks are executed on Public pool. Is it
> > > possible that tasks are launched on Public pool? If Spring Scheduler
> > > insists on its own thread pool, we can have single-thread pool which
> will
> > > execute put of tasks to public pool and immediately return. Is it
> > possible
> > > to do that?
> > >
> > > Regards,
> > > --
> > > Ilya Kasnacheev
> > >
> > >
> > > вт, 25 дек. 2018 г. в 17:46, Alexey Kuznetsov <ak...@apache.org>:
> > >
> > > > Hi, Sergey!
> > > >
> > > > I think we should keep compatibility as much as possible for Ignite
> > 2.x.
> > > > And we can do breaking changes in Ignite 3.x
> > > >
> > > > What do you think?
> > > >
> > > >
> > > > On Mon, Dec 24, 2018 at 11:58 PM Sergey <ma...@gmail.com>
> wrote:
> > > >
> > > > > HI, Igniters!
> > > > >
> > > > > I've updated and rebased implementation to master branch and made
> > some
> > > > > fixes.
> > > > > Also I have a question regarding current implementation.
> > > > >
> > > > > As I found Cron4J source code this implementation checks schedule
> > every
> > > > > minute (seconds not supported) but spawns a thread for every task
> > which
> > > > > scheduling pattern matches the current time. There no any limits to
> > the
> > > > > number of tasks launched silmultaneously.
> > > > >
> > > > >  New implementation is based on
> > > > > org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler
> > > > > with its default parameters currently, i.e thread pool size is 1.
> > > > >
> > > > > Could you advise me, do we need to add some system property or
> > > introduce
> > > > > some attribute to IgniteConfiguration to configure Scheduler thread
> > > pool
> > > > > size? And what should be default value?
> > > > >
> > > > >
> > > > > Best regards,
> > > > > Sergey Kosarev.
> > > > >
> > > > >
> > > > > чт, 10 мая 2018 г. в 20:23, Dmitry Pavlov <dp...@gmail.com>:
> > > > >
> > > > > > Hi Anton,
> > > > > >
> > > > > > Thank you for joining and review.
> > > > > > I hope all proposals will be applied.
> > > > > >
> > > > > > Sincerely,
> > > > > > Dmitriy Pavlov
> > > > > >
> > > > > > пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:
> > > > > >
> > > > > > > Folks,
> > > > > > >
> > > > > > > How can it be at PATCH AVAILABLE since *none* of my latest
> > comments
> > > > > (made
> > > > > > > Feb 8) are resolved at Upsource?
> > > > > > > Changed state to IP.
> > > > > > >
> > > > > > > пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <
> > dpavlov.spb@gmail.com
> > > >:
> > > > > > >
> > > > > > > > Hi Andrey,
> > > > > > > >
> > > > > > > > Could you please pick up review?
> > > > > > > >
> > > > > > > > Sincerely,
> > > > > > > > Dmitriy Pavlov
> > > > > > > >
> > > > > > > > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <
> > > > > dsetrakyan@apache.org
> > > > > > >:
> > > > > > > >
> > > > > > > > > Dmitriy, who is a good candidate within the community to
> > review
> > > > > this
> > > > > > > > > ticket?
> > > > > > > > >
> > > > > > > > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <
> > > > > > dpavlov.spb@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Igniters,
> > > > > > > > > >
> > > > > > > > > > it seems ticket
> > > > > https://issues.apache.org/jira/browse/IGNITE-5565
> > > > > > is
> > > > > > > > > still
> > > > > > > > > > in PA state. What are our next steps?
> > > > > > > > > >
> > > > > > > > > > Who did review of this patch?
> > > > > > > > > >
> > > > > > > > > > Sincerely,
> > > > > > > > > > Dmitriy Pavlov
> > > > > > > > > >
> > > > > > > > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <
> dmagda@apache.org
> > >:
> > > > > > > > > >
> > > > > > > > > > > Yakov,
> > > > > > > > > > >
> > > > > > > > > > > No, the mentioned discussion didn’t turn into a JIRA
> > > ticket.
> > > > > > > > > > >
> > > > > > > > > > > Alex K., please follow to some thoughts from there and
> > wrap
> > > > > them
> > > > > > up
> > > > > > > > in
> > > > > > > > > a
> > > > > > > > > > > form of the ticket.
> > > > > > > > > > >
> > > > > > > > > > > —
> > > > > > > > > > > Denis
> > > > > > > > > > >
> > > > > > > > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <
> > > > > > yzhdanov@apache.org>
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > Guys, I remember we discussed this some time ago.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > > > > > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > > > > > > > >
> > > > > > > > > > > > Denis, do you have any ticket or SoW?
> > > > > > > > > > > >
> > > > > > > > > > > > --Yakov
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > Alexey Kuznetsov
> > > >
> > >
> >
>

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Ilya Kasnacheev <il...@gmail.com>.
Hello!

I've spent considerable amount of time working on your patch yesterday, but
eventually I have come to conclusion that we should Deprecate
IgniteScheduler and ignite-schedule module, and Remove it in 3.0.

The reason for this: IgnoteScheduler interface has 5 methods, of those
callLocal() and runLocal() should probably be moved to IgniteCompute,
runLocal() with timeout should be moved somewhere elsewhere, and
scheduleLocal() which we are discussing should be removed since it does not
offer anything that you can't already do with Spring
ScheduledThreadPoolExecutor. It has no non-Local methods and it is hard to
understand why this functionality should belong to Ignite at all. WDYT? I'd
drop it in 3.0 and call it a day. Since then, avoid touching it.

If we decide to NOT drop it, we should remove ignite-schedule module anyway
and move this code into ignite-spring module. As soon as we have Spring we
can have fully functioning IgniteScheduler so a dedicated module is NOT
needed. Are you ready to do that?

I'm not really ready to merge your patch for two additional reasons:
- 1 thread limitation looks severe and will make scheduling more than 1-2
tasks a dangerous affair - one task may block your scheduler forever and
nothing else will be ran.
- What should happen if you scheduled a task every 5 minutes, task runs for
6 minutes already and it's time to start it again: will it be started
concurrently, will it wait for the old one to finish and then start, will
it be discarded and ran only in 10 minutes after the original one is
started? What is the behavior of cron4j? What is the behavior of proposed
impl? Unfortunately we don't have tests on this and we don't want to, since
they'll take a lot of wall clock time, which we try to avoid to decrease TC
run time.

Regards,
-- 
Ilya Kasnacheev


чт, 27 дек. 2018 г. в 18:53, Sergey <ma...@gmail.com>:

> Hi, Ilya!
> It looks like Spring *ThreadPoolTaskScheduler* is bounded to JDK's
> *ScheduledThreadPoolExecutor*, so to launch scheduled tasks on public pool
> we need provide *ScheduledThreadPoolExecutor* for Public pool. Currently I
> see we create Public pool executor as *IgniteThreadPoolExecutor* and it
> extends *ThreadPoolExecutor*.
> It seems we can easily introduce *IgniteScheduledThreadPoolExecutor* and
> extend it from *ScheduledThreadPoolExecutor* for public pool. How do you
> like it?
>
> Your idea about delegating tasks from Spring 1-threaded pool to Public pool
> looks also workable.
>
> Best regards,
> Sergey Kosarev.
>
>
> вт, 25 дек. 2018 г. в 18:19, Ilya Kasnacheev <il...@gmail.com>:
>
> > Hello!
> >
> > I have started reviewing your pull request.
> > I will expect that scheduled tasks are executed on Public pool. Is it
> > possible that tasks are launched on Public pool? If Spring Scheduler
> > insists on its own thread pool, we can have single-thread pool which will
> > execute put of tasks to public pool and immediately return. Is it
> possible
> > to do that?
> >
> > Regards,
> > --
> > Ilya Kasnacheev
> >
> >
> > вт, 25 дек. 2018 г. в 17:46, Alexey Kuznetsov <ak...@apache.org>:
> >
> > > Hi, Sergey!
> > >
> > > I think we should keep compatibility as much as possible for Ignite
> 2.x.
> > > And we can do breaking changes in Ignite 3.x
> > >
> > > What do you think?
> > >
> > >
> > > On Mon, Dec 24, 2018 at 11:58 PM Sergey <ma...@gmail.com> wrote:
> > >
> > > > HI, Igniters!
> > > >
> > > > I've updated and rebased implementation to master branch and made
> some
> > > > fixes.
> > > > Also I have a question regarding current implementation.
> > > >
> > > > As I found Cron4J source code this implementation checks schedule
> every
> > > > minute (seconds not supported) but spawns a thread for every task
> which
> > > > scheduling pattern matches the current time. There no any limits to
> the
> > > > number of tasks launched silmultaneously.
> > > >
> > > >  New implementation is based on
> > > > org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler
> > > > with its default parameters currently, i.e thread pool size is 1.
> > > >
> > > > Could you advise me, do we need to add some system property or
> > introduce
> > > > some attribute to IgniteConfiguration to configure Scheduler thread
> > pool
> > > > size? And what should be default value?
> > > >
> > > >
> > > > Best regards,
> > > > Sergey Kosarev.
> > > >
> > > >
> > > > чт, 10 мая 2018 г. в 20:23, Dmitry Pavlov <dp...@gmail.com>:
> > > >
> > > > > Hi Anton,
> > > > >
> > > > > Thank you for joining and review.
> > > > > I hope all proposals will be applied.
> > > > >
> > > > > Sincerely,
> > > > > Dmitriy Pavlov
> > > > >
> > > > > пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:
> > > > >
> > > > > > Folks,
> > > > > >
> > > > > > How can it be at PATCH AVAILABLE since *none* of my latest
> comments
> > > > (made
> > > > > > Feb 8) are resolved at Upsource?
> > > > > > Changed state to IP.
> > > > > >
> > > > > > пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <
> dpavlov.spb@gmail.com
> > >:
> > > > > >
> > > > > > > Hi Andrey,
> > > > > > >
> > > > > > > Could you please pick up review?
> > > > > > >
> > > > > > > Sincerely,
> > > > > > > Dmitriy Pavlov
> > > > > > >
> > > > > > > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <
> > > > dsetrakyan@apache.org
> > > > > >:
> > > > > > >
> > > > > > > > Dmitriy, who is a good candidate within the community to
> review
> > > > this
> > > > > > > > ticket?
> > > > > > > >
> > > > > > > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <
> > > > > dpavlov.spb@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Igniters,
> > > > > > > > >
> > > > > > > > > it seems ticket
> > > > https://issues.apache.org/jira/browse/IGNITE-5565
> > > > > is
> > > > > > > > still
> > > > > > > > > in PA state. What are our next steps?
> > > > > > > > >
> > > > > > > > > Who did review of this patch?
> > > > > > > > >
> > > > > > > > > Sincerely,
> > > > > > > > > Dmitriy Pavlov
> > > > > > > > >
> > > > > > > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <dmagda@apache.org
> >:
> > > > > > > > >
> > > > > > > > > > Yakov,
> > > > > > > > > >
> > > > > > > > > > No, the mentioned discussion didn’t turn into a JIRA
> > ticket.
> > > > > > > > > >
> > > > > > > > > > Alex K., please follow to some thoughts from there and
> wrap
> > > > them
> > > > > up
> > > > > > > in
> > > > > > > > a
> > > > > > > > > > form of the ticket.
> > > > > > > > > >
> > > > > > > > > > —
> > > > > > > > > > Denis
> > > > > > > > > >
> > > > > > > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <
> > > > > yzhdanov@apache.org>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > Guys, I remember we discussed this some time ago.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > > > > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > > > > > > >
> > > > > > > > > > > Denis, do you have any ticket or SoW?
> > > > > > > > > > >
> > > > > > > > > > > --Yakov
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> > > --
> > > Alexey Kuznetsov
> > >
> >
>

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Sergey <ma...@gmail.com>.
Hi, Ilya!
It looks like Spring *ThreadPoolTaskScheduler* is bounded to JDK's
*ScheduledThreadPoolExecutor*, so to launch scheduled tasks on public pool
we need provide *ScheduledThreadPoolExecutor* for Public pool. Currently I
see we create Public pool executor as *IgniteThreadPoolExecutor* and it
extends *ThreadPoolExecutor*.
It seems we can easily introduce *IgniteScheduledThreadPoolExecutor* and
extend it from *ScheduledThreadPoolExecutor* for public pool. How do you
like it?

Your idea about delegating tasks from Spring 1-threaded pool to Public pool
looks also workable.

Best regards,
Sergey Kosarev.


вт, 25 дек. 2018 г. в 18:19, Ilya Kasnacheev <il...@gmail.com>:

> Hello!
>
> I have started reviewing your pull request.
> I will expect that scheduled tasks are executed on Public pool. Is it
> possible that tasks are launched on Public pool? If Spring Scheduler
> insists on its own thread pool, we can have single-thread pool which will
> execute put of tasks to public pool and immediately return. Is it possible
> to do that?
>
> Regards,
> --
> Ilya Kasnacheev
>
>
> вт, 25 дек. 2018 г. в 17:46, Alexey Kuznetsov <ak...@apache.org>:
>
> > Hi, Sergey!
> >
> > I think we should keep compatibility as much as possible for Ignite 2.x.
> > And we can do breaking changes in Ignite 3.x
> >
> > What do you think?
> >
> >
> > On Mon, Dec 24, 2018 at 11:58 PM Sergey <ma...@gmail.com> wrote:
> >
> > > HI, Igniters!
> > >
> > > I've updated and rebased implementation to master branch and made some
> > > fixes.
> > > Also I have a question regarding current implementation.
> > >
> > > As I found Cron4J source code this implementation checks schedule every
> > > minute (seconds not supported) but spawns a thread for every task which
> > > scheduling pattern matches the current time. There no any limits to the
> > > number of tasks launched silmultaneously.
> > >
> > >  New implementation is based on
> > > org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler
> > > with its default parameters currently, i.e thread pool size is 1.
> > >
> > > Could you advise me, do we need to add some system property or
> introduce
> > > some attribute to IgniteConfiguration to configure Scheduler thread
> pool
> > > size? And what should be default value?
> > >
> > >
> > > Best regards,
> > > Sergey Kosarev.
> > >
> > >
> > > чт, 10 мая 2018 г. в 20:23, Dmitry Pavlov <dp...@gmail.com>:
> > >
> > > > Hi Anton,
> > > >
> > > > Thank you for joining and review.
> > > > I hope all proposals will be applied.
> > > >
> > > > Sincerely,
> > > > Dmitriy Pavlov
> > > >
> > > > пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:
> > > >
> > > > > Folks,
> > > > >
> > > > > How can it be at PATCH AVAILABLE since *none* of my latest comments
> > > (made
> > > > > Feb 8) are resolved at Upsource?
> > > > > Changed state to IP.
> > > > >
> > > > > пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <dpavlov.spb@gmail.com
> >:
> > > > >
> > > > > > Hi Andrey,
> > > > > >
> > > > > > Could you please pick up review?
> > > > > >
> > > > > > Sincerely,
> > > > > > Dmitriy Pavlov
> > > > > >
> > > > > > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <
> > > dsetrakyan@apache.org
> > > > >:
> > > > > >
> > > > > > > Dmitriy, who is a good candidate within the community to review
> > > this
> > > > > > > ticket?
> > > > > > >
> > > > > > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <
> > > > dpavlov.spb@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Igniters,
> > > > > > > >
> > > > > > > > it seems ticket
> > > https://issues.apache.org/jira/browse/IGNITE-5565
> > > > is
> > > > > > > still
> > > > > > > > in PA state. What are our next steps?
> > > > > > > >
> > > > > > > > Who did review of this patch?
> > > > > > > >
> > > > > > > > Sincerely,
> > > > > > > > Dmitriy Pavlov
> > > > > > > >
> > > > > > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <dm...@apache.org>:
> > > > > > > >
> > > > > > > > > Yakov,
> > > > > > > > >
> > > > > > > > > No, the mentioned discussion didn’t turn into a JIRA
> ticket.
> > > > > > > > >
> > > > > > > > > Alex K., please follow to some thoughts from there and wrap
> > > them
> > > > up
> > > > > > in
> > > > > > > a
> > > > > > > > > form of the ticket.
> > > > > > > > >
> > > > > > > > > —
> > > > > > > > > Denis
> > > > > > > > >
> > > > > > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <
> > > > yzhdanov@apache.org>
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > Guys, I remember we discussed this some time ago.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > > > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > > > > > >
> > > > > > > > > > Denis, do you have any ticket or SoW?
> > > > > > > > > >
> > > > > > > > > > --Yakov
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> > --
> > Alexey Kuznetsov
> >
>

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Ilya Kasnacheev <il...@gmail.com>.
Hello!

I have started reviewing your pull request.
I will expect that scheduled tasks are executed on Public pool. Is it
possible that tasks are launched on Public pool? If Spring Scheduler
insists on its own thread pool, we can have single-thread pool which will
execute put of tasks to public pool and immediately return. Is it possible
to do that?

Regards,
-- 
Ilya Kasnacheev


вт, 25 дек. 2018 г. в 17:46, Alexey Kuznetsov <ak...@apache.org>:

> Hi, Sergey!
>
> I think we should keep compatibility as much as possible for Ignite 2.x.
> And we can do breaking changes in Ignite 3.x
>
> What do you think?
>
>
> On Mon, Dec 24, 2018 at 11:58 PM Sergey <ma...@gmail.com> wrote:
>
> > HI, Igniters!
> >
> > I've updated and rebased implementation to master branch and made some
> > fixes.
> > Also I have a question regarding current implementation.
> >
> > As I found Cron4J source code this implementation checks schedule every
> > minute (seconds not supported) but spawns a thread for every task which
> > scheduling pattern matches the current time. There no any limits to the
> > number of tasks launched silmultaneously.
> >
> >  New implementation is based on
> > org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler
> > with its default parameters currently, i.e thread pool size is 1.
> >
> > Could you advise me, do we need to add some system property or introduce
> > some attribute to IgniteConfiguration to configure Scheduler thread pool
> > size? And what should be default value?
> >
> >
> > Best regards,
> > Sergey Kosarev.
> >
> >
> > чт, 10 мая 2018 г. в 20:23, Dmitry Pavlov <dp...@gmail.com>:
> >
> > > Hi Anton,
> > >
> > > Thank you for joining and review.
> > > I hope all proposals will be applied.
> > >
> > > Sincerely,
> > > Dmitriy Pavlov
> > >
> > > пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:
> > >
> > > > Folks,
> > > >
> > > > How can it be at PATCH AVAILABLE since *none* of my latest comments
> > (made
> > > > Feb 8) are resolved at Upsource?
> > > > Changed state to IP.
> > > >
> > > > пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <dp...@gmail.com>:
> > > >
> > > > > Hi Andrey,
> > > > >
> > > > > Could you please pick up review?
> > > > >
> > > > > Sincerely,
> > > > > Dmitriy Pavlov
> > > > >
> > > > > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <
> > dsetrakyan@apache.org
> > > >:
> > > > >
> > > > > > Dmitriy, who is a good candidate within the community to review
> > this
> > > > > > ticket?
> > > > > >
> > > > > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <
> > > dpavlov.spb@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Igniters,
> > > > > > >
> > > > > > > it seems ticket
> > https://issues.apache.org/jira/browse/IGNITE-5565
> > > is
> > > > > > still
> > > > > > > in PA state. What are our next steps?
> > > > > > >
> > > > > > > Who did review of this patch?
> > > > > > >
> > > > > > > Sincerely,
> > > > > > > Dmitriy Pavlov
> > > > > > >
> > > > > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <dm...@apache.org>:
> > > > > > >
> > > > > > > > Yakov,
> > > > > > > >
> > > > > > > > No, the mentioned discussion didn’t turn into a JIRA ticket.
> > > > > > > >
> > > > > > > > Alex K., please follow to some thoughts from there and wrap
> > them
> > > up
> > > > > in
> > > > > > a
> > > > > > > > form of the ticket.
> > > > > > > >
> > > > > > > > —
> > > > > > > > Denis
> > > > > > > >
> > > > > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <
> > > yzhdanov@apache.org>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > Guys, I remember we discussed this some time ago.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > > > > >
> > > > > > > > > Denis, do you have any ticket or SoW?
> > > > > > > > >
> > > > > > > > > --Yakov
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
> --
> Alexey Kuznetsov
>

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Alexey Kuznetsov <ak...@apache.org>.
Hi, Sergey!

I think we should keep compatibility as much as possible for Ignite 2.x.
And we can do breaking changes in Ignite 3.x

What do you think?


On Mon, Dec 24, 2018 at 11:58 PM Sergey <ma...@gmail.com> wrote:

> HI, Igniters!
>
> I've updated and rebased implementation to master branch and made some
> fixes.
> Also I have a question regarding current implementation.
>
> As I found Cron4J source code this implementation checks schedule every
> minute (seconds not supported) but spawns a thread for every task which
> scheduling pattern matches the current time. There no any limits to the
> number of tasks launched silmultaneously.
>
>  New implementation is based on
> org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler
> with its default parameters currently, i.e thread pool size is 1.
>
> Could you advise me, do we need to add some system property or introduce
> some attribute to IgniteConfiguration to configure Scheduler thread pool
> size? And what should be default value?
>
>
> Best regards,
> Sergey Kosarev.
>
>
> чт, 10 мая 2018 г. в 20:23, Dmitry Pavlov <dp...@gmail.com>:
>
> > Hi Anton,
> >
> > Thank you for joining and review.
> > I hope all proposals will be applied.
> >
> > Sincerely,
> > Dmitriy Pavlov
> >
> > пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:
> >
> > > Folks,
> > >
> > > How can it be at PATCH AVAILABLE since *none* of my latest comments
> (made
> > > Feb 8) are resolved at Upsource?
> > > Changed state to IP.
> > >
> > > пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <dp...@gmail.com>:
> > >
> > > > Hi Andrey,
> > > >
> > > > Could you please pick up review?
> > > >
> > > > Sincerely,
> > > > Dmitriy Pavlov
> > > >
> > > > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <
> dsetrakyan@apache.org
> > >:
> > > >
> > > > > Dmitriy, who is a good candidate within the community to review
> this
> > > > > ticket?
> > > > >
> > > > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <
> > dpavlov.spb@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Igniters,
> > > > > >
> > > > > > it seems ticket
> https://issues.apache.org/jira/browse/IGNITE-5565
> > is
> > > > > still
> > > > > > in PA state. What are our next steps?
> > > > > >
> > > > > > Who did review of this patch?
> > > > > >
> > > > > > Sincerely,
> > > > > > Dmitriy Pavlov
> > > > > >
> > > > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <dm...@apache.org>:
> > > > > >
> > > > > > > Yakov,
> > > > > > >
> > > > > > > No, the mentioned discussion didn’t turn into a JIRA ticket.
> > > > > > >
> > > > > > > Alex K., please follow to some thoughts from there and wrap
> them
> > up
> > > > in
> > > > > a
> > > > > > > form of the ticket.
> > > > > > >
> > > > > > > —
> > > > > > > Denis
> > > > > > >
> > > > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <
> > yzhdanov@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > Guys, I remember we discussed this some time ago.
> > > > > > > >
> > > > > > > >
> > > > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > > > >
> > > > > > > > Denis, do you have any ticket or SoW?
> > > > > > > >
> > > > > > > > --Yakov
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


-- 
Alexey Kuznetsov

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Sergey <ma...@gmail.com>.
HI, Igniters!

I've updated and rebased implementation to master branch and made some
fixes.
Also I have a question regarding current implementation.

As I found Cron4J source code this implementation checks schedule every
minute (seconds not supported) but spawns a thread for every task which
scheduling pattern matches the current time. There no any limits to the
number of tasks launched silmultaneously.

 New implementation is based on
org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler
with its default parameters currently, i.e thread pool size is 1.

Could you advise me, do we need to add some system property or introduce
some attribute to IgniteConfiguration to configure Scheduler thread pool
size? And what should be default value?


Best regards,
Sergey Kosarev.


чт, 10 мая 2018 г. в 20:23, Dmitry Pavlov <dp...@gmail.com>:

> Hi Anton,
>
> Thank you for joining and review.
> I hope all proposals will be applied.
>
> Sincerely,
> Dmitriy Pavlov
>
> пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:
>
> > Folks,
> >
> > How can it be at PATCH AVAILABLE since *none* of my latest comments (made
> > Feb 8) are resolved at Upsource?
> > Changed state to IP.
> >
> > пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <dp...@gmail.com>:
> >
> > > Hi Andrey,
> > >
> > > Could you please pick up review?
> > >
> > > Sincerely,
> > > Dmitriy Pavlov
> > >
> > > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <dsetrakyan@apache.org
> >:
> > >
> > > > Dmitriy, who is a good candidate within the community to review this
> > > > ticket?
> > > >
> > > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <
> dpavlov.spb@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Igniters,
> > > > >
> > > > > it seems ticket https://issues.apache.org/jira/browse/IGNITE-5565
> is
> > > > still
> > > > > in PA state. What are our next steps?
> > > > >
> > > > > Who did review of this patch?
> > > > >
> > > > > Sincerely,
> > > > > Dmitriy Pavlov
> > > > >
> > > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <dm...@apache.org>:
> > > > >
> > > > > > Yakov,
> > > > > >
> > > > > > No, the mentioned discussion didn’t turn into a JIRA ticket.
> > > > > >
> > > > > > Alex K., please follow to some thoughts from there and wrap them
> up
> > > in
> > > > a
> > > > > > form of the ticket.
> > > > > >
> > > > > > —
> > > > > > Denis
> > > > > >
> > > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <
> yzhdanov@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > > Guys, I remember we discussed this some time ago.
> > > > > > >
> > > > > > >
> > > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > > >
> > > > > > > Denis, do you have any ticket or SoW?
> > > > > > >
> > > > > > > --Yakov
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Replace Cron4J with Quartz for ignite-schedule module.

Posted by Dmitry Pavlov <dp...@gmail.com>.
Hi Anton,

Thank you for joining and review.
I hope all proposals will be applied.

Sincerely,
Dmitriy Pavlov

пт, 4 мая 2018 г. в 16:04, Anton Vinogradov <av...@apache.org>:

> Folks,
>
> How can it be at PATCH AVAILABLE since *none* of my latest comments (made
> Feb 8) are resolved at Upsource?
> Changed state to IP.
>
> пн, 23 апр. 2018 г. в 20:00, Dmitry Pavlov <dp...@gmail.com>:
>
> > Hi Andrey,
> >
> > Could you please pick up review?
> >
> > Sincerely,
> > Dmitriy Pavlov
> >
> > пн, 23 апр. 2018 г. в 17:39, Dmitriy Setrakyan <ds...@apache.org>:
> >
> > > Dmitriy, who is a good candidate within the community to review this
> > > ticket?
> > >
> > > On Mon, Apr 23, 2018 at 6:10 AM, Dmitry Pavlov <dp...@gmail.com>
> > > wrote:
> > >
> > > > Hi Igniters,
> > > >
> > > > it seems ticket https://issues.apache.org/jira/browse/IGNITE-5565 is
> > > still
> > > > in PA state. What are our next steps?
> > > >
> > > > Who did review of this patch?
> > > >
> > > > Sincerely,
> > > > Dmitriy Pavlov
> > > >
> > > > ср, 28 июн. 2017 г. в 1:40, Denis Magda <dm...@apache.org>:
> > > >
> > > > > Yakov,
> > > > >
> > > > > No, the mentioned discussion didn’t turn into a JIRA ticket.
> > > > >
> > > > > Alex K., please follow to some thoughts from there and wrap them up
> > in
> > > a
> > > > > form of the ticket.
> > > > >
> > > > > —
> > > > > Denis
> > > > >
> > > > > > On Jun 26, 2017, at 2:58 AM, Yakov Zhdanov <yz...@apache.org>
> > > > wrote:
> > > > > >
> > > > > > Guys, I remember we discussed this some time ago.
> > > > > >
> > > > > >
> > > > > http://apache-ignite-developers.2346864.n4.nabble.
> > > > com/Tasks-Scheduling-and-Chaining-td14293.html
> > > > > >
> > > > > > Denis, do you have any ticket or SoW?
> > > > > >
> > > > > > --Yakov
> > > > >
> > > > >
> > > >
> > >
> >
>