You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/09/04 11:50:57 UTC

[GitHub] [lucene-solr] sigram opened a new pull request #1831: SOLR-14749 the scheduler part

sigram opened a new pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831


   See PR 1758 for the background on this.


----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r483590300



##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/impl/SolrSchedulerImpl.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.solr.cluster.scheduler.impl;
+
+import java.lang.invoke.MethodHandles;
+import java.time.Instant;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.cloud.ClusterSingleton;
+import org.apache.solr.cluster.scheduler.Schedule;
+import org.apache.solr.cluster.scheduler.Schedulable;
+import org.apache.solr.cluster.scheduler.SolrScheduler;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>Scheduled executions are triggered at most with {@link #SCHEDULE_INTERVAL_SEC} interval.
+ * Each registered {@link Schedulable} is processed sequentially and if its next execution time
+ * is in the past its {@link Schedulable#run()} method will be invoked.
+ * <p>NOTE: If the total time of execution of all registered Schedulable-s exceeds any schedule
+ * interval then exact execution times will be silently missed.</p>
+ */
+public class SolrSchedulerImpl implements SolrScheduler, ClusterSingleton {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int SCHEDULE_INTERVAL_SEC = 10;

Review comment:
       Instead of polling for tasks to run every 10 seconds could we be smart and set the next execution time of the scheduler to when the next job needs to be run? Is it worth the additional investment?




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r483580777



##########
File path: solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.solr.cloud;
+
+/**
+ * Intended for {@link org.apache.solr.core.CoreContainer} plugins that should be
+ * enabled only one instance per cluster.
+ * <p>Components that implement this interface are always in one of two states:
+ * <ul>
+ *   <li>STOPPED - the default state. The component is idle and does not perform

Review comment:
       Should we add `STARTING` and `STOPPING` states? Assuming the call to `start()` waits until the plugin has completed its startup (and similarly the call to `stop()` waiting for it to stop) might be expensive, and as implemented `start()` delays the `Overseer` starting in general.
   
   Possibly waiting for `stop()` to complete makes sense (in order to guarantee no two `ClusterSingleton` plugins are running concurrently on the cluster, so maybe state `STOPPING` is not needed), but I'd think we don't need to wait for `start()` to have completed.




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
noblepaul commented on pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#issuecomment-688645297


   Can we just commit the `ClusterSingleton` as one PR please? `Scheduler` and `ClusterSingleton` doesn't have to go in together


----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r483592144



##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/SolrScheduler.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.solr.cluster.scheduler;
+
+/**
+ *
+ */
+public interface SolrScheduler {
+
+  void registerSchedulable(Schedulable schedulable);

Review comment:
       How is this method going to be called in practice? Do we assume each node (in the SolrCloud cluster) will register all tasks locally (those that are not `ClusterSingleton`) and that multiple instances are going to run in parallel? Or will a `ClusterSingleton` task be registering other scheduled tasks that as a consequence will only have a single instance running on the 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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r483581489



##########
File path: solr/core/src/java/org/apache/solr/cloud/Overseer.java
##########
@@ -775,6 +779,42 @@ private void doCompatCheck(BiConsumer<String, Object> consumer) {
     }
   }
 
+  /**
+   * Start {@link ClusterSingleton} plugins when we become the leader.
+   */
+  private void startClusterSingletons() {
+    PluginBag<SolrRequestHandler> handlers = getCoreContainer().getRequestHandlers();
+    if (handlers == null) {
+      return;
+    }
+    handlers.keySet().forEach(handlerName -> {
+      SolrRequestHandler handler = handlers.get(handlerName);
+      if (handler instanceof ClusterSingleton) {
+        try {
+          ((ClusterSingleton) handler).start();
+        } catch (Exception e) {
+          log.warn("Exception starting ClusterSingleton " + handler, e);
+        }
+      }
+    });
+  }
+
+  /**
+   * Stop {@link ClusterSingleton} plugins when we lose leadership.
+   */
+  private void stopClusterSingletons() {
+    PluginBag<SolrRequestHandler> handlers = getCoreContainer().getRequestHandlers();

Review comment:
       Should we stop currently configured `ClusterSingleton` handlers or rather stop all those we've started? Is the configuration immutable?




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r483583874



##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/impl/CompiledSchedule.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.cluster.scheduler.impl;
+
+import java.lang.invoke.MethodHandles;
+import java.text.ParseException;
+import java.time.Instant;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.temporal.ChronoField;
+import java.util.Date;
+import java.util.Locale;
+import java.util.TimeZone;
+
+import org.apache.solr.cluster.scheduler.Schedule;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DateMathParser;
+import org.apache.solr.util.TimeZoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A version of {@link Schedule} where some of the fields are already resolved.
+ */
+class CompiledSchedule {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  final String name;
+  final TimeZone timeZone;
+  final Instant startTime;
+  final String interval;
+  final DateMathParser dateMathParser;
+
+  Instant lastRunAt;
+
+  /**
+   * Compile a schedule.
+   * @param schedule schedule.
+   * @throws Exception if startTime or interval cannot be parsed.
+   */
+  CompiledSchedule(Schedule schedule) throws Exception {
+    this.name = schedule.getName();
+    this.timeZone = TimeZoneUtils.getTimeZone(schedule.getTimeZone());
+    this.startTime = parseStartTime(new Date(), schedule.getStartTime(), timeZone);
+    this.lastRunAt = startTime;
+    this.interval = schedule.getInterval();
+    this.dateMathParser = new DateMathParser(timeZone);
+    // this is just to verify that the interval math is valid
+    shouldRun();
+  }
+
+  private Instant parseStartTime(Date now, String startTimeStr, TimeZone timeZone) throws Exception {
+    try {
+      // try parsing startTime as an ISO-8601 date time string
+      return DateMathParser.parseMath(now, startTimeStr).toInstant();
+    } catch (SolrException e) {
+      if (e.code() != SolrException.ErrorCode.BAD_REQUEST.code) {
+        throw new Exception("startTime: error parsing value '" + startTimeStr + "': " + e.toString());
+      }
+    }
+    DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
+        .append(DateTimeFormatter.ISO_LOCAL_DATE).appendPattern("['T'[HH[:mm[:ss]]]]")
+        .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
+        .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
+        .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
+        .toFormatter(Locale.ROOT).withZone(timeZone.toZoneId());
+    try {
+      return Instant.from(dateTimeFormatter.parse(startTimeStr));
+    } catch (Exception e) {
+      throw new Exception("startTime: error parsing startTime '" + startTimeStr + "': " + e.toString());
+    }
+  }
+
+  /**
+   * Returns true if the last run + run interval is already in the past.
+   */
+  boolean shouldRun() {
+    dateMathParser.setNow(new Date(lastRunAt.toEpochMilli()));
+    Instant nextRunTime;
+    try {
+      Date next = dateMathParser.parseMath(interval);
+      nextRunTime = next.toInstant();
+    } catch (ParseException e) {
+      log.warn("Invalid math expression, skipping: " + e);
+      return false;
+    }
+    if (Instant.now().isAfter(nextRunTime)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * This setter MUST be invoked after each run.
+   * @param lastRunAt time when the schedule was last run.
+   */
+  void setLastRunAt(Instant lastRunAt) {

Review comment:
       Unclear if `lastRunAt` is the time the schedule last started or last completed. If completed, can we simplify by not passing an instant and building it here?




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r483584416



##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/impl/CompiledSchedule.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.cluster.scheduler.impl;
+
+import java.lang.invoke.MethodHandles;
+import java.text.ParseException;
+import java.time.Instant;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.temporal.ChronoField;
+import java.util.Date;
+import java.util.Locale;
+import java.util.TimeZone;
+
+import org.apache.solr.cluster.scheduler.Schedule;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DateMathParser;
+import org.apache.solr.util.TimeZoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A version of {@link Schedule} where some of the fields are already resolved.

Review comment:
       Given this class does not implement `Schedule` (didn't get yet to the point where it's used) maybe its name or this comment should be clarified?




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] sigram commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r485748887



##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/Schedulable.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.cluster.scheduler;
+
+/**
+ * Component to be scheduled and executed according to the schedule.
+ */
+public interface Schedulable {
+
+  Schedule getSchedule();
+
+  /**
+   * Execute the component.
+   * <p>NOTE: this should be a lightweight method that executes quickly, to avoid blocking the
+   * execution of other schedules. If it requires more work it should do this in a separate thread.</p>

Review comment:
       Right. I don't want to bake this level of details into the API - I'll delete the second part of this javadoc.




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] sigram commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r485761720



##########
File path: solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.solr.cloud;
+
+/**
+ * Intended for {@link org.apache.solr.core.CoreContainer} plugins that should be
+ * enabled only one instance per cluster.
+ * <p>Components that implement this interface are always in one of two states:
+ * <ul>
+ *   <li>STOPPED - the default state. The component is idle and does not perform

Review comment:
       I didn't want to complicate the API to include support for more complex state and async operations. Besides, there's only one parent that is supposed to call the `start()` method, and no other actor is supposed to do that, so from the POV of other components the singleton is either running or not, and the STARTING state is beyond their control anyway - from their POV the singleton is in "not running" state during startup.

##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/impl/CompiledSchedule.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.cluster.scheduler.impl;
+
+import java.lang.invoke.MethodHandles;
+import java.text.ParseException;
+import java.time.Instant;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.temporal.ChronoField;
+import java.util.Date;
+import java.util.Locale;
+import java.util.TimeZone;
+
+import org.apache.solr.cluster.scheduler.Schedule;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DateMathParser;
+import org.apache.solr.util.TimeZoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A version of {@link Schedule} where some of the fields are already resolved.
+ */
+class CompiledSchedule {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  final String name;
+  final TimeZone timeZone;
+  final Instant startTime;
+  final String interval;
+  final DateMathParser dateMathParser;
+
+  Instant lastRunAt;
+
+  /**
+   * Compile a schedule.
+   * @param schedule schedule.
+   * @throws Exception if startTime or interval cannot be parsed.
+   */
+  CompiledSchedule(Schedule schedule) throws Exception {
+    this.name = schedule.getName();
+    this.timeZone = TimeZoneUtils.getTimeZone(schedule.getTimeZone());
+    this.startTime = parseStartTime(new Date(), schedule.getStartTime(), timeZone);
+    this.lastRunAt = startTime;
+    this.interval = schedule.getInterval();
+    this.dateMathParser = new DateMathParser(timeZone);
+    // this is just to verify that the interval math is valid
+    shouldRun();
+  }
+
+  private Instant parseStartTime(Date now, String startTimeStr, TimeZone timeZone) throws Exception {
+    try {
+      // try parsing startTime as an ISO-8601 date time string
+      return DateMathParser.parseMath(now, startTimeStr).toInstant();
+    } catch (SolrException e) {
+      if (e.code() != SolrException.ErrorCode.BAD_REQUEST.code) {
+        throw new Exception("startTime: error parsing value '" + startTimeStr + "': " + e.toString());
+      }
+    }
+    DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
+        .append(DateTimeFormatter.ISO_LOCAL_DATE).appendPattern("['T'[HH[:mm[:ss]]]]")
+        .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
+        .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
+        .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
+        .toFormatter(Locale.ROOT).withZone(timeZone.toZoneId());
+    try {
+      return Instant.from(dateTimeFormatter.parse(startTimeStr));
+    } catch (Exception e) {
+      throw new Exception("startTime: error parsing startTime '" + startTimeStr + "': " + e.toString());
+    }
+  }
+
+  /**
+   * Returns true if the last run + run interval is already in the past.
+   */
+  boolean shouldRun() {
+    dateMathParser.setNow(new Date(lastRunAt.toEpochMilli()));
+    Instant nextRunTime;
+    try {
+      Date next = dateMathParser.parseMath(interval);
+      nextRunTime = next.toInstant();
+    } catch (ParseException e) {
+      log.warn("Invalid math expression, skipping: " + e);
+      return false;
+    }
+    if (Instant.now().isAfter(nextRunTime)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * This setter MUST be invoked after each run.
+   * @param lastRunAt time when the schedule was last run.
+   */
+  void setLastRunAt(Instant lastRunAt) {

Review comment:
       Currently it's set at the start of execution. This reduces the schedule drift due to execution times.

##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/Schedulable.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.cluster.scheduler;
+
+/**
+ * Component to be scheduled and executed according to the schedule.
+ */
+public interface Schedulable {
+
+  Schedule getSchedule();
+
+  /**
+   * Execute the component.
+   * <p>NOTE: this should be a lightweight method that executes quickly, to avoid blocking the
+   * execution of other schedules. If it requires more work it should do this in a separate thread.</p>

Review comment:
       As for the schedule being skewed ... yeah, we're not building a Quartz replacement here :) The javadoc clearly states that scheduled runs may be missed if execution of schedulables takes too long.
   
   Having said that, we should probably add some protection in the implementation to avoid situations where one buggy `Schedulable` starves all others.

##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/impl/CompiledSchedule.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.cluster.scheduler.impl;
+
+import java.lang.invoke.MethodHandles;
+import java.text.ParseException;
+import java.time.Instant;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.temporal.ChronoField;
+import java.util.Date;
+import java.util.Locale;
+import java.util.TimeZone;
+
+import org.apache.solr.cluster.scheduler.Schedule;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DateMathParser;
+import org.apache.solr.util.TimeZoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A version of {@link Schedule} where some of the fields are already resolved.

Review comment:
       This is an implementation detail. I can add more javadocs, sure, but this class is package-private.

##########
File path: solr/core/src/java/org/apache/solr/cloud/Overseer.java
##########
@@ -775,6 +779,42 @@ private void doCompatCheck(BiConsumer<String, Object> consumer) {
     }
   }
 
+  /**
+   * Start {@link ClusterSingleton} plugins when we become the leader.
+   */
+  private void startClusterSingletons() {
+    PluginBag<SolrRequestHandler> handlers = getCoreContainer().getRequestHandlers();
+    if (handlers == null) {
+      return;
+    }
+    handlers.keySet().forEach(handlerName -> {
+      SolrRequestHandler handler = handlers.get(handlerName);
+      if (handler instanceof ClusterSingleton) {
+        try {
+          ((ClusterSingleton) handler).start();
+        } catch (Exception e) {
+          log.warn("Exception starting ClusterSingleton " + handler, e);
+        }
+      }
+    });
+  }
+
+  /**
+   * Stop {@link ClusterSingleton} plugins when we lose leadership.
+   */
+  private void stopClusterSingletons() {
+    PluginBag<SolrRequestHandler> handlers = getCoreContainer().getRequestHandlers();

Review comment:
       I don't think it matters what was the configuration - the contract for `ClusterSingleton` requires that we stop all of them.

##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/SolrScheduler.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.solr.cluster.scheduler;
+
+/**
+ *
+ */
+public interface SolrScheduler {
+
+  void registerSchedulable(Schedulable schedulable);

Review comment:
       Huh, it looks like I lost some part of this PR during the split ... :) `SolrSchedulerImpl` is a `ClusterSingleton`, at some point I had a section in `CoreContainer` that would instantiate it and add to the list of singletons to be managed. And another section that would scan all registered handlers to see if any one of them was `Schedulable`.
   
   (A side-note: I feel like we keep adding these small hacks just to compensate for the lack of a proper dependency-injection framework... sigh..)

##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/impl/SolrSchedulerImpl.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.solr.cluster.scheduler.impl;
+
+import java.lang.invoke.MethodHandles;
+import java.time.Instant;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.cloud.ClusterSingleton;
+import org.apache.solr.cluster.scheduler.Schedule;
+import org.apache.solr.cluster.scheduler.Schedulable;
+import org.apache.solr.cluster.scheduler.SolrScheduler;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>Scheduled executions are triggered at most with {@link #SCHEDULE_INTERVAL_SEC} interval.
+ * Each registered {@link Schedulable} is processed sequentially and if its next execution time
+ * is in the past its {@link Schedulable#run()} method will be invoked.
+ * <p>NOTE: If the total time of execution of all registered Schedulable-s exceeds any schedule
+ * interval then exact execution times will be silently missed.</p>
+ */
+public class SolrSchedulerImpl implements SolrScheduler, ClusterSingleton {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int SCHEDULE_INTERVAL_SEC = 10;

Review comment:
       Not sure. This implementation is very simple on purpose, we can always complicate things later ;)




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] sigram commented on pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
sigram commented on pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#issuecomment-689671370


   @noblepaul in this PR the Scheduler depends on ClusterSingleton, that's why I imported it into this PR. Whatever the ClusterSingleton becomes it will be the version from PR-1758.


----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1831: SOLR-14749 the scheduler part

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1831:
URL: https://github.com/apache/lucene-solr/pull/1831#discussion_r483593482



##########
File path: solr/core/src/java/org/apache/solr/cluster/scheduler/Schedulable.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.solr.cluster.scheduler;
+
+/**
+ * Component to be scheduled and executed according to the schedule.
+ */
+public interface Schedulable {
+
+  Schedule getSchedule();
+
+  /**
+   * Execute the component.
+   * <p>NOTE: this should be a lightweight method that executes quickly, to avoid blocking the
+   * execution of other schedules. If it requires more work it should do this in a separate thread.</p>

Review comment:
       If a scheduled component starts a new thread to do its work, the schedule is going to get skewed pretty quickly and we might have multiple "copies" of the scheduled component being started in parallel. We'd be delegating the responsibility of insuring a single executing instance of the component (on a given node where it was registered) to the component itself.




----------------------------------------------------------------
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: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org