Skip to content

Added CronScheduler support as a proof to clock drift while emitting metrics#10448

Merged
leventov merged 14 commits intoapache:masterfrom
miqdigital:cronScheduling
Nov 25, 2020
Merged

Added CronScheduler support as a proof to clock drift while emitting metrics#10448
leventov merged 14 commits intoapache:masterfrom
miqdigital:cronScheduling

Conversation

@ayushkul2910
Copy link
Copy Markdown
Contributor

@ayushkul2910 ayushkul2910 commented Sep 29, 2020

Fixes #9283.

Description

This PR fixes the clock drift issue while emitting metrics. ScheduledExecutorService is very much prone to clock drift especially in certain windows platforms (see this: https://stackoverflow.com/questions/56571647/why-does-the-java-scheduler-exhibit-significant-time-drift-on-windows). I have used CronScheduler instead of ScheduledExecutorService, since CronScheduler provides proof against the clock drift issue.

I have changed the startMonitor method in MonitorScheduler class. In this implementation, I have used CronScheduler.scheduleAtFixedRate method for periodically scheduling the monitor with constant rate. This method requires a CronTask which should be non-blocking, so the monitoring inside the cronTask is happening as an async process using executor thread pool.


This PR has:

  • been self-reviewed.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in [licenses.yaml]
    (https://github.com/apache/druid/blob/master/licenses.yaml)
  • added unit tests to cover new code paths, ensuring the threshold for code coverage is met.

{
log.debug("Scheduling periodically: %s with period %s", callable, rate);
Instant delayInstance = Instant.now().plusMillis(initialDelay.getMillis());
exec.scheduleAt(delayInstance,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this logic: scheduling one-shot task which reschedules itself, is problematic. I think we should remove these methods (together with the Signal enum) because they have negative utility.

In particular, for CronScheduler, constant rescheduling is probably prone to some drift. Instead, CronScheduler's methods like scheduleAtFixedRate() should be used directly from MonitorScheduler. The periodic task can be cancelled using the returned Future.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That makes sense. I'll change this.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey @leventov, do you think the below implementation for startMonitor(final Monitor monitor) method in MonitorScheduler class will suffice? Also, can this cause any inconsistency since scheduledFuture is volatile and is shared amongst all the monitors?

synchronized (lock) {
  monitor.start();
  Long rate = config.getEmitterPeriod().getMillis();
  scheduledFuture = scheduler.scheduleAtFixedRate(
      rate,
      rate,
      TimeUnit.MILLISECONDS,
      new CronTask()
      {
        @Override
        public void run(long scheduledRunTimeMillis)
        {
          try {
            if (monitor.monitor(emitter) && hasMonitor(monitor)) {
              log.trace("Running %s (period %s)", this, rate);
            } else {
              log.debug("Stopping rescheduling %s (delay %s)", this, rate);
              removeMonitor(monitor);
              while (scheduledFuture == null) {
                Thread.sleep(1);
              }
              scheduledFuture.cancel(false);
            }
          } catch (Throwable e) {
            log.error(e, "Uncaught exception.");
          }
        }
      });
}

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There should be a separate future for every monitor. I also think there should be a separate executorService for running monitor.monitor(emitter), and cancelling the future from the first monitor, for two reasons:

  1. Monitor code is not guaranteed to be non-blocking and "cheap", which is the requirement of CronScheduler;
  2. You can avoid having a race condition of cancelling a future on the first iteration when it's not yet created.

Copy link
Copy Markdown
Contributor Author

@ayushkul2910 ayushkul2910 Oct 27, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Keeping the above points in mind, I think this implementation will do.

synchronized (lock) {
  monitor.start();
  Long rate = config.getEmitterPeriod().getMillis();
  Future<?> scheduledFuture = scheduler.scheduleAtFixedRate(
      rate,
      rate,
      TimeUnit.MILLISECONDS,
      new CronTask()
      {
        private volatile Future<Boolean> monitorFuture = null;
        @Override
        public void run(long scheduledRunTimeMillis)
        {
          try {
            if (monitorFuture != null && monitorFuture.isDone()
                && !(monitorFuture.get() && hasMonitor(monitor))) {
              removeMonitor(monitor);
              monitor.getScheduledFuture().cancel(false);
              log.debug("Stopped rescheduling %s (delay %s)", this, rate);
              return;
            }
            
            log.trace("Running %s (period %s)", this, rate);
            monitorFuture = executor.submit(new Callable<Boolean>()
            {
              public Boolean call()
              {
                try {
                  return monitor.monitor(emitter);
                } catch (Throwable e) {
                  log.error(e, "Uncaught exception.");
                  return false;
                }
              }
            });
          } 
          catch (Throwable e) {
            log.error(e, "Uncaught exception.");
          }
        }
      });
  monitor.setScheduledFuture(scheduledFuture);
}

In this:

  1. Each monitor has a separate future.
  2. Cron task is cheap, it checks a boolean condition. If condition is true it cancels the scheduling process for the particular monitor, else submits a callable for monitoring to executor service
  3. No race condition for cancelling scheduledFuture on first iteration.

Please let me know your thoughts on this.

@lgtm-com
Copy link
Copy Markdown

lgtm-com Bot commented Oct 30, 2020

This pull request introduces 1 alert when merging 0de642c into 9c51047 - view on LGTM.com

new alerts:

  • 1 for Boxed variable is never null

@ayushkul2910
Copy link
Copy Markdown
Contributor Author

Hi @leventov, I have updated the PR according to your suggestions. Can you please review once again?

Copy link
Copy Markdown
Member

@leventov leventov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM (apart from the clumsiness of the code around monitor cancellation, but I don't know how to improve it).

But you should also probably secure an approval from someone who is an active maintainer of the Druid project because there is no stake for me to approve this PR so maybe I miss something.

@ayushkul2910
Copy link
Copy Markdown
Contributor Author

Thanks @leventov for reviewing the PR. Can you please let me know, how can I reach out to other active maintainers?

@leventov
Copy link
Copy Markdown
Member

dev@druid.apache.org mailing list

Copy link
Copy Markdown
Member

@QiuMM QiuMM left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm.

@ayushkul2910
Copy link
Copy Markdown
Contributor Author

Thanks for the review. Can we now merge this PR @QiuMM @leventov?

@leventov leventov merged commit d0c2ede into apache:master Nov 25, 2020
@ayushkul2910 ayushkul2910 deleted the cronScheduling branch December 3, 2020 10:00
@jihoonson jihoonson added this to the 0.21.0 milestone Jan 4, 2021
emitter,
monitors,
Executors.newCachedThreadPool()
Execs.multiThreaded(64, "MonitorThread-%d")
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks like an overkill. What was the rationale for using this many threads?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @jihoonson , sorry for delay in response.
I think currently there are ~20 monitors, which can run concurrently with the MonitorScheduler class. Suppose a case in which frequency of scheduling < time taken by the executor thread to do monitor.monitor(...)(Although I am not sure if this case is possible in practical, kind of edge case). This can result in queuing of the tasks if threads are very less. I think we should atleast have no. of threads equal to max number of monitors supported. I may be missing something here. What do you think?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The monitor usually takes less than 1 sec (probably less than 100 ms) while the emission period is large enough to run all monitors (1 min by default). So, I think the scenario you described can happen when there are some failures such as retrying metrics emission due to some network issue. However, I don't think we should handle these failures by employing multiple threads because there is nothing we can do better with more threads. I would rather not schedule a new monitor task if the previous one is still running. I implemented this in #10732.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This makes sense, we should reduce the number of monitor threads.

@jihoonson
Copy link
Copy Markdown
Contributor

jihoonson commented Jan 6, 2021

Hi @leventov, as test coverage is not very great for Druid metrics system, I'm wondering how much the CronScheduler is well-tested in production environment, especially when it runs for a long time period. Could you give me some idea?

@leventov
Copy link
Copy Markdown
Member

leventov commented Jan 7, 2021

Hi @leventov, as test coverage is not very great for Druid metrics system, I'm wondering how much the CronScheduler is well-tested in production environment, especially when it runs for a long time period. Could you give me some idea?

I don't know of any production deployment of CronScheduler. However, I want to note that the fact that metric/monitoring systems are not well-tested is precisely the reason to introduce CronScheduler, which is engineered for the sole purpose to be more stable than standard library executors.

@jihoonson
Copy link
Copy Markdown
Contributor

@leventov yeah, I understand the motivation behind making the CronScheduler. I'm just worried about its maturity because every cluster will be impacted after upgrade if there is any bug in there. I looked at the test coverage of CronScheduler which seems not bad (about 60-65%). I also tested it manually for about an hour, it seems working well. However, I can't be 100% sure there will be no unexpected bugs since I'm not familiar enough with its code. So, I think it would be better to add a feature flag for this just in case (#10732). In my PR, the default still uses CronScheduler, but users can choose the legacy monitor scheduler if they want.

JulianJaffePinterest pushed a commit to JulianJaffePinterest/druid that referenced this pull request Jan 22, 2021
…metrics (apache#10448)

Co-authored-by: Ayush Kulshrestha <ayush.kulshrestha@miqdigital.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

MonitorScheduler is prone to clock drift

5 participants