You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/01/07 02:28:33 UTC

[GitHub] [druid] jihoonson opened a new pull request #10732: Add a config for monitorScheduler type

jihoonson opened a new pull request #10732:
URL: https://github.com/apache/druid/pull/10732


   https://github.com/apache/druid/pull/10448 modified the `MonitorScheduler` to use `CronScheduler` instead of `ScheduledExecutorService`. This change looks good to me except that I'm not sure how well-tested `CronScheduler` is. This PR adds the previous `ScheduledExecutorService`-based `MonitorScheduler` back, and a new config, `druid.monitoring.schedulerClassName`, to determine what type of `MonitorScheduler` to use. This PR doesn't change the default `MonitorScheduler` as I find `CronScheduler` has a reasonable test coverage. However, if there is any unknown bug there, users can still use the old monitorScheduler. The new config is intentionally not documented as no one is supposed to touch it. However, it should be called out in the release notes.
   
   This PR additionally fixes 3 bugs in `MonitorScheduler`.
   - When an exception is thrown in `monitor.monitor()`, the behaviour has changed unexpectedly to stop the monitor. The monitor will ignore exceptions and continue working after this PR as it used to do.
   - There is a race condition between [when a scheduledFuture is set in a monitor](https://github.com/apache/druid/pull/10448/files#diff-e1da51fa67513b22b8110a76b29aa3c5cc8d00de418d7e5afc498c3bc3a8f107R178) and [when the scheduledFuture is used](https://github.com/apache/druid/pull/10448/files#diff-e1da51fa67513b22b8110a76b29aa3c5cc8d00de418d7e5afc498c3bc3a8f107R153). This will not likely happen in production since the first cronTask will be executed after the emitter period, but is possible in theory.
   - https://github.com/apache/druid/pull/10448 changed to use 64 threads for monitoring which seems an overkill to me. This PR changed it back to use a single thread.
   
   <hr>
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [x] added documentation for new or modified features or behaviors.
   - [x] 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 comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r556053575



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       My 2ยข: the best plan is to default to the old one, and then in the future (after some people have enabled the new one in production) we should switch to the new one, and remove the old one and remove the config entirely.
   
   Rationale:
   
   The new scheduler is designed to eliminate potential clock drift for monitors. This reward is real but is pretty small impact. I don't expect anything bad will happen if the schedule drifts a bit. The main risk of the new scheduler, I suppose, is that there's some case where it goes haywire, and either locks up completely or fires much more often than it should. I'm not sure how likely this is, but it's (a) hard to test for, (b) quite bad if it happens.
   
   So, because the potential reward has a small impact, and the potential risk has a large impact, I think it's best to default to the old scheduler for another release or so. Just until such time as people have been able to do long-running tests in production and have found that there are no issues.
   
   At any rate, it's good that this is undocumented, since it's an inside-baseball sort of config that we would only want to exist for a few releases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r556068500



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       I've also thought about this a bunch and have changed my opinion on whether or not we should change the scheduler to a new dependency by default a few times.
   
   While changing to use the CronScheduler might fix a bug, it isn't clear whether any users have run into this in the field. I thought about documenting why a user would want to change the scheduler to CronScheduler instead of the older implementation, and I couldn't think of a good user facing reason to do so. So if we set the default to the old implementation, I don't think anyone would test it in production, so it would continue to live as dead code, and we'll have the same dilemma in the next release or 2 when we ask whether or not this has been run in production.
   
   Setting the default to the older implementation reduces the impact of any bug that might show up in long running tests (even though this library was specifically built to fix issues found with long running processes). The drawback here is finding a reason for some users to try this in production so that we can sunset the feature flag in a release or 2.
   
   Writing out this comment, I now think the more cautious approach - keeping the default the same - is better as it's hard to articulate the benefit for switching the scheduling and taking on the risk associated with changing the older behavior.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson merged pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson merged pull request #10732:
URL: https://github.com/apache/druid/pull/10732


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r556211781



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       Changed the default back to `BasicMonitorScheduler`. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r555599991



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       Yes, I have done some testing before and will do more. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r556211566



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       > So, because the potential reward has a small impact, and the potential risk has a large impact, I think it's best to default to the old scheduler for another release or so. Just until such time as people have been able to do long-running tests in production and have found that there are no issues.
   
   This makes sense to me. I think we can do more extensive testing by ourselves instead of rushing to change the default.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r555262086



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       It was changed in https://github.com/apache/druid/pull/10448, not in this PR. This PR is just to make it configurable because I'm not sure how stable it is. As noted in https://github.com/apache/druid/pull/10448#issuecomment-756367474, CronScheduler seems to have a not-bad test coverage and worked well in my testing.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r555600385



##########
File path: core/src/main/java/org/apache/druid/java/util/metrics/ClockDriftSafeMonitorScheduler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.metrics;
+
+import io.timeandspace.cronscheduler.CronScheduler;
+import io.timeandspace.cronscheduler.CronTask;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link MonitorScheduler} implementation based on {@link CronScheduler}.
+ */
+public class ClockDriftSafeMonitorScheduler extends MonitorScheduler
+{
+  private static final Logger LOG = new Logger(ClockDriftSafeMonitorScheduler.class);
+
+  private final CronScheduler monitorScheduler;
+  private final ExecutorService monitorRunner;
+
+  public ClockDriftSafeMonitorScheduler(
+      MonitorSchedulerConfig config,
+      ServiceEmitter emitter,
+      List<Monitor> monitors,
+      CronScheduler monitorScheduler,
+      ExecutorService monitorRunner
+  )
+  {
+    super(config, emitter, monitors);
+    this.monitorScheduler = monitorScheduler;
+    this.monitorRunner = monitorRunner;
+  }
+
+  @Override
+  void startMonitor(final Monitor monitor)
+  {
+    monitor.start();
+    long rate = getConfig().getEmitterPeriod().getMillis();
+    final AtomicReference<Future<?>> scheduleFutureReference = new AtomicReference<>();
+    Future<?> scheduledFuture = monitorScheduler.scheduleAtFixedRate(
+        rate,
+        rate,
+        TimeUnit.MILLISECONDS,
+        new CronTask()
+        {
+          private Future<?> scheduleFuture = null;

Review comment:
       Ah, forgot to clean them up before commit. Thanks :+1: 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] pjain1 commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
pjain1 commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r555313966



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       I guess it will get tested in the next RC then.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] pjain1 commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
pjain1 commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r555315094



##########
File path: core/src/main/java/org/apache/druid/java/util/metrics/ClockDriftSafeMonitorScheduler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.metrics;
+
+import io.timeandspace.cronscheduler.CronScheduler;
+import io.timeandspace.cronscheduler.CronTask;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link MonitorScheduler} implementation based on {@link CronScheduler}.
+ */
+public class ClockDriftSafeMonitorScheduler extends MonitorScheduler
+{
+  private static final Logger LOG = new Logger(ClockDriftSafeMonitorScheduler.class);
+
+  private final CronScheduler monitorScheduler;
+  private final ExecutorService monitorRunner;
+
+  public ClockDriftSafeMonitorScheduler(
+      MonitorSchedulerConfig config,
+      ServiceEmitter emitter,
+      List<Monitor> monitors,
+      CronScheduler monitorScheduler,
+      ExecutorService monitorRunner
+  )
+  {
+    super(config, emitter, monitors);
+    this.monitorScheduler = monitorScheduler;
+    this.monitorRunner = monitorRunner;
+  }
+
+  @Override
+  void startMonitor(final Monitor monitor)
+  {
+    monitor.start();
+    long rate = getConfig().getEmitterPeriod().getMillis();
+    final AtomicReference<Future<?>> scheduleFutureReference = new AtomicReference<>();
+    Future<?> scheduledFuture = monitorScheduler.scheduleAtFixedRate(
+        rate,
+        rate,
+        TimeUnit.MILLISECONDS,
+        new CronTask()
+        {
+          private Future<?> scheduleFuture = null;
+          private Future<Boolean> monitorFuture = null;
+
+          @Override
+          public void run(long scheduledRunTimeMillis)
+          {
+            waitForScheduleFutureToBeSet();

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r555503062



##########
File path: core/src/main/java/org/apache/druid/java/util/metrics/ClockDriftSafeMonitorScheduler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.metrics;
+
+import io.timeandspace.cronscheduler.CronScheduler;
+import io.timeandspace.cronscheduler.CronTask;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link MonitorScheduler} implementation based on {@link CronScheduler}.
+ */
+public class ClockDriftSafeMonitorScheduler extends MonitorScheduler
+{
+  private static final Logger LOG = new Logger(ClockDriftSafeMonitorScheduler.class);
+
+  private final CronScheduler monitorScheduler;
+  private final ExecutorService monitorRunner;
+
+  public ClockDriftSafeMonitorScheduler(
+      MonitorSchedulerConfig config,
+      ServiceEmitter emitter,
+      List<Monitor> monitors,
+      CronScheduler monitorScheduler,
+      ExecutorService monitorRunner
+  )
+  {
+    super(config, emitter, monitors);
+    this.monitorScheduler = monitorScheduler;
+    this.monitorRunner = monitorRunner;
+  }
+
+  @Override
+  void startMonitor(final Monitor monitor)
+  {
+    monitor.start();
+    long rate = getConfig().getEmitterPeriod().getMillis();
+    final AtomicReference<Future<?>> scheduleFutureReference = new AtomicReference<>();
+    Future<?> scheduledFuture = monitorScheduler.scheduleAtFixedRate(
+        rate,
+        rate,
+        TimeUnit.MILLISECONDS,
+        new CronTask()
+        {
+          private Future<?> scheduleFuture = null;

Review comment:
       nit: I wonder if these can have better names: `scheduledFuture`, `scheduleFuture`, and `scheduleFutureReference` is a bit too close to each other and is sort of confusing at first glance. Should the inner one perhaps be called `cancellationFuture` or something to distinguish it from the external one?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r556054132



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       By the way, if anyone has been running the patch in production for a while already, now would be a good time to speak up. If we have already built up a good amount of confidence then I think it makes sense to default to the new one.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #10732:
URL: https://github.com/apache/druid/pull/10732#issuecomment-755862034


   This pull request **introduces 1 alert** when merging 6b791d120362fdcfd14e7b1f10d30e941a52a373 into 48e576a307da0efba1cb036c274327733658c1d6 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-ba61b23623936b01c7a11557c2a97abbdc2118cf)
   
   **new alerts:**
   
   * 1 for Dereferenced variable may be null


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] pjain1 commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
pjain1 commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r554518647



##########
File path: server/src/main/java/org/apache/druid/server/metrics/DruidMonitorSchedulerConfig.java
##########
@@ -28,9 +29,17 @@
  */
 public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig
 {
+  @JsonProperty
+  private String schedulerClassName = ClockDriftSafeMonitorScheduler.class.getName();

Review comment:
       this changes the default to this monitor schedule type, is this ok/tested ?

##########
File path: core/src/main/java/org/apache/druid/java/util/metrics/ClockDriftSafeMonitorScheduler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.metrics;
+
+import io.timeandspace.cronscheduler.CronScheduler;
+import io.timeandspace.cronscheduler.CronTask;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link MonitorScheduler} implementation based on {@link CronScheduler}.
+ */
+public class ClockDriftSafeMonitorScheduler extends MonitorScheduler
+{
+  private static final Logger LOG = new Logger(ClockDriftSafeMonitorScheduler.class);
+
+  private final CronScheduler monitorScheduler;
+  private final ExecutorService monitorRunner;
+
+  public ClockDriftSafeMonitorScheduler(
+      MonitorSchedulerConfig config,
+      ServiceEmitter emitter,
+      List<Monitor> monitors,
+      CronScheduler monitorScheduler,
+      ExecutorService monitorRunner
+  )
+  {
+    super(config, emitter, monitors);
+    this.monitorScheduler = monitorScheduler;
+    this.monitorRunner = monitorRunner;
+  }
+
+  @Override
+  void startMonitor(final Monitor monitor)
+  {
+    monitor.start();
+    long rate = getConfig().getEmitterPeriod().getMillis();
+    final AtomicReference<Future<?>> scheduleFutureReference = new AtomicReference<>();
+    Future<?> scheduledFuture = monitorScheduler.scheduleAtFixedRate(
+        rate,
+        rate,
+        TimeUnit.MILLISECONDS,
+        new CronTask()
+        {
+          private Future<?> scheduleFuture = null;
+          private Future<Boolean> monitorFuture = null;
+
+          @Override
+          public void run(long scheduledRunTimeMillis)
+          {
+            waitForScheduleFutureToBeSet();

Review comment:
       why not just use a CountDownLatch instead of continuously checking in continuous loop that counts down after `scheduleFutureReference`  is set




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson commented on pull request #10732:
URL: https://github.com/apache/druid/pull/10732#issuecomment-759861714


   @clintropolis @pjain1 @suneet-s @gianm thanks for the review.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10732: Add a config for monitorScheduler type

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10732:
URL: https://github.com/apache/druid/pull/10732#discussion_r555262535



##########
File path: core/src/main/java/org/apache/druid/java/util/metrics/ClockDriftSafeMonitorScheduler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.metrics;
+
+import io.timeandspace.cronscheduler.CronScheduler;
+import io.timeandspace.cronscheduler.CronTask;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link MonitorScheduler} implementation based on {@link CronScheduler}.
+ */
+public class ClockDriftSafeMonitorScheduler extends MonitorScheduler
+{
+  private static final Logger LOG = new Logger(ClockDriftSafeMonitorScheduler.class);
+
+  private final CronScheduler monitorScheduler;
+  private final ExecutorService monitorRunner;
+
+  public ClockDriftSafeMonitorScheduler(
+      MonitorSchedulerConfig config,
+      ServiceEmitter emitter,
+      List<Monitor> monitors,
+      CronScheduler monitorScheduler,
+      ExecutorService monitorRunner
+  )
+  {
+    super(config, emitter, monitors);
+    this.monitorScheduler = monitorScheduler;
+    this.monitorRunner = monitorRunner;
+  }
+
+  @Override
+  void startMonitor(final Monitor monitor)
+  {
+    monitor.start();
+    long rate = getConfig().getEmitterPeriod().getMillis();
+    final AtomicReference<Future<?>> scheduleFutureReference = new AtomicReference<>();
+    Future<?> scheduledFuture = monitorScheduler.scheduleAtFixedRate(
+        rate,
+        rate,
+        TimeUnit.MILLISECONDS,
+        new CronTask()
+        {
+          private Future<?> scheduleFuture = null;
+          private Future<Boolean> monitorFuture = null;
+
+          @Override
+          public void run(long scheduledRunTimeMillis)
+          {
+            waitForScheduleFutureToBeSet();

Review comment:
       We can use it, but it seems not matter much to me since this loop is not supposed to run at all in production.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org